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 0001/1170] 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 0002/1170] 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 0003/1170] 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 0004/1170] 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 0005/1170] 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 0006/1170] 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 0007/1170] 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 0008/1170] 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 0009/1170] 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 0010/1170] 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 0011/1170] 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 0012/1170] 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 0013/1170] 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 0014/1170] 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 0015/1170] 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 0016/1170] 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 0017/1170] 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 0018/1170] 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 0019/1170] 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 0020/1170] 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 0021/1170] 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 0022/1170] 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 0023/1170] 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 0024/1170] 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 0025/1170] 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 0026/1170] 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 0027/1170] 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 0028/1170] 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 0029/1170] 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 0030/1170] 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 0031/1170] 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 0032/1170] 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 0033/1170] 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 0034/1170] 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 0035/1170] 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 0036/1170] 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 0037/1170] 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 0038/1170] 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 0039/1170] 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 852318449f2ab8e0bfaeca9904b90936d751d177 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 20:17:43 +0200 Subject: [PATCH 0040/1170] Check perf tests --- src/Common/PODArray_fwd.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PODArray_fwd.h b/src/Common/PODArray_fwd.h index d570a90e467..bd780eb51b0 100644 --- a/src/Common/PODArray_fwd.h +++ b/src/Common/PODArray_fwd.h @@ -17,12 +17,12 @@ constexpr size_t integerRoundUp(size_t value, size_t dividend) } template , size_t pad_right_ = 0, + typename TAllocator = Allocator, size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray; /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ -template > +template > using PaddedPODArray = PODArray; /** A helper for declaring PODArray that uses inline memory. @@ -32,6 +32,6 @@ using PaddedPODArray = PODArray using PODArrayWithStackMemory = PODArray, rounded_bytes, alignof(T)>>; + AllocatorWithStackMemory, rounded_bytes, alignof(T)>>; } From 7568de2202a7fa99539d8d46092315bf3c7fe5e6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 22:03:53 +0100 Subject: [PATCH 0041/1170] Revert "Check perf tests" This reverts commit 852318449f2ab8e0bfaeca9904b90936d751d177. --- src/Common/PODArray_fwd.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PODArray_fwd.h b/src/Common/PODArray_fwd.h index bd780eb51b0..d570a90e467 100644 --- a/src/Common/PODArray_fwd.h +++ b/src/Common/PODArray_fwd.h @@ -17,12 +17,12 @@ constexpr size_t integerRoundUp(size_t value, size_t dividend) } template , size_t pad_right_ = 0, + typename TAllocator = Allocator, size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray; /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ -template > +template > using PaddedPODArray = PODArray; /** A helper for declaring PODArray that uses inline memory. @@ -32,6 +32,6 @@ using PaddedPODArray = PODArray using PODArrayWithStackMemory = PODArray, rounded_bytes, alignof(T)>>; + AllocatorWithStackMemory, rounded_bytes, alignof(T)>>; } From 57d036e5899b52fd4fdab9447630e01bf3d5382b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 15 Jun 2024 22:04:24 +0100 Subject: [PATCH 0042/1170] impl --- tests/performance/scripts/perf.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index e98c158249a..94f145d82db 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -427,6 +427,8 @@ for query_index in queries_to_run: for conn_index, c in enumerate(this_query_connections): try: + c.execute("SYSTEM JEMALLOC PURGE") + res = c.execute( q, query_id=run_id, 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 0043/1170] 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 0044/1170] 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 0045/1170] 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 0046/1170] 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 0047/1170] 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 0048/1170] 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 0049/1170] 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 0050/1170] 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 38537a00aa9e7185b69c066cd6809c54487ecf4e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 18:16:12 +0000 Subject: [PATCH 0051/1170] Make ActionsDAGPtr unique_ptr. --- src/Core/InterpolateDescription.cpp | 2 +- src/Core/InterpolateDescription.h | 2 +- src/Functions/indexHint.h | 4 +- src/Interpreters/ActionsDAG.cpp | 46 ++++++++------- src/Interpreters/ActionsDAG.h | 11 ++-- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ActionsVisitor.h | 2 +- src/Interpreters/ExpressionActions.cpp | 20 ++++++- src/Interpreters/ExpressionActions.h | 6 +- src/Interpreters/ExpressionAnalyzer.cpp | 27 ++++----- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 14 ++--- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/TableJoin.cpp | 4 +- src/Interpreters/WindowDescription.h | 6 +- src/Interpreters/addMissingDefaults.cpp | 2 +- src/Interpreters/addMissingDefaults.h | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 2 +- src/Interpreters/inplaceBlockConversions.h | 2 +- src/Planner/CollectTableExpressionData.cpp | 6 +- src/Planner/Planner.cpp | 49 ++++++++-------- src/Planner/PlannerActionsVisitor.cpp | 6 +- src/Planner/PlannerJoinTree.cpp | 58 ++++++++++--------- src/Planner/PlannerJoinTree.h | 2 +- src/Planner/PlannerJoins.cpp | 12 ++-- src/Planner/PlannerWindowFunctions.cpp | 13 ++++- src/Planner/PlannerWindowFunctions.h | 2 +- src/Planner/Utils.cpp | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 4 +- src/Processors/QueryPlan/CubeStep.cpp | 4 +- src/Processors/QueryPlan/ExpressionStep.cpp | 12 ++-- src/Processors/QueryPlan/ExpressionStep.h | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 11 ++-- src/Processors/QueryPlan/FilterStep.h | 2 +- .../Optimizations/distinctReadInOrder.cpp | 12 ++-- .../Optimizations/filterPushDown.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizePrimaryKeyCondition.cpp | 6 +- .../Optimizations/optimizeReadInOrder.cpp | 4 +- .../optimizeUseAggregateProjection.cpp | 4 +- .../optimizeUseNormalProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 4 +- .../Optimizations/projectionsCommon.h | 2 +- .../Optimizations/removeRedundantDistinct.cpp | 23 ++++---- .../QueryPlan/ReadFromMergeTree.cpp | 38 ++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- .../QueryPlan/SourceStepWithFilter.cpp | 8 +-- .../QueryPlan/SourceStepWithFilter.h | 3 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 8 +-- src/Processors/QueryPlan/TotalsHavingStep.h | 2 +- src/Processors/QueryPlan/WindowStep.h | 2 +- src/Processors/SourceWithKeyCondition.h | 4 +- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 8 +-- src/Storages/KeyDescription.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 4 +- src/Storages/MergeTree/KeyCondition.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 +-- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +-- .../MergeTree/MergeTreeDataSelectExecutor.h | 4 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 4 +- .../MergeTree/MergeTreeIndexBloomFilter.h | 4 +- .../MergeTreeIndexBloomFilterText.cpp | 4 +- .../MergeTree/MergeTreeIndexBloomFilterText.h | 4 +- .../MergeTree/MergeTreeIndexFullText.cpp | 4 +- .../MergeTree/MergeTreeIndexFullText.h | 4 +- .../MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- .../MergeTree/MergeTreeIndexHypothesis.h | 2 +- .../MergeTree/MergeTreeIndexMinMax.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndexMinMax.h | 4 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 6 +- src/Storages/MergeTree/MergeTreeIndexSet.h | 4 +- .../MergeTree/MergeTreeIndexUSearch.cpp | 2 +- .../MergeTree/MergeTreeIndexUSearch.h | 2 +- src/Storages/MergeTree/MergeTreeIndices.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeSequentialSource.cpp | 4 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 32 +++++----- src/Storages/MergeTree/PartitionPruner.cpp | 2 +- src/Storages/MergeTree/PartitionPruner.h | 2 +- .../StorageObjectStorageSource.cpp | 2 +- src/Storages/SelectQueryInfo.h | 4 +- src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageMerge.cpp | 10 ++-- src/Storages/StorageURL.h | 2 +- src/Storages/StorageValues.cpp | 4 +- src/Storages/StorageView.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 4 +- src/Storages/VirtualColumnUtils.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 12 ++-- 98 files changed, 355 insertions(+), 318 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index d828c2e85e9..76bbefdcfd7 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -14,7 +14,7 @@ namespace DB { InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) - : actions(actions_) + : actions(std::move(actions_)) { for (const auto & name_type : actions->getRequiredColumns()) { diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 62d7120508b..73579aebee4 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -11,7 +11,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; using Aliases = std::unordered_map; /// Interpolate description diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 3b71c7a5585..8fd7b751760 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -2,14 +2,12 @@ #include #include #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; - /** The `indexHint` function takes any number of any arguments and always returns one. * * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 34f3e0a98bd..23e1e5ee152 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -626,7 +626,7 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { - auto actions = std::make_shared(); + auto actions = std::make_unique(); std::unordered_map copy_map; struct Frame @@ -1248,25 +1248,29 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) ActionsDAGPtr ActionsDAG::clone() const { - auto actions = std::make_shared(); + std::unordered_map old_to_new_nodes; + return clone(old_to_new_nodes); +} - std::unordered_map copy_map; +ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const +{ + auto actions = std::make_unique(); for (const auto & node : nodes) { auto & copy_node = actions->nodes.emplace_back(node); - copy_map[&node] = ©_node; + old_to_new_nodes[&node] = ©_node; } for (auto & node : actions->nodes) for (auto & child : node.children) - child = copy_map[child]; + child = old_to_new_nodes[child]; for (const auto & output_node : outputs) - actions->outputs.push_back(copy_map[output_node]); + actions->outputs.push_back(old_to_new_nodes[output_node]); for (const auto & input_node : inputs) - actions->inputs.push_back(copy_map[input_node]); + actions->inputs.push_back(old_to_new_nodes[input_node]); return actions; } @@ -1421,7 +1425,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (add_casted_columns && mode != MatchColumnsMode::Name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); - auto actions_dag = std::make_shared(source); + auto actions_dag = std::make_unique(source); NodeRawConstPtrs projection(num_result_columns); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1549,7 +1553,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) { - auto adding_column_action = std::make_shared(); + auto adding_column_action = std::make_unique(); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1570,7 +1574,7 @@ ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) /// Some actions could become unused. Do not drop inputs to preserve the header. first.removeUnusedActions(false); - return std::make_shared(std::move(first)); + return std::make_unique(std::move(first)); } void ActionsDAG::mergeInplace(ActionsDAG && second) @@ -1963,12 +1967,12 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split second_inputs.push_back(cur.to_second); } - auto first_actions = std::make_shared(); + auto first_actions = std::make_unique(); first_actions->nodes.swap(first_nodes); first_actions->outputs.swap(first_outputs); first_actions->inputs.swap(first_inputs); - auto second_actions = std::make_shared(); + auto second_actions = std::make_unique(); second_actions->nodes.swap(second_nodes); second_actions->outputs.swap(second_outputs); second_actions->inputs.swap(second_inputs); @@ -2302,7 +2306,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti if (conjunction.empty()) return nullptr; - auto actions = std::make_shared(); + auto actions = std::make_unique(); FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); @@ -2866,7 +2870,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( bool visited_children = false; }; - auto result_dag = std::make_shared(); + auto result_dag = std::make_unique(); std::unordered_map result_inputs; std::unordered_map node_to_result_node; @@ -2964,7 +2968,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( const auto & index_hint_args = index_hint->getActions()->getOutputs(); if (index_hint_args.empty()) - index_hint_filter_dag = std::make_shared(); + index_hint_filter_dag = std::make_unique(); else index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, node_name_to_input_node_column, @@ -3108,10 +3112,10 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs( return nodes; } -FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_) - :actions(actions_) +FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_) + //: actions(actions_) { - const auto & actions_outputs = actions->getOutputs(); + const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) { /// find input node which refers to the output node @@ -3147,10 +3151,10 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp return it->second; } -FindAliasForInputName::FindAliasForInputName(const ActionsDAGPtr & actions_) - :actions(actions_) +FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_) + //: actions(actions_) { - const auto & actions_outputs = actions->getOutputs(); + const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) { /// find input node which corresponds to alias diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index c9974fd849c..4a840885b6a 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -12,7 +12,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class IExecutableFunction; using ExecutableFunctionPtr = std::shared_ptr; @@ -262,6 +262,7 @@ public: #endif ActionsDAGPtr clone() const; + ActionsDAGPtr clone(std::unordered_map & old_to_new_nodes) const; static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); @@ -480,11 +481,11 @@ class FindOriginalNodeForOutputName using NameToNodeIndex = std::unordered_map; public: - explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions); + explicit FindOriginalNodeForOutputName(const ActionsDAG & actions); const ActionsDAG::Node * find(const String & output_name); private: - ActionsDAGPtr actions; + //const ActionsDAG & actions; NameToNodeIndex index; }; @@ -493,11 +494,11 @@ class FindAliasForInputName using NameToNodeIndex = std::unordered_map; public: - explicit FindAliasForInputName(const ActionsDAGPtr & actions); + explicit FindAliasForInputName(const ActionsDAG & actions); const ActionsDAG::Node * find(const String & name); private: - ActionsDAGPtr actions; + //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9e56d740e5e..1838a7b04b9 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1009,7 +1009,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & dag.project(args); auto index_hint = std::make_shared(); - index_hint->setActions(std::make_shared(std::move(dag))); + index_hint->setActions(std::make_unique(std::move(dag))); // Arguments are removed. We add function instead of constant column to avoid constant folding. data.addFunction(std::make_unique(index_hint), {}, column_name); @@ -1272,7 +1272,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - std::make_shared(std::move(lambda_dag)), + std::make_unique(std::move(lambda_dag)), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 46d2d60e461..496d9b9b587 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -22,7 +22,7 @@ class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 7f96c927d82..7cbf5afd763 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -73,7 +73,25 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio ExpressionActionsPtr ExpressionActions::clone() const { - return std::make_shared(*this); + auto copy = std::make_shared(ExpressionActions()); + + std::unordered_map copy_map; + copy->actions_dag = actions_dag->clone(copy_map); + copy->actions = actions; + for (auto & action : copy->actions) + action.node = copy_map[action.node]; + + copy->num_columns = num_columns; + + copy->required_columns = required_columns; + copy->input_positions = input_positions; + copy->result_positions = result_positions; + copy->sample_block = sample_block; + + copy->project_inputs = project_inputs; + copy->settings = settings; + + return copy; } namespace diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index ddffe022215..63ea989bd5e 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -84,10 +84,9 @@ private: ExpressionActionsSettings settings; public: - ExpressionActions() = delete; explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); - ExpressionActions(const ExpressionActions &) = default; - ExpressionActions & operator=(const ExpressionActions &) = default; + ExpressionActions(ExpressionActions &&) = default; + ExpressionActions & operator=(ExpressionActions &&) = default; const Actions & getActions() const { return actions; } const std::list & getNodes() const { return actions_dag->getNodes(); } @@ -131,6 +130,7 @@ public: ExpressionActionsPtr clone() const; private: + ExpressionActions() = default; void checkLimits(const ColumnsWithTypeAndName & columns) const; void linearizeActions(const std::unordered_set & lazy_executed_nodes); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 62cddd9caf7..be00e37c751 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -658,7 +658,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, with_alias->getColumnName(), 1 /* direction */, 1 /* nulls_direction */)); - auto actions_dag = std::make_shared(aggregated_columns); + auto actions_dag = std::make_unique(aggregated_columns); getRootActions(column_ast, false, *actions_dag); desc.partition_by_actions.push_back(std::move(actions_dag)); } @@ -679,7 +679,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, order_by_element.direction, order_by_element.nulls_direction)); - auto actions_dag = std::make_shared(aggregated_columns); + auto actions_dag = std::make_unique(aggregated_columns); getRootActions(column_ast, false, *actions_dag); desc.order_by_actions.push_back(std::move(actions_dag)); } @@ -823,13 +823,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) makeWindowDescriptionFromAST(*current_context, window_descriptions, desc, &definition); + auto full_sort_description = desc.full_sort_description; + auto [it, inserted] = window_descriptions.insert( - {default_window_name, desc}); + {default_window_name, std::move(desc)}); if (!inserted) { - assert(it->second.full_sort_description - == desc.full_sort_description); + assert(it->second.full_sort_description == full_sort_description); } it->second.window_functions.push_back(window_function); @@ -1353,10 +1354,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_unique(columns_after_join); getRootActions(child, only_types, *actions_dag); group_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); + std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } } @@ -1606,10 +1607,10 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr { for (const auto & child : select_query->orderBy()->children) { - auto actions_dag = std::make_shared(columns_after_join); + auto actions_dag = std::make_unique(columns_after_join); getRootActions(child, only_types, *actions_dag); order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); + std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } } @@ -1799,7 +1800,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) { - auto actions = std::make_shared(constant_inputs); + auto actions = std::make_unique(constant_inputs); getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); return actions; } @@ -1807,7 +1808,7 @@ ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndNam ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs) { auto actions = getConstActionsDAG(constant_inputs); - return std::make_shared(actions, ExpressionActionsSettings::fromContext(getContext())); + return std::make_shared(std::move(actions), ExpressionActionsSettings::fromContext(getContext())); } std::unique_ptr SelectQueryExpressionAnalyzer::getJoinedPlan() @@ -1878,7 +1879,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (prewhere_dag_and_flags) { - auto dag = std::make_shared(std::move(prewhere_dag_and_flags->dag)); + auto dag = std::make_unique(std::move(prewhere_dag_and_flags->dag)); prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); prewhere_dag_and_flags.reset(); } @@ -1945,7 +1946,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { auto dag = prewhere_dag_and_flags->dag.clone(); ExpressionActions( - dag, + std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 12d6dce8f72..e44a5891e77 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -39,7 +39,7 @@ class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 927bafe4bfb..4dbdebd0d06 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -384,7 +384,7 @@ Chain InterpreterInsertQuery::buildPreSinkChain( context_ptr, null_as_default); - 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)); /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) @@ -597,7 +597,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + auto actions = std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f5b54ec64cb..64a17a7ba87 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1302,7 +1302,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( result_columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); - interpolate_descr = std::make_shared(merge_dag, aliases); + interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } return interpolate_descr; @@ -2042,7 +2042,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(prewhere_info.row_level_filter), + std::make_shared(prewhere_info.row_level_filter->clone()), prewhere_info.row_level_column_name, true); }); } @@ -2050,7 +2050,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(prewhere_info.prewhere_actions), + header, std::make_shared(prewhere_info.prewhere_actions->clone()), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2094,8 +2094,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. - analysis.prewhere_info = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); - analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; + analysis.prewhere_info = std::make_shared(std::move(analysis.filter_info->actions), analysis.filter_info->column_name); + analysis.prewhere_info->remove_prewhere_column = std::move(analysis.filter_info->do_remove_column); analysis.prewhere_info->need_filter = true; analysis.filter_info = nullptr; } @@ -2103,8 +2103,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; - analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name; + analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..1bb770bf561 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1211,7 +1211,7 @@ void MutationsInterpreter::Source::read( MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, part, required_columns, - apply_deleted_mask_, filter, context_, + apply_deleted_mask_, std::move(filter), context_, getLogger("MutationsInterpreter")); } else diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 6191eb73fd4..baf3a743f40 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -470,7 +470,7 @@ static ActionsDAGPtr createWrapWithTupleActions( if (column_names_to_wrap.empty()) return nullptr; - auto actions_dag = std::make_shared(source_columns); + auto actions_dag = std::make_unique(source_columns); FunctionOverloadResolverPtr func_builder = std::make_unique(std::make_shared()); @@ -616,7 +616,7 @@ TableJoin::createConvertingActions( mergeDags(right_dag, std::move(new_right_dag)); } - return {left_dag, right_dag}; + return {std::move(left_dag), std::move(right_dag)}; } template diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index c26e4517c9a..17bfe619c30 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -14,7 +14,7 @@ namespace DB class ASTFunction; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; struct WindowFunctionDescription { @@ -93,8 +93,8 @@ struct WindowDescription // then by ORDER BY. This field holds this combined sort order. SortDescription full_sort_description; - std::vector partition_by_actions; - std::vector order_by_actions; + std::vector> partition_by_actions; + std::vector> order_by_actions; WindowFrame frame; diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index fbf17d7efb7..929999c8c37 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -21,7 +21,7 @@ ActionsDAGPtr addMissingDefaults( ContextPtr context, bool null_as_default) { - auto actions = std::make_shared(header.getColumnsWithTypeAndName()); + auto actions = std::make_unique(header.getColumnsWithTypeAndName()); auto & index = actions->getOutputs(); /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 0a3d4de478c..94afd806dfd 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -15,7 +15,7 @@ class NamesAndTypesList; class ColumnsDescription; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /** Adds three types of columns into block * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 239cce5b427..b000264ae33 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -163,7 +163,7 @@ ActionsDAGPtr createExpressions( auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context}; - auto dag = std::make_shared(header.getNamesAndTypesList()); + auto dag = std::make_unique(header.getNamesAndTypesList()); auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns); dag = ActionsDAG::merge(std::move(*dag), std::move(*actions)); diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index bea44bf6db9..ffc77561e79 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -24,7 +24,7 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Create actions which adds missing defaults to block according to required_columns using columns description /// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index d5e39a9f123..162d3fe8d11 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -88,7 +88,7 @@ public: auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); - ActionsDAGPtr alias_column_actions_dag = std::make_shared(); + ActionsDAGPtr alias_column_actions_dag = std::make_unique(); PlannerActionsVisitor actions_visitor(planner_context, false); auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression()); if (outputs.size() != 1) @@ -97,7 +97,7 @@ public: const auto & column_name = column_node->getColumnName(); const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name); alias_column_actions_dag->addOrReplaceInOutputs(alias_node); - table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns); + table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns); } return; @@ -335,7 +335,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr collect_source_columns_visitor.setKeepAliasColumns(false); collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); - auto prewhere_actions_dag = std::make_shared(); + auto prewhere_actions_dag = std::make_unique(); QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2d42ed73223..681ae7e6ac4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -213,7 +213,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & if (!read_from_dummy) continue; - auto filter_actions = read_from_dummy->getFilterActionsDAG(); + auto filter_actions = read_from_dummy->detachFilterActionsDAG(); const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; } @@ -331,13 +331,13 @@ public: void addExpressionStep(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { auto actions = expression_actions->dag.clone(); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); @@ -346,13 +346,13 @@ void addExpressionStep(QueryPlan & query_plan, void addFilterStep(QueryPlan & query_plan, const FilterAnalysisResult & filter_analysis_result, const std::string & step_description, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { auto actions = filter_analysis_result.filter_actions->dag.clone(); if (filter_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), actions, filter_analysis_result.filter_column_name, @@ -544,7 +544,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -560,7 +560,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions); + result_actions_to_execute.push_back(actions.get()); } auto totals_having_step = std::make_unique( @@ -714,7 +714,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (query_node.hasInterpolate()) { - auto interpolate_actions_dag = std::make_shared(); + auto interpolate_actions_dag = std::make_unique(); auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) { @@ -885,7 +885,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, const QueryTreeNodePtr & query_tree, - std::vector & result_actions_to_execute) + std::vector & result_actions_to_execute) { const auto & query_node = query_tree->as(); @@ -932,14 +932,14 @@ void addWindowSteps(QueryPlan & query_plan, const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - auto window_descriptions = window_analysis_result.window_descriptions; - sortWindowDescriptions(window_descriptions); + const auto & window_descriptions = window_analysis_result.window_descriptions; + auto perm = sortWindowDescriptions(window_descriptions); size_t window_descriptions_size = window_descriptions.size(); for (size_t i = 0; i < window_descriptions_size; ++i) { - const auto & window_description = window_descriptions[i]; + const auto & window_description = window_descriptions[perm[i]]; /** We don't need to sort again if the input from previous window already * has suitable sorting. Also don't create sort steps when there are no @@ -952,8 +952,9 @@ void addWindowSteps(QueryPlan & query_plan, bool need_sort = !window_description.full_sort_description.empty(); if (need_sort && i != 0) { - need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description) - || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[i - 1].partition_by.size()); + auto prev = perm[i - 1]; + need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[prev].full_sort_description) + || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[prev].partition_by.size()); } if (need_sort) { @@ -1054,9 +1055,9 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana } } -void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set & useful_sets) +void collectSetsFromActionsDAG(const ActionsDAG & dag, std::unordered_set & useful_sets) { - for (const auto & node : dag->getNodes()) + for (const auto & node : dag.getNodes()) { if (node.column) { @@ -1075,7 +1076,7 @@ void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set(adaptor->getFunction().get())) { - collectSetsFromActionsDAG(index_hint->getActions(), useful_sets); + collectSetsFromActionsDAG(*index_hint->getActions(), useful_sets); } } } @@ -1086,13 +1087,13 @@ void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) + const std::vector & result_actions_to_execute) { auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; - for (const auto & actions_to_execute : result_actions_to_execute) - collectSetsFromActionsDAG(actions_to_execute, useful_sets); + for (const auto * actions_to_execute : result_actions_to_execute) + collectSetsFromActionsDAG(*actions_to_execute, useful_sets); auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); @@ -1448,7 +1449,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(filters.filter_actions); + table_expression_data.setFilterActions(filters.filter_actions->clone()); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } @@ -1539,15 +1540,15 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info); - std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); + std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) { if (table_expression_data.getPrewhereFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions()); + result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions().get()); if (table_expression_data.getRowLevelFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions()); + result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions().get()); } if (query_processing_info.isIntermediateStage()) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7a12d5d690d..59ec7778e21 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -757,7 +757,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } - auto lambda_actions_dag = std::make_shared(); + auto lambda_actions_dag = std::make_unique(); actions_stack.emplace_back(*lambda_actions_dag, node); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); @@ -765,7 +765,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); - auto lambda_actions = std::make_shared(lambda_actions_dag, expression_actions_settings); + auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); Names captured_column_names; ActionsDAG::NodeRawConstPtrs lambda_children; @@ -879,7 +879,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & function_node = node->as(); auto function_node_name = action_node_name_helper.calculateActionNodeName(node); - auto index_hint_actions_dag = std::make_shared(); + auto index_hint_actions_dag = std::make_unique(); auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); std::unordered_set index_hint_actions_dag_output_node_names; PlannerActionsVisitor actions_visitor(planner_context); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 43b223172e6..918cfad703e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,7 +591,7 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) std::unique_ptr createComputeAliasColumnsStep( const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { - ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared(current_data_stream.header.getColumnsWithTypeAndName()); + ActionsDAGPtr merged_alias_columns_actions_dag = std::make_unique(current_data_stream.header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) @@ -646,7 +646,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions(); + table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions()->clone(); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = prewhere_actions; + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -787,7 +787,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres const auto & columns_names = table_expression_data.getColumnNames(); std::vector> where_filters; - const auto add_filter = [&](const FilterDAGInfo & filter_info, std::string description) + const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description) { if (!filter_info.actions) return; @@ -805,34 +805,34 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = filter_info.actions; + prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = filter_info.actions; + prewhere_info->row_level_filter = std::move(filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } else { - where_filters.emplace_back(filter_info, std::move(description)); + where_filters.emplace_back(std::move(filter_info), std::move(description)); } } else { - where_filters.emplace_back(filter_info, std::move(description)); + where_filters.emplace_back(std::move(filter_info), std::move(description)); } }; auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); - add_filter(row_policy_filter_info, "Row-level security filter"); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions); + table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions->clone()); + add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { @@ -1063,7 +1063,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (from_stage == QueryProcessingStage::FetchColumns) { - auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto rename_actions_dag = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; for (auto & output_node : rename_actions_dag->getOutputs()) @@ -1077,7 +1077,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); - auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); rename_step->setStepDescription("Change column names to column identifiers"); query_plan.addStep(std::move(rename_step)); } @@ -1117,7 +1117,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); for (auto & output_node : cast_actions_dag->getOutputs()) { @@ -1178,6 +1178,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); + left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); + left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); + join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); @@ -1223,7 +1226,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); for (auto & output_node : cast_actions_dag->getOutputs()) { @@ -1381,9 +1384,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_clauses_and_actions.mixed_join_expressions_actions) { + left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( - join_clauses_and_actions.mixed_join_expressions_actions, + std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); } } @@ -1537,7 +1541,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto drop_unused_columns_after_join_actions_dag = std::make_unique(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; std::optional first_skipped_column_node_index; @@ -1582,14 +1586,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` - for (auto && action_dag : right_join_tree_query_plan.actions_dags) + for (const auto * action_dag : right_join_tree_query_plan.actions_dags) left_join_tree_query_plan.actions_dags.emplace_back(action_dag); - if (join_clauses_and_actions.left_join_expressions_actions) - left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions)); - if (join_clauses_and_actions.right_join_expressions_actions) - left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions)); - if (join_clauses_and_actions.mixed_join_expressions_actions) - left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions); + // if (join_clauses_and_actions.left_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); + // if (join_clauses_and_actions.right_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); + // if (join_clauses_and_actions.mixed_join_expressions_actions) + // left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping); auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping; @@ -1619,7 +1623,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto plan = std::move(join_tree_query_plan.query_plan); auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr array_join_action_dag = std::make_shared(plan_output_columns); + ActionsDAGPtr array_join_action_dag = std::make_unique(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); std::unordered_set array_join_expressions_output_nodes; @@ -1642,13 +1646,13 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); - join_tree_query_plan.actions_dags.push_back(array_join_action_dag); + join_tree_query_plan.actions_dags.push_back(array_join_action_dag.get()); - auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), array_join_action_dag); + auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); plan.addStep(std::move(array_join_actions)); - auto drop_unused_columns_before_array_join_actions_dag = std::make_shared(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_before_array_join_actions_dag_updated_outputs_names; diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index 9110b2bfef9..675079427eb 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -16,7 +16,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - std::vector actions_dags{}; + std::vector actions_dags{}; std::unordered_map query_node_to_plan_step_mapping{}; }; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 84efdd21336..45842c0d705 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -376,8 +376,8 @@ JoinClausesAndActions buildJoinClausesAndActions( const JoinNode & join_node, const PlannerContextPtr & planner_context) { - ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); - ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); + ActionsDAGPtr left_join_actions = std::make_unique(left_table_expression_columns); + ActionsDAGPtr right_join_actions = std::make_unique(right_table_expression_columns); ColumnsWithTypeAndName mixed_table_expression_columns; for (const auto & left_column : left_table_expression_columns) { @@ -387,7 +387,7 @@ JoinClausesAndActions buildJoinClausesAndActions( { mixed_table_expression_columns.push_back(right_column); } - ActionsDAGPtr mixed_join_actions = std::make_shared(mixed_table_expression_columns); + ActionsDAGPtr mixed_join_actions = std::make_unique(mixed_table_expression_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -601,7 +601,7 @@ JoinClausesAndActions buildJoinClausesAndActions( /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - auto mixed_join_expressions_actions = std::make_shared(mixed_table_expression_columns); + auto mixed_join_expressions_actions = std::make_unique(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) @@ -611,14 +611,14 @@ JoinClausesAndActions buildJoinClausesAndActions( mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; mixed_join_expressions_actions->removeUnusedActions(required_names); - result.mixed_join_expressions_actions = mixed_join_expressions_actions; + result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } else { const auto & join_clause = result.join_clauses.front(); const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes(); auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); - result.mixed_join_expressions_actions = mixed_join_expressions_actions; + result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } auto outputs = result.mixed_join_expressions_actions->getOutputs(); if (outputs.size() != 1) diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index ce74d82c08d..9deceeef9a3 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -120,7 +120,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & return result; } -void sortWindowDescriptions(std::vector & window_descriptions) +std::vector sortWindowDescriptions(const std::vector & window_descriptions) { auto window_description_comparator = [](const WindowDescription & lhs, const WindowDescription & rhs) { @@ -151,7 +151,16 @@ void sortWindowDescriptions(std::vector & window_descriptions return left.size() > right.size(); }; - ::sort(window_descriptions.begin(), window_descriptions.end(), window_description_comparator); + auto comparator = [&](size_t lhs, size_t rhs) + { + return window_description_comparator(window_descriptions[lhs], window_descriptions[rhs]); + }; + + std::vector perm(window_descriptions.size()); + std::iota(perm.begin(), perm.end(), 0U); + ::sort(perm.begin(), perm.end(), comparator); + + return perm; } } diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h index 1552ef5a71f..3039ecefc4b 100644 --- a/src/Planner/PlannerWindowFunctions.h +++ b/src/Planner/PlannerWindowFunctions.h @@ -15,6 +15,6 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & /** Try to sort window descriptions in such an order that the window with the longest * sort description goes first, and all window that use its prefixes follow. */ -void sortWindowDescriptions(std::vector & window_descriptions); +std::vector sortWindowDescriptions(const std::vector & window_descriptions); } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 18a6d297838..493ecf5ef53 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -440,7 +440,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSets(filter_query_tree, *planner_context); - auto filter_actions_dag = std::make_shared(); + auto filter_actions_dag = std::make_unique(); PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 0d7e05af1de..64ba7f7cd2a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -303,7 +303,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B const auto & header = ports[set_counter]->getHeader(); /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - auto dag = std::make_shared(header.getColumnsWithTypeAndName()); + auto dag = std::make_unique(header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(output_header.columns() + 1); @@ -347,7 +347,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B } dag->getOutputs().swap(outputs); - auto expression = std::make_shared(dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); connect(*ports[set_counter], transform->getInputPort()); diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index d010a3327a6..b6c70061987 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -36,7 +36,7 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { - auto dag = std::make_shared(header.getColumnsWithTypeAndName()); + auto dag = std::make_unique(header.getColumnsWithTypeAndName()); auto & outputs = dag->getOutputs(); if (use_nulls) @@ -59,7 +59,7 @@ ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, b grouping_node = &dag->materializeNode(*grouping_node); outputs.insert(outputs.begin(), grouping_node); - auto expression = std::make_shared(dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); return std::make_shared(header, expression); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 0ccb0c4492a..90ac94a1ace 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -30,13 +30,13 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDA input_stream_, ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(actions_dag_) + , actions_dag(actions_dag_->clone()) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); + auto expression = std::make_shared(actions_dag->clone(), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -49,7 +49,7 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu pipeline.getHeader().getColumnsWithTypeAndName(), output_stream->header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto convert_actions = std::make_shared(convert_actions_dag, settings.getActionsSettings()); + auto convert_actions = std::make_shared(std::move(convert_actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } @@ -79,7 +79,7 @@ void ExpressionStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(actions_dag); + FindAliasForInputName alias_finder(*actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 3eef14ac129..ebbac8217cb 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -5,7 +5,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class ExpressionTransform; class JoiningTransform; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 72934665b5c..ef9f1d17822 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -46,7 +46,6 @@ FilterStep::FilterStep( filter_column_name_, remove_filter_column_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_)) - , actions_dag(actions_dag_) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { @@ -56,7 +55,7 @@ FilterStep::FilterStep( void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); + auto expression = std::make_shared(std::move(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { @@ -70,7 +69,7 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ pipeline.getHeader().getColumnsWithTypeAndName(), output_stream->header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto convert_actions = std::make_shared(convert_actions_dag, settings.getActionsSettings()); + auto convert_actions = std::make_shared(std::move(convert_actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -88,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); expression->describeActions(settings.out, prefix); } @@ -97,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } @@ -111,7 +110,7 @@ void FilterStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(actions_dag); + FindAliasForInputName alias_finder(*actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index 939d0900c86..0f894a570b7 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -5,7 +5,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; /// Implements WHERE, HAVING operations. See FilterTransform. class FilterStep : public ITransformingStep diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 0a3a4094a66..87e16b5a244 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -10,7 +10,7 @@ namespace DB::QueryPlanOptimizations { /// build actions DAG from stack of steps -static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) +static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) return nullptr; @@ -27,10 +27,10 @@ static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_st } static std::set -getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) +getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector & dag_stack) { auto actions = buildActionsForPlanPath(dag_stack); - FindOriginalNodeForOutputName original_node_finder(actions); + FindOriginalNodeForOutputName original_node_finder(*actions); std::set original_distinct_columns; for (const auto & column : distinct_columns) { @@ -65,7 +65,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) /// (3) gather actions DAG to find original names for columns in distinct step later std::vector steps_to_update; QueryPlan::Node * node = parent_node; - std::vector dag_stack; + std::vector dag_stack; while (!node->children.empty()) { auto * step = dynamic_cast(node->step.get()); @@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) steps_to_update.push_back(step); if (const auto * const expr = typeid_cast(step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 263598bdca7..ff1cefff09a 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -611,7 +611,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{filter->getExpression(), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{filter->getExpression()->clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 1badd315200..13b691da888 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = filter_step->getExpression(); + auto filter_expression = filter_step->getExpression()->clone(); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index dbcaf5f00a7..e57d3319076 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -18,15 +18,15 @@ void optimizePrimaryKeyCondition(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions, storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter, storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) - source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(filter_step->getExpression()->clone(), filter_step->getFilterColumnName()); /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 537555afa2a..8e782e68db8 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 70327bc95b4..7c45ef48252 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -486,7 +486,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, candidate.dag->getRequiredColumnsNames(), - (dag.filter_node ? dag.dag : nullptr), + (dag.filter_node ? dag.dag.get() : nullptr), parts, max_added_blocks.get(), context); @@ -675,7 +675,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu query_info, context, max_added_blocks, - candidate.dag); + candidate.dag.get()); if (!analyzed) continue; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 0af3869ccf1..c7e96d66817 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -172,7 +172,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod query_info, context, max_added_blocks, - query.filter_node ? query.dag : nullptr); + query.filter_node ? query.dag.get() : nullptr); if (!analyzed) continue; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index af1578d6af8..d8b40b22904 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -213,7 +213,7 @@ bool analyzeProjectionCandidate( const SelectQueryInfo & query_info, const ContextPtr & context, const std::shared_ptr & max_added_blocks, - const ActionsDAGPtr & dag) + const ActionsDAG * dag) { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; @@ -238,7 +238,7 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = dag; + projection_query_info.filter_actions_dag = dag->clone(); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index e1e106b988e..59ad3a43b97 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -60,6 +60,6 @@ bool analyzeProjectionCandidate( const SelectQueryInfo & query_info, const ContextPtr & context, const std::shared_ptr & max_added_blocks, - const ActionsDAGPtr & dag); + const ActionsDAG * dag); } diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 51df25b35f4..d3c75c988e7 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -65,7 +65,7 @@ namespace } /// build actions DAG from stack of steps - ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) + ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) return nullptr; @@ -83,7 +83,7 @@ namespace } bool compareAggregationKeysWithDistinctColumns( - const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector> actions_chain) + const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector> actions_chain) { logDebug("aggregation_keys", aggregation_keys); logDebug("aggregation_keys size", aggregation_keys.size()); @@ -93,7 +93,8 @@ namespace std::set source_columns; for (auto & actions : actions_chain) { - FindOriginalNodeForOutputName original_node_finder(buildActionsForPlanPath(actions)); + auto tmp_actions = buildActionsForPlanPath(actions); + FindOriginalNodeForOutputName original_node_finder(*tmp_actions); for (const auto & column : current_columns) { logDebug("distinct column name", column); @@ -152,8 +153,8 @@ namespace const DistinctStep * distinct_step = typeid_cast(distinct_node->step.get()); chassert(distinct_step); - std::vector dag_stack; - std::vector> actions_chain; + std::vector dag_stack; + std::vector> actions_chain; const DistinctStep * inner_distinct_step = nullptr; const IQueryPlanStep * aggregation_before_distinct = nullptr; const QueryPlan::Node * node = distinct_node; @@ -182,9 +183,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); if (inner_distinct_step = typeid_cast(node->step.get()); inner_distinct_step) @@ -222,7 +223,7 @@ namespace chassert(distinct_step); const auto distinct_columns = getDistinctColumns(distinct_step); - std::vector dag_stack; + std::vector dag_stack; const DistinctStep * inner_distinct_step = nullptr; const QueryPlan::Node * node = distinct_node; while (!node->children.empty()) @@ -235,9 +236,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression()); + dag_stack.push_back(expr->getExpression().get()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression()); + dag_stack.push_back(filter->getExpression().get()); node = node->children.front(); inner_distinct_step = typeid_cast(node->step.get()); @@ -267,7 +268,7 @@ namespace logActionsDAG("distinct pass: merged DAG", path_actions); /// compare columns of two DISTINCTs - FindOriginalNodeForOutputName original_node_finder(path_actions); + FindOriginalNodeForOutputName original_node_finder(*path_actions); for (const auto & column : distinct_columns) { const auto * alias_node = original_node_finder.find(String(column)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b2d8aa0e218..e5370c1c130 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -174,7 +174,7 @@ static void updateSortDescriptionForOutputStream( { if (prewhere_info->prewhere_actions) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + FindOriginalNodeForOutputName original_column_finder(*prewhere_info->prewhere_actions); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -185,7 +185,7 @@ static void updateSortDescriptionForOutputStream( if (prewhere_info->row_level_filter) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->row_level_filter); + FindOriginalNodeForOutputName original_column_finder(*prewhere_info->row_level_filter); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -830,10 +830,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ pipes[0].getHeader().getColumnsWithTypeAndName(), pipes[1].getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); + auto converting_expr = std::make_shared(std::move(conversion_action)); pipes[0].addSimpleTransform( - [conversion_action](const Block & header) + [converting_expr](const Block & header) { - auto converting_expr = std::make_shared(conversion_action); return std::make_shared(header, converting_expr); }); return Pipe::unitePipes(std::move(pipes)); @@ -849,7 +849,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ static ActionsDAGPtr createProjection(const Block & header) { - return std::make_shared(header.getNamesAndTypesList()); + return std::make_unique(header.getNamesAndTypesList()); } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( @@ -1046,7 +1046,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( for (size_t j = 0; j < prefix_size; ++j) sort_description.emplace_back(sorting_columns[j], input_order_info->direction); - auto sorting_key_expr = std::make_shared(sorting_key_prefix_expr); + auto sorting_key_expr = std::make_shared(std::move(sorting_key_prefix_expr)); auto merge_streams = [&](Pipe & pipe) { @@ -1341,10 +1341,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( pipes[0].getHeader().getColumnsWithTypeAndName(), pipes[1].getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); + auto converting_expr = std::make_shared(std::move(conversion_action)); pipes[0].addSimpleTransform( - [conversion_action](const Block & header) + [converting_expr](const Block & header) { - auto converting_expr = std::make_shared(conversion_action); return std::make_shared(header, converting_expr); }); return Pipe::unitePipes(std::move(pipes)); @@ -1378,7 +1378,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( static void buildIndexes( std::optional & indexes, - ActionsDAGPtr filter_actions_dag, + const ActionsDAG * filter_actions_dag, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, const ContextPtr & context, @@ -1518,11 +1518,11 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. - query_info.filter_actions_dag = filter_actions_dag; + query_info.filter_actions_dag = std::move(filter_actions_dag); buildIndexes( indexes, - filter_actions_dag, + query_info.filter_actions_dag.get(), data, prepared_parts, context, @@ -1564,7 +1564,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & primary_key_column_names = primary_key.column_names; if (!indexes) - buildIndexes(indexes, query_info_.filter_actions_dag, data, parts, context_, query_info_, metadata_snapshot); + buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot); if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); @@ -1993,7 +1993,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(result.sampling.filter_expression); + auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2031,7 +2031,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(result_projection); + auto projection_actions = std::make_shared(result_projection->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2048,7 +2048,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons ActionsDAG::MatchColumnsMode::Name, true); - auto converting_dag_expr = std::make_shared(convert_actions_dag); + auto converting_dag_expr = std::make_shared(std::move(convert_actions_dag)); pipe.addSimpleTransform([&](const Block & header) { @@ -2126,7 +2126,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -2135,7 +2135,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -2161,7 +2161,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2171,7 +2171,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index caa8aa2e1bd..e32507e1f22 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -23,7 +23,7 @@ struct MergeTreeDataSelectSamplingData bool read_nothing = false; Float64 used_sample_factor = 1.0; std::shared_ptr filter_function; - ActionsDAGPtr filter_expression; + std::shared_ptr filter_expression; }; struct UsefulSkipIndexes diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 11371578c79..b9b239c721b 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -441,7 +441,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() chassert(numbers_storage.step != UInt64{0}); /// Build rpn of query filters - KeyCondition condition(filter_actions_dag, context, column_names, key_expression); + KeyCondition condition(filter_actions_dag.get(), context, column_names, key_expression); if (condition.extractPlainRanges(ranges)) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index ad0940b90b9..b91debc8239 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 126d4824fff..8ac0cc24ed1 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -33,6 +33,7 @@ public: } const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; } + ActionsDAGPtr detachFilterActionsDAG() { return std::move(filter_actions_dag); } const SelectQueryInfo & getQueryInfo() const { return query_info; } const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } @@ -53,7 +54,7 @@ public: void applyFilters() { applyFilters(std::move(filter_nodes)); - filter_dags = {}; + filter_dags.clear(); } virtual void applyFilters(ActionDAGNodes added_filter_nodes); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index ac5e144bf4a..45de6c31d24 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(actions_dag_) + , actions_dag(actions_dag_->clone()) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(actions_dag, settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(actions_dag->clone(), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(actions_dag); + auto expression = std::make_shared(actions_dag->clone()); map.add("Expression", expression->toTree()); } } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index a81bc7bb1a9..52ef5437701 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -6,7 +6,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; enum class TotalsMode : uint8_t; diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index 74a0e5930c7..47883e5edf6 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -7,7 +7,7 @@ namespace DB { class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; class WindowTransform; diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index ee155d6f78c..fcf576637ff 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -16,13 +16,13 @@ protected: /// Represents pushed down filters in source std::shared_ptr key_condition; - void setKeyConditionImpl(const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const Block & keys) + void setKeyConditionImpl(const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & keys) { key_condition = std::make_shared( filter_actions_dag, context, keys.getNames(), - std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); + std::make_shared(std::make_unique(keys.getColumnsWithTypeAndName()))); } public: diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index bb38c3e1dc5..95267bc24e0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(interpolate_description->actions); + interpolate_actions = std::make_shared(interpolate_description->actions->clone()); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 28d8128e052..2cd51259549 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -516,7 +516,7 @@ void StorageHive::initMinMaxIndexExpression() partition_names = partition_name_types.getNames(); partition_types = partition_name_types.getTypes(); partition_minmax_idx_expr = std::make_shared( - std::make_shared(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); + std::make_unique(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); } NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); @@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression() hivefile_name_types.push_back(column); } hivefile_minmax_idx_expr = std::make_shared( - std::make_shared(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); + std::make_unique(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); } ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) @@ -647,7 +647,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(filter_actions_dag.get(), getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -715,7 +715,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(filter_actions_dag.get(), getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index 2a697fa5654..e03ecc05064 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -160,7 +160,7 @@ KeyDescription KeyDescription::buildEmptyKey() { KeyDescription result; result.expression_list_ast = std::make_shared(); - result.expression = std::make_shared(std::make_shared(), ExpressionActionsSettings{}); + result.expression = std::make_shared(std::make_unique(), ExpressionActionsSettings{}); return result; } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f8cf19120c7..48ec5529af0 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -691,7 +691,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { - auto res = std::make_shared(); + auto res = std::make_unique(); std::unordered_map to_inverted; @@ -777,7 +777,7 @@ void KeyCondition::getAllSpaceFillingCurves() } KeyCondition::KeyCondition( - ActionsDAGPtr filter_dag, + const ActionsDAG * filter_dag, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 2bc3b108e02..14ef74ea113 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -41,7 +41,7 @@ class KeyCondition public: /// Construct key condition from ActionsDAG nodes KeyCondition( - ActionsDAGPtr filter_dag, + const ActionsDAG * filter_dag, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e0ea4cdbcd..7b642c34f37 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -487,7 +487,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP ASTPtr expression_ast; ConditionSelectivityEstimator result; - PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context); + PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag.get(), local_context); if (partition_pruner.isUseless()) { @@ -746,7 +746,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti if (!partition_key.column_names.empty()) partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); - return std::make_shared(std::make_shared(partition_key_columns), settings); + return std::make_shared(std::make_unique(partition_key_columns), settings); } Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) @@ -1152,7 +1152,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (!virtual_columns_block.has(input->result_name)) valid = false; - PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */); + PartitionPruner partition_pruner(metadata_snapshot, filter_dag.get(), local_context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; @@ -6819,7 +6819,7 @@ using PartitionIdToMaxBlock = std::unordered_map; Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6f736a4afd..52916d85fef 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -403,7 +403,7 @@ public: Block getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, const DataPartsVector & parts, const PartitionIdToMaxBlock * max_block_numbers_to_read, ContextPtr query_context) const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2e287ff3042..61b8b6fdaa8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -442,7 +442,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( } void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( - std::optional & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context) + std::optional & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context) { if (!filter_dag) return; @@ -463,10 +463,10 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( return; part_offset_condition.emplace(KeyCondition{ - dag, + dag.get(), context, sample.getNames(), - std::make_shared(std::make_shared(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), + std::make_shared(std::make_unique(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), {}}); } @@ -474,7 +474,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context) { if (!filter_dag) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 788355c1e59..39bff5eacd6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -161,7 +161,7 @@ public: /// If possible, construct optional key condition from predicates containing _part_offset column. static void buildKeyConditionFromPartOffset( - std::optional & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context); + std::optional & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context); /// If possible, filter using expression on virtual columns. /// Example: SELECT count() FROM table WHERE _part = 'part_name' @@ -170,7 +170,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context); /// Filter parts using minmax index and partition key. diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e492ca0aec2..457c85eaa46 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -332,7 +332,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec return std::make_shared(index, query, distance_function, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index d511ab84859..282920c608e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -99,7 +99,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; bool isVectorSearch() const override { return true; } private: diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index fc5147bb56c..c6a00751f25 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -201,7 +201,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & } MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) + const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_) : WithContext(context_), header(header_), hash_functions(hash_functions_) { if (!filter_actions_dag) @@ -897,7 +897,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con return std::make_shared(bits_per_row, hash_functions, index.column_names); } -MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(filter_actions_dag, context, index.sample_block, hash_functions); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index d66c4b8b6ca..bd1b137176a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -69,7 +69,7 @@ public: std::vector> predicate; }; - MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); + MergeTreeIndexConditionBloomFilter(const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_); bool alwaysUnknownOrTrue() const override; @@ -142,7 +142,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const override; private: size_t bits_per_row; diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 8cf58687125..5b6813d12e3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t } MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -733,7 +733,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator } MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition( - const ActionsDAGPtr & filter_dag, ContextPtr context) const + const ActionsDAG * filter_dag, ContextPtr context) const { return std::make_shared(filter_dag, context, index.sample_block, params, token_extractor.get()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h index 6fd969030df..fe042884550 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.h @@ -62,7 +62,7 @@ class MergeTreeConditionBloomFilterText final : public IMergeTreeIndexCondition { public: MergeTreeConditionBloomFilterText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const BloomFilterParameters & params_, @@ -163,7 +163,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_dag, ContextPtr context) const override; + const ActionsDAG * filter_dag, ContextPtr context) const override; BloomFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 47ce24b91eb..cd6af68ebcc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -186,7 +186,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, } MergeTreeConditionFullText::MergeTreeConditionFullText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & index_sample_block, const GinFilterParameters & params_, @@ -768,7 +768,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregatorForPart } MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(filter_actions_dag, context, index.sample_block, params, token_extractor.get()); }; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index 1a5e848e5ac..8e0b1a22acb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -63,7 +63,7 @@ class MergeTreeConditionFullText final : public IMergeTreeIndexCondition, WithCo { public: MergeTreeConditionFullText( - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & index_sample_block, const GinFilterParameters & params_, @@ -170,7 +170,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const override; GinFilterParameters params; /// Function for selecting next token. diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 0995e2724ec..cd8065ecadf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -79,7 +79,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(cons } MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( - const ActionsDAGPtr &, ContextPtr) const + const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 130e708d76f..e60335fe724 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -69,7 +69,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 20dfed8cf8f..c60d63a59ba 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -157,7 +157,7 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s namespace { -KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) +KeyCondition buildCondition(const IndexDescription & index, const ActionsDAG * filter_actions_dag, ContextPtr context) { return KeyCondition{filter_actions_dag, context, index.column_names, index.expression}; } @@ -165,7 +165,7 @@ KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr } MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax( - const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context) + const IndexDescription & index, const ActionsDAG * filter_actions_dag, ContextPtr context) : index_data_types(index.data_types) , condition(buildCondition(index, filter_actions_dag, context)) { @@ -198,7 +198,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const Me } MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(index, filter_actions_dag, context); } diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index dca26fb7b28..c5031ccbb27 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -50,7 +50,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition public: MergeTreeIndexConditionMinMax( const IndexDescription & index, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -77,7 +77,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; const char* getSerializedFileExtension() const override { return ".idx2"; } MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index b11cbf1e034..7c65381b05b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -245,7 +245,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context) : index_name(index_name_) , max_rows(max_rows_) @@ -272,9 +272,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); - actions = std::make_shared(filter_actions_dag); actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; + actions = std::make_shared(std::move(filter_actions_dag)); } bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const @@ -544,7 +544,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge } MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const + const ActionsDAG * filter_actions_dag, ContextPtr context) const { return std::make_shared(index.name, index.sample_block, max_rows, filter_actions_dag, context); } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index 6efc2effafd..abd40b3cf9d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -83,7 +83,7 @@ public: const String & index_name_, const Block & index_sample_block, size_t max_rows_, - const ActionsDAGPtr & filter_dag, + const ActionsDAG * filter_dag, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -138,7 +138,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + const ActionsDAG * filter_actions_dag, ContextPtr context) const override; size_t max_rows = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index c9df7210569..59a4b0fbf9c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -367,7 +367,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel return std::make_shared(index, query, distance_function, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index 5107cfee371..41de94402c9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -101,7 +101,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const; - MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; bool isVectorSearch() const override { return true; } private: diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index a9f1fa9378f..1be73e1c811 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -167,7 +167,7 @@ struct IMergeTreeIndex } virtual MergeTreeIndexConditionPtr createIndexCondition( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const = 0; + const ActionsDAG * filter_actions_dag, ContextPtr context) const = 0; virtual bool isVectorSearch() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 78b67de1a7e..8fa5b2cc955 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->row_level_filter, actions_settings), + .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 02f8d6f4f6a..98b35a3ca2c 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -376,7 +376,7 @@ public: { const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - KeyCondition key_condition(filter, context, primary_key_column_names, primary_key.expression); + KeyCondition key_condition(filter.get(), context, primary_key_column_names, primary_key.expression); LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) @@ -437,7 +437,7 @@ void createReadFromPartStep( auto reading = std::make_unique(type, storage, storage_snapshot, std::move(data_part), std::move(columns_to_read), apply_deleted_mask, - filter, std::move(context), log); + std::move(filter), std::move(context), log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 43e3b0c505a..25596b42951 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -50,7 +50,7 @@ void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map; const ActionsDAG::Node & addClonedDAGToDAG( size_t step, const ActionsDAG::Node * original_dag_node, - ActionsDAGPtr new_dag, + const ActionsDAGPtr & new_dag, OriginalToNewNodeMap & node_remap, NodeNameToLastUsedStepMap & node_to_step_map) { @@ -72,7 +72,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { /// If the node is already in the new DAG, return it const auto & node_ref = node_remap.at(node_name); - if (node_ref.dag == new_dag) + if (node_ref.dag == new_dag.get()) return *node_ref.node; /// If the node is known from the previous steps, add it as an input, except for constants @@ -80,7 +80,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { node_ref.dag->addOrReplaceInOutputs(*node_ref.node); const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); - node_remap[node_name] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is it always correct? + node_remap[node_name] = {new_dag.get(), &new_node}; /// TODO: here we update the node reference. Is it always correct? /// Remember the index of the last step which reuses this node. /// We cannot remove this node from the outputs before that step. @@ -93,7 +93,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( if (original_dag_node->type == ActionsDAG::ActionType::INPUT) { const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -102,7 +102,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { const auto & new_node = new_dag->addColumn( ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -110,7 +110,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( { const auto & alias_child = addClonedDAGToDAG(step, original_dag_node->children[0], new_dag, node_remap, node_to_step_map); const auto & new_node = new_dag->addAlias(alias_child, original_dag_node->result_name); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -125,7 +125,7 @@ const ActionsDAG::Node & addClonedDAGToDAG( } const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); - node_remap[node_name] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag.get(), &new_node}; return new_node; } @@ -133,13 +133,13 @@ const ActionsDAG::Node & addClonedDAGToDAG( } const ActionsDAG::Node & addFunction( - ActionsDAGPtr new_dag, + const ActionsDAGPtr & new_dag, const FunctionOverloadResolverPtr & function, ActionsDAG::NodeRawConstPtrs children, OriginalToNewNodeMap & node_remap) { const auto & new_node = new_dag->addFunction(function, children, ""); - node_remap[new_node.result_name] = {new_dag, &new_node}; + node_remap[new_node.result_name] = {new_dag.get(), &new_node}; return new_node; } @@ -147,7 +147,7 @@ const ActionsDAG::Node & addFunction( /// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, /// but it might be required for further steps with its original uncasted type. const ActionsDAG::Node & addCast( - ActionsDAGPtr dag, + const ActionsDAGPtr & dag, const ActionsDAG::Node & node_to_cast, const String & type_name, OriginalToNewNodeMap & node_remap) @@ -173,7 +173,7 @@ const ActionsDAG::Node & addCast( /// 1. produces a result with the proper Nullable or non-Nullable UInt8 type and /// 2. makes sure that the result contains only 0 or 1 values even if the source column contains non-boolean values. const ActionsDAG::Node & addAndTrue( - ActionsDAGPtr dag, + const ActionsDAGPtr & dag, const ActionsDAG::Node & filter_node_to_normalize, OriginalToNewNodeMap & node_remap) { @@ -258,7 +258,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction for (size_t step_index = 0; step_index < condition_groups.size(); ++step_index) { const auto & condition_group = condition_groups[step_index]; - ActionsDAGPtr step_dag = std::make_shared(); + ActionsDAGPtr step_dag = std::make_unique(); String result_name; std::vector new_condition_nodes; @@ -299,7 +299,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } } - steps.push_back({step_dag, result_name}); + steps.push_back({std::move(step_dag), result_name}); } /// 6. Find all outputs of the original DAG @@ -345,11 +345,11 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { for (size_t step_index = 0; step_index < steps.size(); ++step_index) { - const auto & step = steps[step_index]; + auto & step = steps[step_index]; PrewhereExprStep new_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(step.actions, actions_settings), + .actions = std::make_shared(std::move(step.actions), actions_settings), .filter_column_name = step.column_name, /// Don't remove if it's in the list of original outputs .remove_filter_column = diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 9de7b238f57..6df7b5aa054 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -4,7 +4,7 @@ namespace DB { -PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict) +PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, const ActionsDAG * filter_actions_dag, ContextPtr context, bool strict) : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */) , useless((strict && partition_condition.isRelaxed()) || partition_condition.alwaysUnknownOrTrue()) diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index ca24559ca01..d89dfb7b245 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -13,7 +13,7 @@ namespace DB class PartitionPruner { public: - PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict = false); + PartitionPruner(const StorageMetadataPtr & metadata, const ActionsDAG * filter_actions_dag, ContextPtr context, bool strict = false); bool canBePruned(const IMergeTreeDataPart & part) const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2fc6993369d..f640fb9ba0a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -79,7 +79,7 @@ StorageObjectStorageSource::~StorageObjectStorageSource() void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); + setKeyConditionImpl(filter_actions_dag.get(), context_, read_from_format_info.format_header); } std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 52b6674c93d..654b8b788fe 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -19,7 +19,7 @@ class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::shared_ptr; +using ActionsDAGPtr = std::unique_ptr; struct PrewhereInfo; using PrewhereInfoPtr = std::shared_ptr; @@ -192,7 +192,7 @@ struct SelectQueryInfo ASTPtr parallel_replica_custom_key_ast; /// Filter actions dag for current storage - ActionsDAGPtr filter_actions_dag; + std::shared_ptr filter_actions_dag; ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a3f6b6afc5d..9bddf4f0230 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->row_level_filter, actions_settings), + std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->prewhere_actions, actions_settings), + std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5048ef4788e..be421e8e2bc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1120,7 +1120,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) if (!source) return nullptr; - return source->getFilterActionsDAG(); + return source->detachFilterActionsDAG(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index aaf84f6f82c..702c257bfb6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1235,7 +1235,7 @@ StorageFileSource::~StorageFileSource() void StorageFileSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); + setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e2f92f08d7a..c42e3058347 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1235,7 +1235,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(actions_dag, + filter_actions = std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1273,12 +1273,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(actions_dag, filter_column_name); + step->addFilter(actions_dag->clone(), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag, filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag->clone(), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } @@ -1471,7 +1471,7 @@ void ReadFromMerge::convertAndFilterSourceStream( { pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - auto actions_dag = std::make_shared(pipe_columns); + auto actions_dag = std::make_unique(pipe_columns); QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context); query_tree->setAlias(alias.name); @@ -1486,7 +1486,7 @@ void ReadFromMerge::convertAndFilterSourceStream( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index f550ccb2bc4..c336f597f41 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -187,7 +187,7 @@ public: void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override { - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); + setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); } Chunk generate() override; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 894b1404a21..4d73f8e5c87 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -48,14 +48,14 @@ Pipe StorageValues::read( if (!prepared_pipe.empty()) { - auto dag = std::make_shared(prepared_pipe.getHeader().getColumnsWithTypeAndName()); + auto dag = std::make_unique(prepared_pipe.getHeader().getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(column_names.size()); for (const auto & name : column_names) outputs.push_back(dag->getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); dag->getOutputs().swap(outputs); - auto expression = std::make_shared(dag); + auto expression = std::make_shared(std::move(dag)); prepared_pipe.addSimpleTransform([&](const Block & header) { diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 016de94c17c..2c0d5c5ca85 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -177,7 +177,7 @@ void StorageView::read( /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - auto materializing_actions = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto materializing_actions = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); materializing_actions->addMaterializingOutputActions(); auto materializing = std::make_unique(query_plan.getCurrentDataStream(), std::move(materializing_actions)); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f831465277d..56f65b57367 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -183,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType dag->getOutputs() = {col}; dag->removeUnusedActions(); - result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context_copy)); + result.expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); return result; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..6f7d1d4c39f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,10 +77,10 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) } } -void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) +void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(dag); + auto actions = std::make_shared(dag->clone()); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..0cf8470bc60 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,7 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context); +void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context); /// Builds sets used by ActionsDAG inplace. void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8bca1c97aad..da4e751a88a 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -567,7 +567,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) builder.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(filter_expression), filter_function->getColumnName(), true); + header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); }); /// Adding window column @@ -592,7 +592,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) new_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); @@ -700,7 +700,7 @@ inline void StorageWindowView::fire(UInt32 watermark) getTargetTable()->getInMemoryMetadataPtr()->getColumns(), getContext(), getContext()->getSettingsRef().insert_null_as_default); - 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); @@ -711,7 +711,7 @@ inline void StorageWindowView::fire(UInt32 watermark) block_io.pipeline.getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared( - convert_actions_dag, + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); pipe.addSimpleTransform([&](const Block & stream_header) { @@ -1475,7 +1475,7 @@ void StorageWindowView::writeIntoWindowView( pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(filter_expression), + header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); }); } @@ -1583,7 +1583,7 @@ void StorageWindowView::writeIntoWindowView( output->getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto convert_actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); + std::move(convert_actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, convert_actions); }); } 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 0052/1170] 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 0053/1170] 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 e348186ba26072eb76e549cd2a0adcd801c92bc6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 18 Jun 2024 21:30:37 +0000 Subject: [PATCH 0054/1170] clear hint in table engine and sources --- src/Access/Common/AccessRightsElement.cpp | 6 +- src/Access/Common/AccessRightsElement.h | 1 + src/Access/ContextAccess.cpp | 80 ++++++++++++++++++----- 3 files changed, 70 insertions(+), 17 deletions(-) diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 24ff4e7631b..2ee13d6b94f 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -224,7 +224,11 @@ void AccessRightsElement::replaceEmptyDatabase(const String & current_database) String AccessRightsElement::toString() const { return toStringImpl(*this, true); } String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); } - +String AccessRightsElement::toStringWithoutONClause() const +{ + String result{access_flags.toKeywords().front()}; + return result + " ON {db.table}"; +} bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index ba625fc43df..49764fc727f 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -89,6 +89,7 @@ struct AccessRightsElement /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; + String toStringWithoutONClause() const; }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 2a658d7aaa2..3ce30a0b681 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -37,6 +37,24 @@ namespace ErrorCodes namespace { + static const std::vector> source_and_table_engines = { + {AccessType::FILE, "File"}, + {AccessType::URL, "URL"}, + {AccessType::REMOTE, "Distributed"}, + {AccessType::MONGO, "MongoDB"}, + {AccessType::REDIS, "Redis"}, + {AccessType::MYSQL, "MySQL"}, + {AccessType::POSTGRES, "PostgreSQL"}, + {AccessType::SQLITE, "SQLite"}, + {AccessType::ODBC, "ODBC"}, + {AccessType::JDBC, "JDBC"}, + {AccessType::HDFS, "HDFS"}, + {AccessType::S3, "S3"}, + {AccessType::HIVE, "Hive"}, + {AccessType::AZURE, "AzureBlobStorage"} + }; + + AccessRights mixAccessRightsFromUserAndRoles(const User & user, const EnabledRolesInfo & roles_info) { AccessRights res = user.access; @@ -205,22 +223,6 @@ namespace } /// There is overlap between AccessType sources and table engines, so the following code avoids user granting twice. - static const std::vector> source_and_table_engines = { - {AccessType::FILE, "File"}, - {AccessType::URL, "URL"}, - {AccessType::REMOTE, "Distributed"}, - {AccessType::MONGO, "MongoDB"}, - {AccessType::REDIS, "Redis"}, - {AccessType::MYSQL, "MySQL"}, - {AccessType::POSTGRES, "PostgreSQL"}, - {AccessType::SQLITE, "SQLite"}, - {AccessType::ODBC, "ODBC"}, - {AccessType::JDBC, "JDBC"}, - {AccessType::HDFS, "HDFS"}, - {AccessType::S3, "S3"}, - {AccessType::HIVE, "Hive"}, - {AccessType::AZURE, "AzureBlobStorage"} - }; /// Sync SOURCE and TABLE_ENGINE, so only need to check TABLE_ENGINE later. if (access_control.doesTableEnginesRequireGrant()) @@ -555,6 +557,18 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() return nothing_granted; } +/// Just Dummy to pass compile. +template +static std::string_view getTableEngineName(const Args &... args[[maybe_unused]]) +{ + return ""; +} + +template +static std::string_view getTableEngineName(std::string_view name, const Args &... args[[maybe_unused]]) +{ + return name; +} template bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const @@ -611,6 +625,40 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading since + /// SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. + if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) + { + AccessFlags newFlags; + + String table_engine_name{getTableEngineName(args...)}; + for (const auto & source_and_table_engine : source_and_table_engines) + { + const auto & table_engine = std::get<1>(source_and_table_engine); + if (table_engine != table_engine_name) continue; + const auto & source = std::get<0>(source_and_table_engine); + /// Set the flags from Table Engine to SOURCES so that prompts can be meaningful. + newFlags = source; + break; + } + + if (newFlags == AccessType::NONE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); + + if (grant_option && acs->isGranted(flags, args...)) + { + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. " + "The required privileges have been granted, but without grant option. " + "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", + AccessRightsElement{newFlags}.toStringWithoutONClause()); + } + + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", + AccessRightsElement{newFlags}.toStringWithoutONClause() + (grant_option ? " WITH GRANT OPTION" : "")); + } + if (grant_option && acs->isGranted(flags, args...)) { return access_denied(ErrorCodes::ACCESS_DENIED, From f7e81e1ae2752020c076990395349ccd2d69cf2b Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 18 Jun 2024 21:59:07 +0000 Subject: [PATCH 0055/1170] fix --- src/Access/ContextAccess.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 3ce30a0b681..de0e7e3d777 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -625,8 +625,8 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { - /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading since - /// SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading + /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) { AccessFlags newFlags; From a2ee0668f12c8cd1b88b8c4ad46c15271a5a1fd2 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 19 Jun 2024 02:20:22 +0000 Subject: [PATCH 0056/1170] fix --- src/Access/ContextAccess.cpp | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index de0e7e3d777..4620561053b 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -37,7 +37,7 @@ namespace ErrorCodes namespace { - static const std::vector> source_and_table_engines = { + const std::vector> source_and_table_engines = { {AccessType::FILE, "File"}, {AccessType::URL, "URL"}, {AccessType::REMOTE, "Distributed"}, @@ -268,6 +268,11 @@ namespace template std::string_view getDatabase(std::string_view arg1, const OtherArgs &...) { return arg1; } + + std::string_view getTableEngine() { return {}; } + + template + std::string_view getTableEngine(std::string_view arg1, const OtherArgs &...) { return arg1; } } @@ -557,18 +562,6 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() return nothing_granted; } -/// Just Dummy to pass compile. -template -static std::string_view getTableEngineName(const Args &... args[[maybe_unused]]) -{ - return ""; -} - -template -static std::string_view getTableEngineName(std::string_view name, const Args &... args[[maybe_unused]]) -{ - return name; -} template bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const @@ -631,7 +624,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg { AccessFlags newFlags; - String table_engine_name{getTableEngineName(args...)}; + String table_engine_name{getTableEngine(args...)}; for (const auto & source_and_table_engine : source_and_table_engines) { const auto & table_engine = std::get<1>(source_and_table_engine); @@ -642,7 +635,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg break; } - if (newFlags == AccessType::NONE) + if (newFlags.isEmpty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); if (grant_option && acs->isGranted(flags, args...)) From 8ca47905ef000d3eb72a89d9dfcd9b989fce4203 Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 19 Jun 2024 14:06:36 +0800 Subject: [PATCH 0057/1170] 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 0058/1170] 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 0059/1170] 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 0060/1170] 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 b125f8166f32648572c2ed0d540ded56a97ac628 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 19 Jun 2024 16:41:50 +0100 Subject: [PATCH 0061/1170] impl --- tests/performance/array_reduce.xml | 15 ++++++--------- tests/performance/scripts/perf.py | 14 ++++++++++++-- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/performance/array_reduce.xml b/tests/performance/array_reduce.xml index 8e271c155f3..0f50eba43bf 100644 --- a/tests/performance/array_reduce.xml +++ b/tests/performance/array_reduce.xml @@ -1,11 +1,8 @@ - - - - SELECT arrayReduce('count', range(100000000)) - SELECT arrayReduce('sum', range(100000000)) - SELECT arrayReduceInRanges('count', [(1, 100000000)], range(100000000)) - SELECT arrayReduceInRanges('sum', [(1, 100000000)], range(100000000)) - SELECT arrayReduceInRanges('count', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] - SELECT arrayReduceInRanges('sum', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] + SELECT arrayReduce('count', range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduce('sum', range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('count', [(1, 1000000)], range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('sum', [(1, 1000000)], range(1000000)) FROM numbers_mt(500000000) format Null + SELECT arrayReduceInRanges('count', arrayZip(range(1000000), range(1000000)), range(1000000))[123456] + SELECT arrayReduceInRanges('sum', arrayZip(range(1000000), range(1000000)), range(1000000))[123456] diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index 94f145d82db..f89784a0e0b 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -345,6 +345,18 @@ for query_index in queries_to_run: print(f"display-name\t{query_index}\t{tsv_escape(query_display_name)}") + for conn_index, c in enumerate(all_connections): + try: + c.execute("SYSTEM JEMALLOC PURGE") + + print( + f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}" + ) + except KeyboardInterrupt: + raise + except: + continue + # Prewarm: run once on both servers. Helps to bring the data into memory, # precompile the queries, etc. # A query might not run on the old server if it uses a function added in the @@ -427,8 +439,6 @@ for query_index in queries_to_run: for conn_index, c in enumerate(this_query_connections): try: - c.execute("SYSTEM JEMALLOC PURGE") - res = c.execute( q, query_id=run_id, From fdfa6adbfa9b44f0943a39f5188285aff4329640 Mon Sep 17 00:00:00 2001 From: skyoct Date: Thu, 20 Jun 2024 22:54:33 +0800 Subject: [PATCH 0062/1170] 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 0063/1170] 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 6f1f416700a32cf95af15b79543e27cbcffe2f14 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 17:37:05 +0100 Subject: [PATCH 0064/1170] one more test --- tests/performance/final_big_column.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 1fd586d2d90..4bfdfdf804f 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -1,6 +1,7 @@ - 1 + + 8 20G @@ -10,8 +11,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers_mt(5000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers_mt(5000000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null From 16baecf5a67083ecd23d621f10ec0c6250178e32 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 13 Jun 2024 22:01:57 +0000 Subject: [PATCH 0065/1170] attach_gdb.lib: print registers before all stacks --- docker/test/stateless/attach_gdb.lib | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index eb54f920b98..251fb886df6 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -25,8 +25,11 @@ handle SIG$RTMIN nostop noprint pass info signals continue backtrace full -thread apply all backtrace full info registers +p "top 1 KiB of the stack:" +p/x *(uint64_t[128]*)$sp +maintenance info sections +thread apply all backtrace full disassemble /s up disassemble /s From 557cd2c08f4791e10e1f2914bd4974c51531bf41 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 18:56:06 +0000 Subject: [PATCH 0066/1170] Escape the $ --- docker/test/stateless/attach_gdb.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index 251fb886df6..a3616ac1a04 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -27,7 +27,7 @@ continue backtrace full info registers p "top 1 KiB of the stack:" -p/x *(uint64_t[128]*)$sp +p/x *(uint64_t[128]*)"'$sp'" maintenance info sections thread apply all backtrace full disassemble /s From 3ab8ba0d4ab0e4446883bf57ba3d859d1ee49a52 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 27 May 2024 21:33:54 +0000 Subject: [PATCH 0067/1170] Update zlib-ng from 2.0.2 to 2.1.6 --- contrib/zlib-ng | 2 +- contrib/zlib-ng-cmake/CMakeLists.txt | 84 +++++++++++++++------------- 2 files changed, 47 insertions(+), 39 deletions(-) diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 50f0eae1a41..bfb184bb0fb 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 50f0eae1a411764cd6d1e85b3ce471438acd3c1c +Subproject commit bfb184bb0fbdabe82f3a36d209e56c3e2c33866a diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index 79f343bfc75..78a5f1ae3a8 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -14,6 +14,7 @@ add_definitions(-DHAVE_VISIBILITY_HIDDEN) add_definitions(-DHAVE_VISIBILITY_INTERNAL) add_definitions(-DHAVE_BUILTIN_CTZ) add_definitions(-DHAVE_BUILTIN_CTZLL) +add_definitions(-DHAVE_ATTRIBUTE_ALIGNED) set(ZLIB_ARCH_SRCS) set(ZLIB_ARCH_HDRS) @@ -26,14 +27,15 @@ if(ARCH_AARCH64) add_definitions(-DARM_FEATURES) add_definitions(-DARM_AUXV_HAS_CRC32 -DARM_ASM_HWCAP) add_definitions(-DARM_AUXV_HAS_NEON) - add_definitions(-DARM_ACLE_CRC_HASH) - add_definitions(-DARM_NEON_ADLER32 -DARM_NEON_CHUNKSET -DARM_NEON_SLIDEHASH) + add_definitions(-DARM_ACLE) + add_definitions(-DARM_NEON) - list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/arm.h) - list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/armfeature.c) + list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/arm_features.h) + list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/arm_features.c) set(ACLE_SRCS ${ARCHDIR}/crc32_acle.c ${ARCHDIR}/insert_string_acle.c) list(APPEND ZLIB_ARCH_SRCS ${ACLE_SRCS}) - set(NEON_SRCS ${ARCHDIR}/adler32_neon.c ${ARCHDIR}/chunkset_neon.c ${ARCHDIR}/slide_neon.c) + set(NEON_SRCS ${ARCHDIR}/adler32_neon.c ${ARCHDIR}/chunkset_neon.c + ${ARCHDIR}/compare256_neon.c ${ARCHDIR}/slide_hash_neon.c) list(APPEND ZLIB_ARCH_SRCS ${NEON_SRCS}) elseif(ARCH_PPC64LE) @@ -41,50 +43,47 @@ elseif(ARCH_PPC64LE) add_definitions(-DPOWER8) add_definitions(-DPOWER_FEATURES) - add_definitions(-DPOWER8_VSX_ADLER32) - add_definitions(-DPOWER8_VSX_SLIDEHASH) + add_definitions(-DPOWER8_VSX) + add_definitions(-DPOWER8_VSX_CRC32) - list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/power.h) - list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/power.c) - set(POWER8_SRCS ${ARCHDIR}/adler32_power8.c ${ARCHDIR}/slide_hash_power8.c) + list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/power_features.h) + list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/power_features.c) + set(POWER8_SRCS ${ARCHDIR}/adler32_power8.c ${ARCHDIR}/chunkset_power8.c ${ARCHDIR}/slide_hash_power8.c) + list(APPEND POWER8_SRCS ${ARCHDIR}/crc32_power8.c) list(APPEND ZLIB_ARCH_SRCS ${POWER8_SRCS}) elseif(ARCH_AMD64) set(ARCHDIR "${SOURCE_DIR}/arch/x86") add_definitions(-DX86_FEATURES) - list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/x86.h) - list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/x86.c) + list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/x86_features.h) + list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/x86_features.c) if(ENABLE_AVX2) - add_definitions(-DX86_AVX2 -DX86_AVX2_ADLER32 -DX86_AVX_CHUNKSET) - set(AVX2_SRCS ${ARCHDIR}/slide_avx.c) - list(APPEND AVX2_SRCS ${ARCHDIR}/chunkset_avx.c) - list(APPEND AVX2_SRCS ${ARCHDIR}/compare258_avx.c) - list(APPEND AVX2_SRCS ${ARCHDIR}/adler32_avx.c) + add_definitions(-DX86_AVX2) + set(AVX2_SRCS ${ARCHDIR}/slide_hash_avx2.c) + list(APPEND AVX2_SRCS ${ARCHDIR}/chunkset_avx2.c) + list(APPEND AVX2_SRCS ${ARCHDIR}/compare256_avx2.c) + list(APPEND AVX2_SRCS ${ARCHDIR}/adler32_avx2.c) list(APPEND ZLIB_ARCH_SRCS ${AVX2_SRCS}) endif() if(ENABLE_SSE42) - add_definitions(-DX86_SSE42_CRC_HASH) - set(SSE42_SRCS ${ARCHDIR}/insert_string_sse.c) - list(APPEND ZLIB_ARCH_SRCS ${SSE42_SRCS}) - add_definitions(-DX86_SSE42_CRC_INTRIN) - add_definitions(-DX86_SSE42_CMP_STR) - set(SSE42_SRCS ${ARCHDIR}/compare258_sse.c) + add_definitions(-DX86_SSE42) + set(SSE42_SRCS ${ARCHDIR}/adler32_sse42.c ${ARCHDIR}/insert_string_sse42.c) list(APPEND ZLIB_ARCH_SRCS ${SSE42_SRCS}) endif() if(ENABLE_SSSE3) - add_definitions(-DX86_SSSE3 -DX86_SSSE3_ADLER32) - set(SSSE3_SRCS ${ARCHDIR}/adler32_ssse3.c) + add_definitions(-DX86_SSSE3) + set(SSSE3_SRCS ${ARCHDIR}/adler32_ssse3.c ${ARCHDIR}/chunkset_ssse3.c) list(APPEND ZLIB_ARCH_SRCS ${SSSE3_SRCS}) endif() if(ENABLE_PCLMULQDQ) add_definitions(-DX86_PCLMULQDQ_CRC) - set(PCLMULQDQ_SRCS ${ARCHDIR}/crc_folding.c) + set(PCLMULQDQ_SRCS ${ARCHDIR}/crc32_pclmulqdq.c) list(APPEND ZLIB_ARCH_SRCS ${PCLMULQDQ_SRCS}) endif() - add_definitions(-DX86_SSE2 -DX86_SSE2_CHUNKSET -DX86_SSE2_SLIDEHASH) - set(SSE2_SRCS ${ARCHDIR}/chunkset_sse.c ${ARCHDIR}/slide_sse.c) + add_definitions(-DX86_SSE2) + set(SSE2_SRCS ${ARCHDIR}/chunkset_sse2.c ${ARCHDIR}/compare256_sse2.c ${ARCHDIR}/slide_hash_sse2.c) list(APPEND ZLIB_ARCH_SRCS ${SSE2_SRCS}) add_definitions(-DX86_NOCHECK_SSE2) endif () @@ -106,39 +105,45 @@ generate_cmakein(${SOURCE_DIR}/zconf.h.in ${CMAKE_CURRENT_BINARY_DIR}/zconf.h.cm set(ZLIB_SRCS ${SOURCE_DIR}/adler32.c + ${SOURCE_DIR}/adler32_fold.c ${SOURCE_DIR}/chunkset.c - ${SOURCE_DIR}/compare258.c + ${SOURCE_DIR}/compare256.c ${SOURCE_DIR}/compress.c - ${SOURCE_DIR}/crc32.c - ${SOURCE_DIR}/crc32_comb.c + ${SOURCE_DIR}/cpu_features.c + ${SOURCE_DIR}/crc32_braid.c + ${SOURCE_DIR}/crc32_braid_comb.c + ${SOURCE_DIR}/crc32_fold.c ${SOURCE_DIR}/deflate.c ${SOURCE_DIR}/deflate_fast.c + ${SOURCE_DIR}/deflate_huff.c ${SOURCE_DIR}/deflate_medium.c ${SOURCE_DIR}/deflate_quick.c + ${SOURCE_DIR}/deflate_rle.c ${SOURCE_DIR}/deflate_slow.c + ${SOURCE_DIR}/deflate_stored.c ${SOURCE_DIR}/functable.c ${SOURCE_DIR}/infback.c - ${SOURCE_DIR}/inffast.c ${SOURCE_DIR}/inflate.c ${SOURCE_DIR}/inftrees.c ${SOURCE_DIR}/insert_string.c + ${SOURCE_DIR}/insert_string_roll.c + ${SOURCE_DIR}/slide_hash.c ${SOURCE_DIR}/trees.c ${SOURCE_DIR}/uncompr.c ${SOURCE_DIR}/zutil.c +) + +set(ZLIB_GZFILE_SRCS ${SOURCE_DIR}/gzlib.c - ${SOURCE_DIR}/gzread.c + ${CMAKE_CURRENT_BINARY_DIR}/gzread.c ${SOURCE_DIR}/gzwrite.c ) -set(ZLIB_ALL_SRCS ${ZLIB_SRCS} ${ZLIB_ARCH_SRCS}) +set(ZLIB_ALL_SRCS ${ZLIB_SRCS} ${ZLIB_ARCH_SRCS} ${ZLIB_GZFILE_SRCS}) add_library(_zlib ${ZLIB_ALL_SRCS}) add_library(ch_contrib::zlib ALIAS _zlib) -# https://github.com/zlib-ng/zlib-ng/pull/733 -# This is disabed by default -add_compile_definitions(Z_TLS=__thread) - if(HAVE_UNISTD_H) SET(ZCONF_UNISTD_LINE "#if 1 /* was set to #if 1 by configure/cmake/etc */") else() @@ -153,6 +158,9 @@ endif() set(ZLIB_PC ${CMAKE_CURRENT_BINARY_DIR}/zlib.pc) configure_file(${SOURCE_DIR}/zlib.pc.cmakein ${ZLIB_PC} @ONLY) configure_file(${CMAKE_CURRENT_BINARY_DIR}/zconf.h.cmakein ${CMAKE_CURRENT_BINARY_DIR}/zconf.h @ONLY) +configure_file(${SOURCE_DIR}/zlib.h.in ${CMAKE_CURRENT_BINARY_DIR}/zlib.h @ONLY) +configure_file(${SOURCE_DIR}/zlib_name_mangling.h.in ${CMAKE_CURRENT_BINARY_DIR}/zlib_name_mangling.h @ONLY) +configure_file(${SOURCE_DIR}/gzread.c.in ${CMAKE_CURRENT_BINARY_DIR}/gzread.c @ONLY) # We should use same defines when including zlib.h as used when zlib compiled target_compile_definitions (_zlib PUBLIC ZLIB_COMPAT WITH_GZFILEOP) From 62d6e3d3396fbcbcc6f666ac9b7bc2b60fe828b2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 29 May 2024 00:19:25 +0000 Subject: [PATCH 0068/1170] Fix ARM --- contrib/zlib-ng-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index 78a5f1ae3a8..40397e14f20 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -28,7 +28,9 @@ if(ARCH_AARCH64) add_definitions(-DARM_AUXV_HAS_CRC32 -DARM_ASM_HWCAP) add_definitions(-DARM_AUXV_HAS_NEON) add_definitions(-DARM_ACLE) + add_definitions(-DHAVE_ARM_ACLE_H) add_definitions(-DARM_NEON) + add_definitions(-DARM_NEON_HASLD4) list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/arm_features.h) list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/arm_features.c) From 8115926eb61779d669509f16863730557882b3c8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 29 May 2024 04:49:19 +0000 Subject: [PATCH 0069/1170] Fix ARM some more --- contrib/zlib-ng-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index 40397e14f20..1ed1d98ed66 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -25,6 +25,7 @@ if(ARCH_AARCH64) set(ARCHDIR "${SOURCE_DIR}/arch/arm") add_definitions(-DARM_FEATURES) + add_definitions(-DHAVE_SYS_AUXV_H) add_definitions(-DARM_AUXV_HAS_CRC32 -DARM_ASM_HWCAP) add_definitions(-DARM_AUXV_HAS_NEON) add_definitions(-DARM_ACLE) From 06781efcb785a0504d9599e6a1446e235caef13d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 22:59:36 +0000 Subject: [PATCH 0070/1170] Switch to a cleaner update, hopefully fix builds --- contrib/zlib-ng | 2 +- contrib/zlib-ng-cmake/CMakeLists.txt | 13 ++++++++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/contrib/zlib-ng b/contrib/zlib-ng index bfb184bb0fb..c19ba056b7c 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit bfb184bb0fbdabe82f3a36d209e56c3e2c33866a +Subproject commit c19ba056b7cc8029bb80f509956090c7ded58032 diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index 1ed1d98ed66..91a8eb2e0a3 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -15,6 +15,7 @@ add_definitions(-DHAVE_VISIBILITY_INTERNAL) add_definitions(-DHAVE_BUILTIN_CTZ) add_definitions(-DHAVE_BUILTIN_CTZLL) add_definitions(-DHAVE_ATTRIBUTE_ALIGNED) +add_definitions(-DHAVE_POSIX_MEMALIGN) set(ZLIB_ARCH_SRCS) set(ZLIB_ARCH_HDRS) @@ -44,10 +45,16 @@ if(ARCH_AARCH64) elseif(ARCH_PPC64LE) set(ARCHDIR "${SOURCE_DIR}/arch/power") - add_definitions(-DPOWER8) add_definitions(-DPOWER_FEATURES) - add_definitions(-DPOWER8_VSX) - add_definitions(-DPOWER8_VSX_CRC32) + add_definitions(-DHAVE_SYS_AUXV_H) + + if(POWER9) + add_definitions(-DPOWER9) + else() + add_definitions(-DPOWER8) + add_definitions(-DPOWER8_VSX) + add_definitions(-DPOWER8_VSX_CRC32) + endif() list(APPEND ZLIB_ARCH_HDRS ${ARCHDIR}/power_features.h) list(APPEND ZLIB_ARCH_SRCS ${ARCHDIR}/power_features.c) From 91dc9a69d844d781c7d4f94ca01d0a9bbe1a1f29 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 21 Jun 2024 22:03:38 +0100 Subject: [PATCH 0071/1170] fix final_big_column --- tests/performance/final_big_column.xml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 4bfdfdf804f..bc7c3570db6 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -1,7 +1,6 @@ - - 8 + 1 20G @@ -11,8 +10,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers_mt(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers_mt(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(1000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(1000000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null From cb0a692ba061d910712f0144cb2b5308db5d033f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 21 Jun 2024 23:42:56 +0100 Subject: [PATCH 0072/1170] fix read_from_comp_parts --- tests/performance/read_from_comp_parts.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/performance/read_from_comp_parts.xml b/tests/performance/read_from_comp_parts.xml index a625075588a..055df51d12d 100644 --- a/tests/performance/read_from_comp_parts.xml +++ b/tests/performance/read_from_comp_parts.xml @@ -5,15 +5,16 @@ ORDER BY (c1, c2) SETTINGS min_rows_for_wide_part = 1000000000 AS SELECT * - FROM generateRandom('c1 UInt32, c2 UInt64, s1 String, arr1 Array(UInt32), c3 UInt64, s2 String', 0, 30, 30) + FROM generateRandom('c1 UInt32, c2 UInt64, s1 String, arr1 Array(UInt32), c3 UInt64, s2 String', 0, 5, 6) LIMIT 50000000 + SETTINGS max_insert_threads = 8 8 - SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1) + SELECT count() FROM mt_comp_parts WHERE NOT ignore(s1) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c2, s1, arr1, s2) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1, s1, c3) SELECT count() FROM mt_comp_parts WHERE NOT ignore(c1, c2, c3) 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 0073/1170] 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 0074/1170] 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 0075/1170] 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 051290e6c912108986c896916db087c71230a121 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 24 Jun 2024 12:26:46 +0000 Subject: [PATCH 0076/1170] Add throughput introspection for scheduler queues --- src/Common/EventRateMeter.h | 18 ++++++++++++++++-- src/Common/Scheduler/ISchedulerNode.h | 14 ++++++++++++++ src/Common/Scheduler/Nodes/FairPolicy.h | 3 +-- src/Common/Scheduler/Nodes/FifoQueue.h | 3 +-- src/Common/Scheduler/Nodes/PriorityPolicy.h | 3 +-- .../Scheduler/Nodes/SemaphoreConstraint.h | 3 +-- .../Scheduler/Nodes/ThrottlerConstraint.h | 3 +-- src/Common/Scheduler/SchedulerRoot.h | 3 +-- src/Storages/System/StorageSystemScheduler.cpp | 2 ++ 9 files changed, 38 insertions(+), 14 deletions(-) diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index 3a21a80ce8b..4c38d1d9371 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -14,8 +14,9 @@ namespace DB class EventRateMeter { public: - explicit EventRateMeter(double now, double period_) + explicit EventRateMeter(double now, double period_, double step_ = 0.0) : period(period_) + , step(step_) , half_decay_time(period * std::numbers::ln2) // for `ExponentiallySmoothedAverage::sumWeights()` to be equal to `1/period` { reset(now); @@ -38,7 +39,16 @@ public: if (now - period <= start) // precise counting mode events = ExponentiallySmoothedAverage(events.value + count, now); else // exponential smoothing mode - events.add(count, now, half_decay_time); + { + // Adding events too often lead to low precision due to smoothing too often, so we buffer new events and add them in steps + step_count += count; + if (step_start + step <= now) + { + events.add(step_count, now, half_decay_time); + step_start = now; + step_count = 0; + } + } } /// Compute average event rate throughout `[now - period, now]` period. @@ -58,16 +68,20 @@ public: void reset(double now) { start = now; + step_start = now; events = ExponentiallySmoothedAverage(); data_points = 0; } private: const double period; + const double step; // duration of a step const double half_decay_time; double start; // Instant in past without events before it; when measurement started or reset ExponentiallySmoothedAverage events; // Estimated number of events in the last `period` size_t data_points = 0; + double step_start; // start instant of the last step + double step_count = 0.0; // number of events accumulated since step start }; } diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 81b491b0eda..c051829e336 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -176,6 +178,14 @@ protected: /// Postponed to be handled in scheduler thread, so it is intended to be called from outside. void scheduleActivation(); + /// Helper for introspection metrics + void incrementDequeued(ResourceCost cost) + { + dequeued_requests++; + dequeued_cost += cost; + throughput.add(static_cast(clock_gettime_ns())/1e9, cost); + } + public: EventQueue * const event_queue; String basename; @@ -189,6 +199,10 @@ public: std::atomic dequeued_cost{0}; std::atomic canceled_cost{0}; std::atomic busy_periods{0}; + + /// Average dequeued_cost per second + /// WARNING: Should only be accessed from the scheduler thread, so that locking is not required + EventRateMeter throughput{static_cast(clock_gettime_ns())/1e9, 2, 1}; }; using SchedulerNodePtr = std::shared_ptr; diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index 0a4e55c253b..fba637e979e 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -188,8 +188,7 @@ public: if (request) { - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, heap_size > 0}; } } diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 9ec997c06d2..9fbc6d1ae65 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -59,8 +59,7 @@ public: if (requests.empty()) busy_periods++; queue_cost -= result->cost; - dequeued_requests++; - dequeued_cost += result->cost; + incrementDequeued(result->cost); return {result, !requests.empty()}; } diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 22a5155cfeb..91dc95600d5 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -122,8 +122,7 @@ public: if (request) { - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, !items.empty()}; } } diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 10fce536f5d..92c6af9db18 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -81,8 +81,7 @@ public: child_active = child_now_active; if (!active()) busy_periods++; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, active()}; } diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index f4a5795bb2b..56866336f50 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -89,8 +89,7 @@ public: child_active = child_now_active; if (!active()) busy_periods++; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, active()}; } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 7af42fdbbea..5307aadc3cc 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -162,8 +162,7 @@ public: if (request == nullptr) // Possible in case of request cancel, just retry continue; - dequeued_requests++; - dequeued_cost += request->cost; + incrementDequeued(request->cost); return {request, current != nullptr}; } } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 339a59e88a5..b42c807d6fc 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -31,6 +31,7 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() {"dequeued_requests", std::make_shared(), "The total number of resource requests dequeued from this node."}, {"canceled_requests", std::make_shared(), "The total number of resource requests canceled from this node."}, {"dequeued_cost", std::make_shared(), "The sum of costs (e.g. size in bytes) of all requests dequeued from this node."}, + {"throughput", std::make_shared(), "Current average throughput (dequeued cost per second)."}, {"canceled_cost", std::make_shared(), "The sum of costs (e.g. size in bytes) of all requests canceled from this node."}, {"busy_periods", std::make_shared(), "The total number of deactivations of this node."}, {"vruntime", std::make_shared(std::make_shared()), @@ -96,6 +97,7 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c res_columns[i++]->insert(node->dequeued_requests.load()); res_columns[i++]->insert(node->canceled_requests.load()); res_columns[i++]->insert(node->dequeued_cost.load()); + res_columns[i++]->insert(node->throughput.rate(static_cast(clock_gettime_ns())/1e9)); res_columns[i++]->insert(node->canceled_cost.load()); res_columns[i++]->insert(node->busy_periods.load()); From b0ac0327d4d38a918e9be8fc499038cc43cafb2c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 25 Jun 2024 11:48:29 +0000 Subject: [PATCH 0077/1170] Fix bug in EventRateMeter It was relying on ExponentiallySmoothedCounter::get() which is designed for specific 1 second time interval between points. Now sum of weights is computed separatly in `duration` field, giving very accurate measurements independent of interval. --- src/Common/EventRateMeter.h | 52 ++++++---------- src/Common/ProgressIndication.h | 2 +- src/Common/tests/gtest_event_rate_meter.cpp | 68 +++++++++++++++++++++ 3 files changed, 86 insertions(+), 36 deletions(-) create mode 100644 src/Common/tests/gtest_event_rate_meter.cpp diff --git a/src/Common/EventRateMeter.h b/src/Common/EventRateMeter.h index 4c38d1d9371..b8a9112428f 100644 --- a/src/Common/EventRateMeter.h +++ b/src/Common/EventRateMeter.h @@ -4,8 +4,6 @@ #include -#include - namespace DB { @@ -14,10 +12,10 @@ namespace DB class EventRateMeter { public: - explicit EventRateMeter(double now, double period_, double step_ = 0.0) + explicit EventRateMeter(double now, double period_, size_t heating_ = 0) : period(period_) - , step(step_) - , half_decay_time(period * std::numbers::ln2) // for `ExponentiallySmoothedAverage::sumWeights()` to be equal to `1/period` + , max_interval(period * 10) + , heating(heating_) { reset(now); } @@ -30,25 +28,11 @@ public: { // Remove data for initial heating stage that can present at the beginning of a query. // Otherwise it leads to wrong gradual increase of average value, turning algorithm into not very reactive. - if (count != 0.0 && ++data_points < 5) - { - start = events.time; - events = ExponentiallySmoothedAverage(); - } + if (count != 0.0 && data_points++ <= heating) + reset(events.time, data_points); - if (now - period <= start) // precise counting mode - events = ExponentiallySmoothedAverage(events.value + count, now); - else // exponential smoothing mode - { - // Adding events too often lead to low precision due to smoothing too often, so we buffer new events and add them in steps - step_count += count; - if (step_start + step <= now) - { - events.add(step_count, now, half_decay_time); - step_start = now; - step_count = 0; - } - } + duration.add(std::min(max_interval, now - duration.time), now, period); + events.add(count, now, period); } /// Compute average event rate throughout `[now - period, now]` period. @@ -59,29 +43,27 @@ public: add(now, 0); if (unlikely(now <= start)) return 0; - if (now - period <= start) // precise counting mode - return events.value / (now - start); - else // exponential smoothing mode - return events.get(half_decay_time); // equals to `events.value / period` + + // We do not use .get() because sum of weights will anyway be canceled out (optimization) + return events.value / duration.value; } - void reset(double now) + void reset(double now, size_t data_points_ = 0) { start = now; - step_start = now; events = ExponentiallySmoothedAverage(); - data_points = 0; + duration = ExponentiallySmoothedAverage(); + data_points = data_points_; } private: const double period; - const double step; // duration of a step - const double half_decay_time; + const double max_interval; + const size_t heating; double start; // Instant in past without events before it; when measurement started or reset - ExponentiallySmoothedAverage events; // Estimated number of events in the last `period` + ExponentiallySmoothedAverage duration; // Current duration of a period + ExponentiallySmoothedAverage events; // Estimated number of events in last `duration` seconds size_t data_points = 0; - double step_start; // start instant of the last step - double step_count = 0.0; // number of events accumulated since step start }; } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index a9965785889..d925077a072 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -91,7 +91,7 @@ private: bool write_progress_on_update = false; - EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 2'000'000'000 /*ns*/}; // average cpu utilization last 2 second + EventRateMeter cpu_usage_meter{static_cast(clock_gettime_ns()), 2'000'000'000 /*ns*/, 4}; // average cpu utilization last 2 second, skip first 4 points HostToTimesMap hosts_data; /// In case of all of the above: /// - clickhouse-local diff --git a/src/Common/tests/gtest_event_rate_meter.cpp b/src/Common/tests/gtest_event_rate_meter.cpp new file mode 100644 index 00000000000..91ceec5eef7 --- /dev/null +++ b/src/Common/tests/gtest_event_rate_meter.cpp @@ -0,0 +1,68 @@ +#include + +#include + +#include + + +TEST(EventRateMeter, ExponentiallySmoothedAverage) +{ + double target = 100.0; + + // The test is only correct for timestep of 1 second because of + // how sum of weights is implemented inside `ExponentiallySmoothedAverage` + double time_step = 1.0; + + for (double half_decay_time : { 0.1, 1.0, 10.0, 100.0}) + { + DB::ExponentiallySmoothedAverage esa; + + int steps = static_cast(half_decay_time * 30 / time_step); + for (int i = 1; i <= steps; ++i) + esa.add(target * time_step, i * time_step, half_decay_time); + double measured = esa.get(half_decay_time); + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } +} + +TEST(EventRateMeter, ConstantRate) +{ + double target = 100.0; + + for (double period : {0.1, 1.0, 10.0}) + { + for (double time_step : {0.001, 0.01, 0.1, 1.0}) + { + DB::EventRateMeter erm(0.0, period); + + int steps = static_cast(period * 30 / time_step); + for (int i = 1; i <= steps; ++i) + erm.add(i * time_step, target * time_step); + double measured = erm.rate(steps * time_step); + // std::cout << "T=" << period << " dt=" << time_step << " measured=" << measured << std::endl; + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } + } +} + +TEST(EventRateMeter, PreciseStart) +{ + double target = 100.0; + + for (double period : {0.1, 1.0, 10.0}) + { + for (double time_step : {0.001, 0.01, 0.1, 1.0}) + { + DB::EventRateMeter erm(0.0, period); + + int steps = static_cast(period / time_step); + for (int i = 1; i <= steps; ++i) + { + erm.add(i * time_step, target * time_step); + double measured = erm.rate(i * time_step); + // std::cout << "T=" << period << " dt=" << time_step << " measured=" << measured << std::endl; + ASSERT_LE(std::fabs(measured - target), 1e-5 * target); + } + } + } +} From 5d9d29e37086e19dba852097e7820aff83072a8d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 15:00:07 +0000 Subject: [PATCH 0078/1170] Fixing some crashes --- src/Interpreters/ActionsDAG.cpp | 15 ++++++++----- src/Interpreters/ActionsDAG.h | 5 +++-- src/Interpreters/ExpressionActions.cpp | 8 ++++--- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 14 ++++++------ src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Planner/Planner.cpp | 18 ++++++++------- src/Planner/PlannerJoinTree.cpp | 22 +++++++++---------- src/Planner/PlannerJoins.cpp | 4 ++-- src/Planner/PlannerWindowFunctions.cpp | 4 +++- src/Processors/QueryPlan/ExpressionStep.cpp | 8 +++---- src/Processors/QueryPlan/FilterStep.cpp | 6 ++--- .../Optimizations/distinctReadInOrder.cpp | 4 ++-- .../Optimizations/filterPushDown.cpp | 4 ++-- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- .../optimizePrimaryKeyCondition.cpp | 6 ++--- .../Optimizations/optimizeReadInOrder.cpp | 8 +++---- .../optimizeUseAggregateProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 6 ++--- .../Optimizations/removeRedundantDistinct.cpp | 4 ++-- .../QueryPlan/ReadFromMergeTree.cpp | 16 +++++++------- .../QueryPlan/SourceStepWithFilter.cpp | 8 +++---- src/Processors/QueryPlan/TotalsHavingStep.cpp | 8 +++---- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 ++-- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 ++-- src/Storages/SelectQueryInfo.h | 4 ++-- src/Storages/StorageBuffer.cpp | 8 +++---- src/Storages/StorageMerge.cpp | 6 ++--- src/Storages/VirtualColumnUtils.cpp | 4 ++-- 31 files changed, 112 insertions(+), 102 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 23e1e5ee152..c2626285235 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1246,17 +1246,20 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) return true; } -ActionsDAGPtr ActionsDAG::clone() const +ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) { std::unordered_map old_to_new_nodes; - return clone(old_to_new_nodes); + return ActionsDAG::clone(from, old_to_new_nodes); } -ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const +ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes) { + if (!from) + return nullptr; + auto actions = std::make_unique(); - for (const auto & node : nodes) + for (const auto & node : from->nodes) { auto & copy_node = actions->nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; @@ -1266,10 +1269,10 @@ ActionsDAGPtr ActionsDAG::clone(std::unordered_map & old_t for (auto & child : node.children) child = old_to_new_nodes[child]; - for (const auto & output_node : outputs) + for (const auto & output_node : from->outputs) actions->outputs.push_back(old_to_new_nodes[output_node]); - for (const auto & input_node : inputs) + for (const auto & input_node : from->inputs) actions->inputs.push_back(old_to_new_nodes[input_node]); return actions; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 4a840885b6a..7ca3d1c1b0d 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,8 +261,9 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - ActionsDAGPtr clone() const; - ActionsDAGPtr clone(std::unordered_map & old_to_new_nodes) const; + static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } + static ActionsDAGPtr clone(const ActionsDAG * from); + static ActionsDAGPtr clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes); static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 7cbf5afd763..2eca31fc75e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -53,7 +53,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio : project_inputs(project_inputs_) , settings(settings_) { - actions_dag = actions_dag_->clone(); + actions_dag = ActionsDAG::clone(actions_dag_); /// It's important to determine lazy executed nodes before compiling expressions. std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation); @@ -76,15 +76,17 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = actions_dag->clone(copy_map); + copy->actions_dag = ActionsDAG::clone(actions_dag.get(), copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; + for (const auto * input : copy->actions_dag->getInputs()) + copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name)); + copy->num_columns = num_columns; copy->required_columns = required_columns; - copy->input_positions = input_positions; copy->result_positions = result_positions; copy->sample_block = sample_block; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index be00e37c751..6b49365b492 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,7 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - auto dag = prewhere_dag_and_flags->dag.clone(); + auto dag = ActionsDAG::clone(&prewhere_dag_and_flags->dag); ExpressionActions( std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); @@ -1980,7 +1980,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - before_where->dag.clone(), + ActionsDAG::clone(&before_where->dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 64a17a7ba87..71eb7dc64f8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1300,7 +1300,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), result_columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); + ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(* ActionsDAG::clone(actions)), std::move(*conv_dag)); interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } @@ -2042,7 +2042,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(prewhere_info.row_level_filter->clone()), + std::make_shared(ActionsDAG::clone(prewhere_info.row_level_filter)), prewhere_info.row_level_column_name, true); }); } @@ -2050,7 +2050,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(prewhere_info.prewhere_actions->clone()), + header, std::make_shared(ActionsDAG::clone(prewhere_info.prewhere_actions)), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2578,7 +2578,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2752,7 +2752,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2770,7 +2770,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( ActionsDAGPtr dag; if (expression) { - dag = expression->dag.clone(); + dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2819,7 +2819,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - auto dag = expression->dag.clone(); + auto dag = ActionsDAG::clone(&expression->dag); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 1bb770bf561..704c5ce7d8b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1278,7 +1278,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = step->actions()->dag.clone(); + auto dag = ActionsDAG::clone(&step->actions()->dag); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. @@ -1286,7 +1286,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } else { - auto dag = step->actions()->dag.clone(); + auto dag = ActionsDAG::clone(&step->actions()->dag); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 681ae7e6ac4..dddb7531519 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -333,12 +333,12 @@ void addExpressionStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = expression_actions->dag.clone(); + auto actions = ActionsDAG::clone(&expression_actions->dag); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions.get()); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); + result_actions_to_execute.push_back(expression_step->getExpression().get()); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -348,15 +348,15 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = filter_analysis_result.filter_actions->dag.clone(); + auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - result_actions_to_execute.push_back(actions.get()); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), actions, filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); + result_actions_to_execute.push_back(where_step->getExpression().get()); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -556,11 +556,9 @@ void addTotalsHavingStep(QueryPlan & query_plan, ActionsDAGPtr actions; if (having_analysis_result.filter_actions) { - actions = having_analysis_result.filter_actions->dag.clone(); + actions = ActionsDAG::clone(&having_analysis_result.filter_actions->dag); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - - result_actions_to_execute.push_back(actions.get()); } auto totals_having_step = std::make_unique( @@ -573,6 +571,10 @@ void addTotalsHavingStep(QueryPlan & query_plan, settings.totals_mode, settings.totals_auto_threshold, need_finalize); + + if (having_analysis_result.filter_actions) + result_actions_to_execute.push_back(totals_having_step->getActions().get()); + query_plan.addStep(std::move(totals_having_step)); } @@ -1449,7 +1451,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(filters.filter_actions->clone()); + table_expression_data.setFilterActions(ActionsDAG::clone(filters.filter_actions)); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 918cfad703e..16b5e363bfd 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -646,7 +646,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions()->clone(); + table_expression_query_info.filter_actions_dag = ActionsDAG::clone(table_expression_data.getFilterActions()); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -831,7 +831,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions->clone()); + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(row_policy_filter_info.actions)); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) @@ -1178,17 +1178,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); - left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); - left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); - join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); + left_join_tree_query_plan.actions_dags.emplace_back(left_join_expressions_actions_step->getExpression().get()); left_plan.addStep(std::move(left_join_expressions_actions_step)); join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); + right_join_tree_query_plan.actions_dags.emplace_back(right_join_expressions_actions_step->getExpression().get()); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1434,7 +1433,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto result_plan = QueryPlan(); - if (join_algorithm->isFilled()) + bool is_filled_join = join_algorithm->isFilled(); + if (is_filled_join) { auto filled_join_step = std::make_unique( left_plan.getCurrentDataStream(), @@ -1586,8 +1586,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` - for (const auto * action_dag : right_join_tree_query_plan.actions_dags) - left_join_tree_query_plan.actions_dags.emplace_back(action_dag); + if (!is_filled_join) + for (const auto * action_dag : right_join_tree_query_plan.actions_dags) + left_join_tree_query_plan.actions_dags.emplace_back(action_dag); // if (join_clauses_and_actions.left_join_expressions_actions) // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); // if (join_clauses_and_actions.right_join_expressions_actions) @@ -1646,10 +1647,9 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); - join_tree_query_plan.actions_dags.push_back(array_join_action_dag.get()); - auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); + join_tree_query_plan.actions_dags.push_back(array_join_actions->getExpression().get()); plan.addStep(std::move(array_join_actions)); auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 45842c0d705..23b6a805ab9 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -588,10 +588,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = left_join_actions->clone(); + result.left_join_expressions_actions = ActionsDAG::clone(left_join_actions); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = right_join_actions->clone(); + result.right_join_expressions_actions = ActionsDAG::clone(right_join_actions); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 9deceeef9a3..b9e11578dbc 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -157,7 +157,9 @@ std::vector sortWindowDescriptions(const std::vector }; std::vector perm(window_descriptions.size()); - std::iota(perm.begin(), perm.end(), 0U); + for (size_t i = 0; i < perm.size(); ++i) + perm[i] = i; + ::sort(perm.begin(), perm.end(), comparator); return perm; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 90ac94a1ace..50bc2e1533e 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -30,13 +30,13 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDA input_stream_, ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(actions_dag_->clone()) + , actions_dag(ActionsDAG::clone(actions_dag_)) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(actions_dag->clone(), settings.getActionsSettings()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index ef9f1d17822..7883461f45a 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -49,7 +49,7 @@ FilterStep::FilterStep( , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { - actions_dag = actions_dag->clone(); + actions_dag = ActionsDAG::clone(actions_dag_); actions_dag->removeAliasesForFilter(filter_column_name); } @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 87e16b5a244..6cdc3cb4eb0 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -15,11 +15,11 @@ static ActionsDAGPtr buildActionsForPlanPath(std::vector & d if (dag_stack.empty()) return nullptr; - ActionsDAGPtr path_actions = dag_stack.back()->clone(); + ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = dag_stack.back()->clone(); + ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); path_actions->mergeInplace(std::move(*clone)); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index ff1cefff09a..f26cd79dd97 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -597,7 +597,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - filter->getExpression()->clone(), + ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -611,7 +611,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{filter->getExpression()->clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 35d8b1a35e4..4629bc0af53 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - expression->getExpression()->clone()); + ActionsDAG::clone(expression->getExpression())); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 13b691da888..afe1406b65f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = filter_step->getExpression()->clone(); + auto filter_expression = ActionsDAG::clone(filter_step->getExpression()); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index e57d3319076..e5ded92b105 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -18,15 +18,15 @@ void optimizePrimaryKeyCondition(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) - source_step_with_filter->addFilter(filter_step->getExpression()->clone(), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(ActionsDAG::clone(filter_step->getExpression()), filter_step->getFilterColumnName()); /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 8e782e68db8..a8bd98d7460 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -173,9 +173,9 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) { if (dag) - dag->mergeInplace(std::move(*expression->clone())); + dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); else - dag = expression->clone(); + dag = ActionsDAG::clone(expression); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 7c45ef48252..da057bd25c2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -77,7 +77,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = analysis_result.before_aggregation->dag.clone(); + info.before_aggregation = ActionsDAG::clone(&analysis_result.before_aggregation->dag); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index d8b40b22904..0e2ad96a419 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -67,9 +67,9 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea void QueryDAG::appendExpression(const ActionsDAGPtr & expression) { if (dag) - dag->mergeInplace(std::move(*expression->clone())); + dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); else - dag = expression->clone(); + dag = ActionsDAG::clone(expression); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -238,7 +238,7 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = dag->clone(); + projection_query_info.filter_actions_dag = ActionsDAG::clone(dag); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index d3c75c988e7..81a8a537830 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -70,11 +70,11 @@ namespace if (dag_stack.empty()) return nullptr; - ActionsDAGPtr path_actions = dag_stack.back()->clone(); + ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = dag_stack.back()->clone(); + ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); logActionsDAG("DAG to merge", clone); dag_stack.pop_back(); path_actions->mergeInplace(std::move(*clone)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e5370c1c130..d711b3e8472 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -798,7 +798,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone()); + auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -1211,7 +1211,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone()); + auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); if (prewhere_info) { @@ -1993,7 +1993,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); + auto sampling_actions = std::make_shared(ActionsDAG::clone(result.sampling.filter_expression.get())); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2031,7 +2031,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(result_projection->clone()); + auto projection_actions = std::make_shared(ActionsDAG::clone(result_projection)); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2126,7 +2126,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); expression->describeActions(format_settings.out, prefix); } @@ -2135,7 +2135,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); expression->describeActions(format_settings.out, prefix); } } @@ -2161,7 +2161,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2171,7 +2171,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index b91debc8239..79b225e7f93 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); + auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 45de6c31d24..19632b1862f 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(actions_dag_->clone()) + , actions_dag(ActionsDAG::clone(actions_dag_)) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(actions_dag->clone(), settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(actions_dag->clone()); + auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); map.add("Expression", expression->toTree()); } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 95267bc24e0..bbe57fc6441 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(interpolate_description->actions->clone()); + interpolate_actions = std::make_shared(ActionsDAG::clone(interpolate_description->actions)); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 7c65381b05b..8d4ef69b1b9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,7 +265,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = filter_dag->clone(); + auto filter_actions_dag = ActionsDAG::clone(filter_dag); const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); std::unordered_map node_to_result_node; @@ -319,7 +319,7 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node return index_hint.getActions()->getOutputs(); /// Import the DAG and map argument pointers. - ActionsDAGPtr actions_clone = index_hint.getActions()->clone(); + ActionsDAGPtr actions_clone = ActionsDAG::clone(index_hint.getActions()); chassert(storage); result_dag_or_null->mergeNodes(std::move(*actions_clone), storage); return *storage; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 8fa5b2cc955..e924f853524 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), + .actions = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), + .actions = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 654b8b788fe..0b7035504ae 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = row_level_filter->clone(); + prewhere_info->row_level_filter = ActionsDAG::clone(row_level_filter); if (prewhere_actions) - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9bddf4f0230..695b31d0c80 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -312,7 +312,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*actions_dag->clone()), + std::move(*ActionsDAG::clone(actions_dag)), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -321,7 +321,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*actions_dag->clone()), + std::move(*ActionsDAG::clone(actions_dag)), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings), + std::make_shared(ActionsDAG::clone(query_info.prewhere_info->row_level_filter), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings), + std::make_shared(ActionsDAG::clone(query_info.prewhere_info->prewhere_actions), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c42e3058347..d21a6dc20dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - filter_info.actions->clone(), + ActionsDAG::clone(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); @@ -1273,12 +1273,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(actions_dag->clone(), filter_column_name); + step->addFilter(ActionsDAG::clone(actions_dag), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag->clone(), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), ActionsDAG::clone(actions_dag), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 6f7d1d4c39f..1bd5e80a4f9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,7 +80,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(dag->clone()); + auto actions = std::make_shared(ActionsDAG::clone(dag)); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); @@ -318,7 +318,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = index_hint->getActions()->clone(); + auto index_hint_dag = ActionsDAG::clone(index_hint->getActions()); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag->getOutputs()) if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) From 068e1c55545dfc4e004b8b5970b3f237033444cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 15:59:00 +0000 Subject: [PATCH 0079/1170] Fixing build. --- src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 6ace1b3b5ce..97de69b1134 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -84,12 +84,12 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (child_actions->hasArrayJoin()) return 0; - auto actions = child_actions->clone(); + auto actions = ActionsDAG::clone(child_actions); const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName()); if (child_filter->removesFilterColumn()) removeFromOutputs(*actions, child_filter_node); - actions->mergeInplace(std::move(*parent_actions->clone())); + actions->mergeInplace(std::move(*ActionsDAG::clone(parent_actions))); const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); if (parent_filter->removesFilterColumn()) From 3149e51e9254b268c41ade796d3652d0c2dec8f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Jun 2024 17:36:07 +0000 Subject: [PATCH 0080/1170] Fix other crashes. --- src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5822e46f9f8..8f39f0da5af 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -563,11 +563,11 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) auto syntax_result = TreeRewriter(getContext()).analyze(filter_function, builder.getHeader().getNamesAndTypesList()); auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, getContext()).getActionsDAG(false); + auto filter_actions = std::make_shared(std::move(filter_expression)); builder.addSimpleTransform([&](const Block & header) { - return std::make_shared( - header, std::make_shared(std::move(filter_expression)), filter_function->getColumnName(), true); + return std::make_shared(header, filter_actions, filter_function->getColumnName(), true); }); /// Adding window column From 15b36c1ece293157b636a3efdb4727b73c58637f Mon Sep 17 00:00:00 2001 From: skyoct Date: Wed, 26 Jun 2024 22:28:31 +0800 Subject: [PATCH 0081/1170] 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 0082/1170] 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 0083/1170] 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 c998ec1e4f1b91f8ca20c2bd5a7acb6ac8d2e1b1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 29 Jun 2024 02:40:22 +0000 Subject: [PATCH 0084/1170] add test and better naming --- src/Access/Common/AccessRightsElement.cpp | 4 +- src/Access/Common/AccessRightsElement.h | 2 +- src/Access/ContextAccess.cpp | 10 +-- ...xml => config_with_table_engine_grant.xml} | 0 .../config_without_table_engine_grant.xml | 5 ++ ...est.py => test_with_table_engine_grant.py} | 2 +- .../test_without_table_engine_grant.py | 81 +++++++++++++++++++ 7 files changed, 95 insertions(+), 9 deletions(-) rename tests/integration/test_grant_and_revoke/configs/{config.xml => config_with_table_engine_grant.xml} (100%) create mode 100644 tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml rename tests/integration/test_grant_and_revoke/{test.py => test_with_table_engine_grant.py} (99%) create mode 100644 tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 2ee13d6b94f..63bda09a51b 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -224,10 +224,10 @@ void AccessRightsElement::replaceEmptyDatabase(const String & current_database) String AccessRightsElement::toString() const { return toStringImpl(*this, true); } String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); } -String AccessRightsElement::toStringWithoutONClause() const +String AccessRightsElement::toStringForAccessTypeSource() const { String result{access_flags.toKeywords().front()}; - return result + " ON {db.table}"; + return result + " ON *.*"; } bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 49764fc727f..78e94e6f2e4 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -89,7 +89,7 @@ struct AccessRightsElement /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; - String toStringWithoutONClause() const; + String toStringForAccessTypeSource() const; }; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 4620561053b..8ff1fc8ed21 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -622,7 +622,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) { - AccessFlags newFlags; + AccessFlags new_flags; String table_engine_name{getTableEngine(args...)}; for (const auto & source_and_table_engine : source_and_table_engines) @@ -631,11 +631,11 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (table_engine != table_engine_name) continue; const auto & source = std::get<0>(source_and_table_engine); /// Set the flags from Table Engine to SOURCES so that prompts can be meaningful. - newFlags = source; + new_flags = source; break; } - if (newFlags.isEmpty()) + if (new_flags.isEmpty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); if (grant_option && acs->isGranted(flags, args...)) @@ -644,12 +644,12 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg "{}: Not enough privileges. " "The required privileges have been granted, but without grant option. " "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", - AccessRightsElement{newFlags}.toStringWithoutONClause()); + AccessRightsElement{new_flags}.toStringForAccessTypeSource()); } return access_denied(ErrorCodes::ACCESS_DENIED, "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", - AccessRightsElement{newFlags}.toStringWithoutONClause() + (grant_option ? " WITH GRANT OPTION" : "")); + AccessRightsElement{new_flags}.toStringForAccessTypeSource() + (grant_option ? " WITH GRANT OPTION" : "")); } if (grant_option && acs->isGranted(flags, args...)) diff --git a/tests/integration/test_grant_and_revoke/configs/config.xml b/tests/integration/test_grant_and_revoke/configs/config_with_table_engine_grant.xml similarity index 100% rename from tests/integration/test_grant_and_revoke/configs/config.xml rename to tests/integration/test_grant_and_revoke/configs/config_with_table_engine_grant.xml diff --git a/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml b/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml new file mode 100644 index 00000000000..d3571f281f5 --- /dev/null +++ b/tests/integration/test_grant_and_revoke/configs/config_without_table_engine_grant.xml @@ -0,0 +1,5 @@ + + + false + + diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py similarity index 99% rename from tests/integration/test_grant_and_revoke/test.py rename to tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index e533cced1e4..25ca7913e4e 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -5,7 +5,7 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", - main_configs=["configs/config.xml"], + main_configs=["configs/config_with_table_engine_grant.xml"], user_configs=["configs/users.d/users.xml"], ) diff --git a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py new file mode 100644 index 00000000000..210bb8ec465 --- /dev/null +++ b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py @@ -0,0 +1,81 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=["configs/config_without_table_engine_grant.xml"], + user_configs=["configs/users.d/users.xml"], +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + + instance.query("CREATE DATABASE test") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + instance.query("DROP USER IF EXISTS A") + instance.query("DROP TABLE IF EXISTS test.table1") + + +def test_table_engine_and_source_grant(): + instance.query("DROP USER IF EXISTS A") + instance.query("CREATE USER A") + instance.query("GRANT CREATE TABLE ON test.table1 TO A") + + instance.query("GRANT POSTGRES ON *.* TO A") + + instance.query( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("DROP TABLE test.table1") + + instance.query("REVOKE POSTGRES ON *.* FROM A") + + assert "Not enough privileges" in instance.query_and_get_error( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + # expecting grant POSTGRES instead of grant PostgreSQL due to discrepancy between source access type and table engine + assert "grant POSTGRES ON *.*" in instance.query_and_get_error( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("GRANT SOURCES ON *.* TO A") + + instance.query( + """ + CREATE TABLE test.table1(a Integer) + engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy'); + """, + user="A", + ) + + instance.query("DROP TABLE test.table1") From 02928bb207348019a0968454e0f7d6baa6e511af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jun 2024 05:09:53 +0200 Subject: [PATCH 0085/1170] Use Atomic database by default in clickhouse-local --- programs/local/LocalServer.cpp | 12 ++++++------ src/Databases/DatabaseAtomic.cpp | 14 +++++++++++--- src/Databases/DatabaseAtomic.h | 1 + src/Databases/DatabaseOnDisk.cpp | 12 ++++++++++-- src/Databases/DatabaseOnDisk.h | 4 +++- src/Databases/DatabaseOrdinary.cpp | 2 +- 6 files changed, 32 insertions(+), 13 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 670dc378b97..27fe90f68cc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -48,7 +49,6 @@ #include #include #include -#include #include #include #include @@ -192,11 +192,11 @@ 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 databaseCombiner = std::make_shared(name_, context_); - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); - databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); + auto databaseCombiner = std::make_shared(name_, context); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context)); return databaseCombiner; } @@ -341,7 +341,7 @@ std::string LocalServer::getInitialCreateTableQuery() else table_structure = "(" + table_structure + ")"; - return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});", + return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});", table_name, table_structure, data_format, table_file); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ccab72cfbae..985d79773aa 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -49,9 +49,6 @@ 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_) @@ -59,6 +56,13 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, C { } +void DatabaseAtomic::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); @@ -95,6 +99,7 @@ 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(); @@ -309,6 +314,7 @@ 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 @@ -572,6 +578,7 @@ 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; @@ -663,4 +670,5 @@ void registerDatabaseAtomic(DatabaseFactory & factory) }; factory.registerDatabase("Atomic", create_fn); } + } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index b59edd479ba..26ab7657354 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -75,6 +75,7 @@ protected: using DetachedTables = std::unordered_map; [[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex); + void createDirectories(); void tryCreateMetadataSymlink(); virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index b8154372116..3a56c124726 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -170,7 +170,12 @@ DatabaseOnDisk::DatabaseOnDisk( , metadata_path(metadata_path_) , data_path(data_path_) { - fs::create_directories(local_context->getPath() + data_path); +} + + +void DatabaseOnDisk::createDirectories() +{ + fs::create_directories(std::filesystem::path(getContext()->getPath()) / data_path); fs::create_directories(metadata_path); } @@ -188,6 +193,8 @@ 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()); @@ -255,7 +262,6 @@ void DatabaseOnDisk::createTable( } commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context); - removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false); } @@ -283,6 +289,8 @@ 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)); diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 12656068643..00e7a2850b8 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 data_path + escapeForFileName(table_name) + "/"; } + String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; } String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); } String getMetadataPath() const override { return metadata_path; } @@ -99,6 +99,8 @@ protected: virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach); virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {} + void createDirectories(); + const String metadata_path; const String data_path; }; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 7d4bb07e8ef..5b5c09b039d 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -52,7 +52,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_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) + : DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_) { } From eea2d51cfe908767d6c5602a5c8f3137c757065b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 03:07:10 +0200 Subject: [PATCH 0086/1170] Fix fast test --- src/Databases/DatabaseAtomic.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 985d79773aa..c3e93a8a324 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -451,6 +451,9 @@ 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)) { From 79fc80a3ea25309ca7f1fe7cf61d1be56526cdbc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 03:57:48 +0200 Subject: [PATCH 0087/1170] Fix error --- src/Databases/DatabaseOnDisk.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 3a56c124726..832769fd043 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -611,6 +611,10 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const { + fs::path metadata_path = getMetadataPath(); + if (!fs::exists(metadata_path)) + return; + auto process_tmp_drop_metadata_file = [&](const String & file_name) { assert(getUUID() == UUIDHelpers::Nil); @@ -634,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat std::vector> metadata_files; fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it) { String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. From 4c9238a1dd9433bac7de920b6c0955ecd53a3df9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 03:58:27 +0200 Subject: [PATCH 0088/1170] Fix error --- src/Databases/DatabaseOnDisk.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 832769fd043..c6b4c38a656 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -611,7 +611,6 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const { - fs::path metadata_path = getMetadataPath(); if (!fs::exists(metadata_path)) return; From 2b6a47c13ee054ddf63db41445aab8a0b7187340 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 06:10:13 +0200 Subject: [PATCH 0089/1170] Fix errors --- src/Databases/DatabaseAtomic.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index c3e93a8a324..bebb645670b 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -189,6 +189,7 @@ 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); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index c6b4c38a656..faac4b23701 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -416,6 +416,7 @@ 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); From ed68a29c8b3e64fa42ff8ea4f445789f72d39d46 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 06:34:33 +0200 Subject: [PATCH 0090/1170] Fix error --- src/Databases/DatabasesOverlay.cpp | 12 ++++++++++++ src/Databases/DatabasesOverlay.h | 1 + 2 files changed, 13 insertions(+) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 801356b3dd7..02a0aab8230 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -178,6 +178,18 @@ 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 b0c7e7e4032..5f6d4e601d3 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -41,6 +41,7 @@ public: 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; From 229ff9af6e1c89b8d6c3e0f29945d0baccde273a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 06:34:54 +0200 Subject: [PATCH 0091/1170] Remove old comment --- src/Interpreters/StorageID.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index f9afbc7b98d..ad55d16e284 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -27,7 +27,6 @@ class ASTQueryWithTableAndOutput; class ASTTableIdentifier; class Context; -// TODO(ilezhankin): refactor and merge |ASTTableIdentifier| struct StorageID { String database_name; From f58f6cfa737b1f18bab76ac8869f769b1908ea09 Mon Sep 17 00:00:00 2001 From: skyoct Date: Mon, 1 Jul 2024 18:26:54 +0800 Subject: [PATCH 0092/1170] 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 679da847da0b9593d5187997ef55116424b63b8e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Jul 2024 05:32:12 +0200 Subject: [PATCH 0093/1170] Fix tests --- src/Databases/DatabaseAtomic.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index bebb645670b..c06fc98d0b9 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -195,6 +195,9 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ 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); From 9be404c9c7b1e14410928a4aef8396664d1e364e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Jul 2024 18:02:57 +0000 Subject: [PATCH 0094/1170] Fix another case. --- src/Planner/PlannerJoinTree.cpp | 2 +- src/Storages/StorageMerge.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 16b5e363bfd..604d3366484 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1383,11 +1383,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_clauses_and_actions.mixed_join_expressions_actions) { - left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); + left_join_tree_query_plan.actions_dags.push_back(&mixed_join_expression->getActionsDAG()); } } else if (join_node.isUsingJoinExpression()) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5c5ce4ecc2e..c3fdad3a8f2 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1240,7 +1240,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(std::move(actions_dag), + filter_actions = std::make_shared(ActionsDAG::clone(actions_dag), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); From 57818990f201562d0b6938c1b8de78d16bac471f Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 2 Jul 2024 20:41:08 +0000 Subject: [PATCH 0095/1170] fix the test --- src/Access/ContextAccess.cpp | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 8ff1fc8ed21..f534c334318 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -618,6 +618,22 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (!granted) { + auto access_denied_no_grant = [&](AccessFlags access_flags, FmtArgs && ...fmt_args) + { + if (grant_option && acs->isGranted(access_flags, fmt_args...)) + { + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. " + "The required privileges have been granted, but without grant option. " + "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", + AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions()); + } + + return access_denied(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", + AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : "")); + }; + /// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading /// since SOURCES is not granted actually. In order to solve this, turn the prompt logic back to Sources. if (flags & AccessType::TABLE_ENGINE && !access_control->doesTableEnginesRequireGrant()) @@ -635,8 +651,9 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg break; } + /// Might happen in the case of grant Table Engine on A (but not source), then revoke A. if (new_flags.isEmpty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Didn't find the target Source from the Table Engine"); + return access_denied_no_grant(flags, args...); if (grant_option && acs->isGranted(flags, args...)) { @@ -652,18 +669,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg AccessRightsElement{new_flags}.toStringForAccessTypeSource() + (grant_option ? " WITH GRANT OPTION" : "")); } - if (grant_option && acs->isGranted(flags, args...)) - { - return access_denied(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. " - "The required privileges have been granted, but without grant option. " - "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", - AccessRightsElement{flags, args...}.toStringWithoutOptions()); - } - - return access_denied(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", - AccessRightsElement{flags, args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : "")); + return access_denied_no_grant(flags, args...); } struct PrecalculatedFlags From 9df30e4a0fe501c8e5eb1815e7a01c73d0c37ad2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 3 Jul 2024 02:02:24 +0000 Subject: [PATCH 0096/1170] Fix 'Not-ready Set is passed' in system tables --- src/Interpreters/PreparedSets.h | 15 ++++++++++--- src/Planner/Planner.cpp | 2 +- src/Storages/RocksDB/StorageSystemRocksDB.cpp | 8 +++++++ src/Storages/RocksDB/StorageSystemRocksDB.h | 1 + src/Storages/StorageMergeTreeIndex.cpp | 22 ++++++++++++++----- src/Storages/StorageMergeTreeIndex.h | 2 +- .../System/IStorageSystemOneBlock.cpp | 18 ++++++++++++--- src/Storages/System/IStorageSystemOneBlock.h | 8 +++++++ src/Storages/System/StorageSystemColumns.cpp | 20 +++++++++++++---- .../StorageSystemDataSkippingIndices.cpp | 17 +++++++++++--- .../System/StorageSystemDatabases.cpp | 8 +++++++ src/Storages/System/StorageSystemDatabases.h | 1 + .../System/StorageSystemDistributionQueue.cpp | 7 ++++++ .../System/StorageSystemDistributionQueue.h | 1 + .../System/StorageSystemMutations.cpp | 7 ++++++ src/Storages/System/StorageSystemMutations.h | 1 + .../StorageSystemPartMovesBetweenShards.cpp | 8 +++++++ .../StorageSystemPartMovesBetweenShards.h | 1 + src/Storages/System/StorageSystemReplicas.cpp | 19 +++++++++++++--- .../System/StorageSystemReplicationQueue.cpp | 8 +++++++ .../System/StorageSystemReplicationQueue.h | 1 + src/Storages/VirtualColumnUtils.h | 9 ++++++++ .../0_stateless/02841_not_ready_set_bug.sh | 17 ++++++++++++++ 23 files changed, 177 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index bf99a8ece3c..a6aee974d0e 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -90,9 +90,18 @@ private: using FutureSetFromTuplePtr = std::shared_ptr; -/// Set from subquery can be built inplace for PK or in CreatingSet step. -/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created, -/// but ordinary set would be created instead. +/// Set from subquery can be filled (by running the subquery) in one of two ways: +/// 1. During query analysis. Specifically, inside `SourceStepWithFilter::applyFilters()`. +/// Useful if the query plan depends on the set contents, e.g. to determine which files to read. +/// 2. During query execution. This is the preferred way. +/// Sets are created by CreatingSetStep, which runs before other steps. +/// Be careful: to build the set during query analysis, the `buildSetInplace()` call must happen +/// inside `SourceStepWithFilter::applyFilters()`. Calling it later, e.g. from `initializePipeline()` +/// will result in LOGICAL_ERROR "Not-ready Set is passed" (because a CreatingSetStep was already +/// added to pipeline but hasn't executed yet). +/// +/// If use_index_for_in_with_subqueries_max_values is reached, the built set won't be suitable for +/// key analysis, but will work with function IN (the set will contain only hashes of elements). class FutureSetFromSubquery final : public FutureSet { public: diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2d42ed73223..8d855b8e619 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1103,7 +1103,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( auto query_tree = subquery->detachQueryTree(); auto subquery_options = select_query_options.subquery(); /// I don't know if this is a good decision, - /// But for now it is done in the same way as in old analyzer. + /// but for now it is done in the same way as in old analyzer. /// This would not ignore limits for subqueries (affects mutations only). /// See test_build_sets_from_multiple_threads-analyzer. subquery_options.ignore_limits = false; diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index 5105b190fd9..b6cd58f58cc 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -40,6 +40,14 @@ ColumnsDescription StorageSystemRocksDB::getColumnsDescription() } +Block StorageSystemRocksDB::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "database" }, + { {}, std::make_shared(), "table" }, + }; +} + void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.h b/src/Storages/RocksDB/StorageSystemRocksDB.h index ec351c75446..be3bfaa860c 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.h +++ b/src/Storages/RocksDB/StorageSystemRocksDB.h @@ -22,6 +22,7 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index 0b1ad02f8c9..90d01d356e9 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -275,7 +275,7 @@ public: private: std::shared_ptr storage; Poco::Logger * log; - const ActionsDAG::Node * predicate = nullptr; + ActionsDAGPtr virtual_columns_filter; }; void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) @@ -283,7 +283,17 @@ void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); + { + Block block_to_filter + { + { {}, std::make_shared(), StorageMergeTreeIndex::part_name_column.name }, + }; + + virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + + if (virtual_columns_filter) + VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + } } void StorageMergeTreeIndex::read( @@ -335,7 +345,7 @@ void StorageMergeTreeIndex::read( void ReadFromMergeTreeIndex::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto filtered_parts = storage->getFilteredDataParts(predicate, context); + auto filtered_parts = storage->getFilteredDataParts(virtual_columns_filter, context); LOG_DEBUG(log, "Reading index{}from {} parts of table {}", storage->with_marks ? " with marks " : " ", @@ -345,9 +355,9 @@ void ReadFromMergeTreeIndex::initializePipeline(QueryPipelineBuilder & pipeline, pipeline.init(Pipe(std::make_shared(getOutputStream().header, storage->key_sample_block, std::move(filtered_parts), context, storage->with_marks))); } -MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(const ActionsDAG::Node * predicate, const ContextPtr & context) const +MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(ActionsDAGPtr virtual_columns_filter, const ContextPtr & context) const { - if (!predicate) + if (!virtual_columns_filter) return data_parts; auto all_part_names = ColumnString::create(); @@ -355,7 +365,7 @@ MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(const all_part_names->insert(part->name); Block filtered_block{{std::move(all_part_names), std::make_shared(), part_name_column.name}}; - VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, filtered_block, context); if (!filtered_block.rows()) return {}; diff --git a/src/Storages/StorageMergeTreeIndex.h b/src/Storages/StorageMergeTreeIndex.h index a1fb61d5a56..652a2d6eeaf 100644 --- a/src/Storages/StorageMergeTreeIndex.h +++ b/src/Storages/StorageMergeTreeIndex.h @@ -36,7 +36,7 @@ public: private: friend class ReadFromMergeTreeIndex; - MergeTreeData::DataPartsVector getFilteredDataParts(const ActionsDAG::Node * predicate, const ContextPtr & context) const; + MergeTreeData::DataPartsVector getFilteredDataParts(ActionsDAGPtr virtual_columns_filter, const ContextPtr & context) const; StoragePtr source_table; bool with_marks; diff --git a/src/Storages/System/IStorageSystemOneBlock.cpp b/src/Storages/System/IStorageSystemOneBlock.cpp index 456b7c4f90b..7cde31905aa 100644 --- a/src/Storages/System/IStorageSystemOneBlock.cpp +++ b/src/Storages/System/IStorageSystemOneBlock.cpp @@ -5,6 +5,7 @@ // #include #include #include +#include #include #include #include @@ -44,7 +45,7 @@ public: private: std::shared_ptr storage; std::vector columns_mask; - const ActionsDAG::Node * predicate = nullptr; + ActionsDAGPtr filter; }; void IStorageSystemOneBlock::read( @@ -81,6 +82,7 @@ void ReadFromSystemOneBlock::initializePipeline(QueryPipelineBuilder & pipeline, { const auto & sample_block = getOutputStream().header; MutableColumns res_columns = sample_block.cloneEmptyColumns(); + auto predicate = filter ? filter->getOutputs().at(0) : nullptr; storage->fillData(res_columns, context, predicate, std::move(columns_mask)); UInt64 num_rows = res_columns.at(0)->size(); @@ -93,8 +95,18 @@ void ReadFromSystemOneBlock::applyFilters(ActionDAGNodes added_filter_nodes) { SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); + if (!filter_actions_dag) + return; + + Block sample = storage->getFilterSampleBlock(); + if (sample.columns() == 0) + return; + + filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &sample); + + /// Must prepare sets here, initializePipeline() would be too late, see comment on FutureSetFromSubquery. + if (filter) + VirtualColumnUtils::buildSetsForDAG(filter, context); } } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index a20434fd97e..a47875c2537 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -22,8 +22,16 @@ class Context; class IStorageSystemOneBlock : public IStorage { protected: + /// If this method uses `predicate`, getFilterSampleBlock() must list all columns to which + /// it's applied. (Otherwise there'll be a LOGICAL_ERROR "Not-ready Set is passed" on subqueries.) virtual void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const = 0; + /// Columns to which fillData() applies the `predicate`. + virtual Block getFilterSampleBlock() const + { + return {}; + } + virtual bool supportsColumnsMask() const { return false; } friend class ReadFromSystemOneBlock; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 8dd8d3b6154..9502a7ee2a3 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -337,7 +337,7 @@ private: std::shared_ptr storage; std::vector columns_mask; const size_t max_block_size; - const ActionsDAG::Node * predicate = nullptr; + ActionsDAGPtr virtual_columns_filter; }; void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) @@ -345,7 +345,17 @@ void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); + { + Block block_to_filter; + block_to_filter.insert(ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "database")); + block_to_filter.insert(ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "table")); + + virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + + /// Must prepare sets here, initializePipeline() would be too late, see comment on FutureSetFromSubquery. + if (virtual_columns_filter) + VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + } } void StorageSystemColumns::read( @@ -407,7 +417,8 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared(), "database")); /// Filter block with `database` column. - VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithPredicate(virtual_columns_filter->getOutputs().at(0), block_to_filter, context); if (!block_to_filter.rows()) { @@ -455,7 +466,8 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, } /// Filter block with `database` and `table` columns. - VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context); + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, block_to_filter, context); if (!block_to_filter.rows()) { diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 093adc59cc6..a6bba44e257 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -214,7 +214,7 @@ private: std::shared_ptr storage; std::vector columns_mask; const size_t max_block_size; - const ActionsDAG::Node * predicate = nullptr; + ActionsDAGPtr virtual_columns_filter; }; void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter_nodes) @@ -222,7 +222,17 @@ void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); + { + Block block_to_filter + { + { ColumnString::create(), std::make_shared(), "database" }, + }; + + virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + + if (virtual_columns_filter) + VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + } } void StorageSystemDataSkippingIndices::read( @@ -268,7 +278,8 @@ void ReadFromSystemDataSkippingIndices::initializePipeline(QueryPipelineBuilder /// Condition on "database" in a query acts like an index. Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, block, context); ColumnPtr & filtered_databases = block.getByPosition(0).column; pipeline.init(Pipe(std::make_shared( diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 1dbb187c418..0585506a661 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -73,6 +73,14 @@ static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database return engine_full; } +Block StorageSystemDatabases::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "engine" }, + { {}, std::make_shared(), "uuid" }, + }; +} + static ColumnPtr getFilteredDatabases(const Databases & databases, const ActionsDAG::Node * predicate, ContextPtr context) { MutableColumnPtr name_column = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemDatabases.h b/src/Storages/System/StorageSystemDatabases.h index fa55f0aea32..d10b350435b 100644 --- a/src/Storages/System/StorageSystemDatabases.h +++ b/src/Storages/System/StorageSystemDatabases.h @@ -27,6 +27,7 @@ protected: bool supportsColumnsMask() const override { return true; } void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector columns_mask) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index e2058448904..dab318a9c1c 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -107,6 +107,13 @@ ColumnsDescription StorageSystemDistributionQueue::getColumnsDescription() }; } +Block StorageSystemDistributionQueue::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "database" }, + { {}, std::make_shared(), "table" }, + }; +} void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { diff --git a/src/Storages/System/StorageSystemDistributionQueue.h b/src/Storages/System/StorageSystemDistributionQueue.h index 159a86bf082..27d777a4762 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.h +++ b/src/Storages/System/StorageSystemDistributionQueue.h @@ -22,6 +22,7 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 94656008029..df9a71310e5 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -46,6 +46,13 @@ ColumnsDescription StorageSystemMutations::getColumnsDescription() }; } +Block StorageSystemMutations::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "database" }, + { {}, std::make_shared(), "table" }, + }; +} void StorageSystemMutations::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { diff --git a/src/Storages/System/StorageSystemMutations.h b/src/Storages/System/StorageSystemMutations.h index c60157cd853..5341838a65e 100644 --- a/src/Storages/System/StorageSystemMutations.h +++ b/src/Storages/System/StorageSystemMutations.h @@ -22,6 +22,7 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 9cba92bca12..ab74b205a96 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -43,6 +43,14 @@ ColumnsDescription StorageSystemPartMovesBetweenShards::getColumnsDescription() } +Block StorageSystemPartMovesBetweenShards::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "database" }, + { {}, std::make_shared(), "table" }, + }; +} + void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.h b/src/Storages/System/StorageSystemPartMovesBetweenShards.h index 6a859d4de80..bc6133fcaaa 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.h +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.h @@ -20,6 +20,7 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 3bd5fd290db..9fb4dc5ed6f 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -285,7 +285,7 @@ private: const bool with_zk_fields; const size_t max_block_size; std::shared_ptr impl; - const ActionsDAG::Node * predicate = nullptr; + ActionsDAGPtr virtual_columns_filter; }; void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) @@ -293,7 +293,19 @@ void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); + { + Block block_to_filter + { + { ColumnString::create(), std::make_shared(), "database" }, + { ColumnString::create(), std::make_shared(), "table" }, + { ColumnString::create(), std::make_shared(), "engine" }, + }; + + virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + + if (virtual_columns_filter) + VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + } } void StorageSystemReplicas::read( @@ -430,7 +442,8 @@ void ReadFromSystemReplicas::initializePipeline(QueryPipelineBuilder & pipeline, { col_engine, std::make_shared(), "engine" }, }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, filtered_block, context); if (!filtered_block.rows()) { diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 14b641f46c7..a50982de5f0 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -62,6 +62,14 @@ ColumnsDescription StorageSystemReplicationQueue::getColumnsDescription() } +Block StorageSystemReplicationQueue::getFilterSampleBlock() const +{ + return { + { {}, std::make_shared(), "database" }, + { {}, std::make_shared(), "table" }, + }; +} + void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); diff --git a/src/Storages/System/StorageSystemReplicationQueue.h b/src/Storages/System/StorageSystemReplicationQueue.h index a9e57851be1..bcf351381ee 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.h +++ b/src/Storages/System/StorageSystemReplicationQueue.h @@ -21,6 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; + Block getFilterSampleBlock() const override; }; } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..ebec807189e 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -18,6 +18,15 @@ class NamesAndTypesList; namespace VirtualColumnUtils { +/// The filtering functions are tricky to use correctly. +/// There are 2 ways: +/// 1. Call filterBlockWithPredicate() or filterBlockWithDAG() inside SourceStepWithFilter::applyFilters(). +/// 2. Call splitFilterDagForAllowedInputs() and buildSetsForDAG() inside SourceStepWithFilter::applyFilters(). +/// Then call filterBlockWithPredicate() or filterBlockWithDAG() in initializePipeline(). +/// +/// Otherwise calling filter*() outside applyFilters() will throw "Not-ready Set is passed" +/// if there are subqueries. + /// Similar to filterBlockWithQuery, but uses ActionsDAG as a predicate. /// Basically it is filterBlockWithDAG(splitFilterDagForAllowedInputs). void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); diff --git a/tests/queries/0_stateless/02841_not_ready_set_bug.sh b/tests/queries/0_stateless/02841_not_ready_set_bug.sh index 9b2f3b0698e..556e2f52de2 100755 --- a/tests/queries/0_stateless/02841_not_ready_set_bug.sh +++ b/tests/queries/0_stateless/02841_not_ready_set_bug.sh @@ -11,3 +11,20 @@ $CLICKHOUSE_CLIENT --max_threads=2 --max_result_rows=1 --result_overflow_mode=br $CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE 1 in (SELECT number from numbers(2)) AND database = currentDatabase() format Null" $CLICKHOUSE_CLIENT -q "SELECT xor(1, 0) FROM system.parts WHERE 1 IN (SELECT 1) FORMAT Null" + +# (Not all of these tests are effective because some of these tables are empty.) +$CLICKHOUSE_CLIENT -nq " + select * from system.columns where table in (select '123'); + select * from system.replicas where database in (select '123'); + select * from system.data_skipping_indices where database in (select '123'); + select * from system.databases where name in (select '123'); + select * from system.mutations where table in (select '123'); + select * from system.part_moves_between_shards where database in (select '123'); + select * from system.replication_queue where database in (select '123'); + select * from system.distribution_queue where database in (select '123'); +" +$CLICKHOUSE_CLIENT -nq " + create table a (x Int8) engine MergeTree order by x; + insert into a values (1); + select * from mergeTreeIndex(currentDatabase(), 'a') where part_name in (select '123'); +" From 8a8170d00c90545bea3e8e4881feee7b8a7fb4b7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 3 Jul 2024 03:19:31 +0000 Subject: [PATCH 0097/1170] Style --- src/Storages/System/StorageSystemReplicationQueue.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemReplicationQueue.h b/src/Storages/System/StorageSystemReplicationQueue.h index bcf351381ee..82a4d68f300 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.h +++ b/src/Storages/System/StorageSystemReplicationQueue.h @@ -21,7 +21,7 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const override; - Block getFilterSampleBlock() const override; + Block getFilterSampleBlock() const override; }; } From 6a06024983a78aaab9b7cbe6e9533255debebdb3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jul 2024 10:25:40 +0000 Subject: [PATCH 0098/1170] Fix for prewhere optimization. --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index afe1406b65f..f203d831750 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -59,7 +59,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans. QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node; - const auto * filter_step = typeid_cast(filter_node->step.get()); + auto * filter_step = typeid_cast(filter_node->step.get()); if (!filter_step) return; @@ -108,7 +108,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->need_filter = true; prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); - auto filter_expression = ActionsDAG::clone(filter_step->getExpression()); + auto filter_expression = std::move(filter_step->getExpression()); const auto & filter_column_name = filter_step->getFilterColumnName(); if (prewhere_info->remove_prewhere_column) @@ -121,7 +121,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true); + auto split_result = filter_expression->split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. From 41c62ca6636572b4d7654dc0dc329740bb5c7425 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Jul 2024 14:44:47 +0000 Subject: [PATCH 0099/1170] better --- src/Planner/Planner.cpp | 91 +++++++++++---------------------- src/Planner/PlannerJoinTree.cpp | 25 ++++----- src/Planner/PlannerJoinTree.h | 4 +- src/Planner/Utils.cpp | 30 +++++++++++ src/Planner/Utils.h | 3 ++ 5 files changed, 75 insertions(+), 78 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index dddb7531519..16ee6de73c4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -10,7 +10,6 @@ #include #include -#include #include #include @@ -331,14 +330,14 @@ public: void addExpressionStep(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { auto actions = ActionsDAG::clone(&expression_actions->dag); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); - result_actions_to_execute.push_back(expression_step->getExpression().get()); + appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -346,7 +345,7 @@ void addExpressionStep(QueryPlan & query_plan, void addFilterStep(QueryPlan & query_plan, const FilterAnalysisResult & filter_analysis_result, const std::string & step_description, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) @@ -356,7 +355,7 @@ void addFilterStep(QueryPlan & query_plan, actions, filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); - result_actions_to_execute.push_back(where_step->getExpression().get()); + appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -544,7 +543,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -573,7 +572,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, need_finalize); if (having_analysis_result.filter_actions) - result_actions_to_execute.push_back(totals_having_step->getActions().get()); + appendSetsFromActionsDAG(*totals_having_step->getActions(), useful_sets); query_plan.addStep(std::move(totals_having_step)); } @@ -887,7 +886,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, const QueryTreeNodePtr & query_tree, - std::vector & result_actions_to_execute) + UsefulSets & useful_sets) { const auto & query_node = query_tree->as(); @@ -919,7 +918,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, if (expressions_analysis_result.hasLimitBy()) { const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); - addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute); + addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1057,47 +1056,15 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana } } -void collectSetsFromActionsDAG(const ActionsDAG & dag, std::unordered_set & useful_sets) -{ - for (const auto & node : dag.getNodes()) - { - if (node.column) - { - const IColumn * column = node.column.get(); - if (const auto * column_const = typeid_cast(column)) - column = &column_const->getDataColumn(); - - if (const auto * column_set = typeid_cast(column)) - useful_sets.insert(column_set->getData().get()); - } - - if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") - { - ActionsDAG::NodeRawConstPtrs children; - if (const auto * adaptor = typeid_cast(node.function_base.get())) - { - if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) - { - collectSetsFromActionsDAG(*index_hint->getActions(), useful_sets); - } - } - } - } -} - void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) + const UsefulSets & useful_sets) { auto subqueries = planner_context->getPreparedSets().getSubqueries(); - std::unordered_set useful_sets; - for (const auto * actions_to_execute : result_actions_to_execute) - collectSetsFromActionsDAG(*actions_to_execute, useful_sets); - - auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); subqueries.erase(it, subqueries.end()); @@ -1542,15 +1509,15 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info); - std::vector result_actions_to_execute = std::move(join_tree_query_plan.actions_dags); + auto useful_sets = std::move(join_tree_query_plan.useful_sets); for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) { if (table_expression_data.getPrewhereFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions().get()); + appendSetsFromActionsDAG(*table_expression_data.getPrewhereFilterActions(), useful_sets); if (table_expression_data.getRowLevelFilterActions()) - result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions().get()); + appendSetsFromActionsDAG(*table_expression_data.getRowLevelFilterActions(), useful_sets); } if (query_processing_info.isIntermediateStage()) @@ -1561,7 +1528,7 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info, query_tree, - result_actions_to_execute); + useful_sets); if (expression_analysis_result.hasAggregation()) { @@ -1573,13 +1540,13 @@ void Planner::buildPlanForQueryNode() if (query_processing_info.isFirstStage()) { if (expression_analysis_result.hasWhere()) - addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", useful_sets); if (expression_analysis_result.hasAggregation()) { const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); if (aggregation_analysis_result.before_aggregation_actions) - addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", result_actions_to_execute); + addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets); addAggregationStep(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info); } @@ -1598,7 +1565,7 @@ void Planner::buildPlanForQueryNode() */ const auto & window_analysis_result = expression_analysis_result.getWindow(); if (window_analysis_result.before_window_actions) - addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", result_actions_to_execute); + addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets); } else { @@ -1607,7 +1574,7 @@ void Planner::buildPlanForQueryNode() * now, on shards (first_stage). */ const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) { @@ -1623,7 +1590,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { const auto & sort_analysis_result = expression_analysis_result.getSort(); - addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute); + addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } } @@ -1634,7 +1601,7 @@ void Planner::buildPlanForQueryNode() planner_context, query_processing_info, query_tree, - result_actions_to_execute); + useful_sets); } if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState()) @@ -1656,14 +1623,14 @@ void Planner::buildPlanForQueryNode() if (query_node.isGroupByWithTotals()) { - addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, result_actions_to_execute); + addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, useful_sets); having_executed = true; } addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node); if (!having_executed && expression_analysis_result.hasHaving()) - addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", useful_sets); } if (query_processing_info.isFromAggregationState()) @@ -1678,16 +1645,16 @@ void Planner::buildPlanForQueryNode() { const auto & window_analysis_result = expression_analysis_result.getWindow(); if (expression_analysis_result.hasAggregation()) - addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", result_actions_to_execute); + addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets); addWindowSteps(query_plan, planner_context, window_analysis_result); } if (expression_analysis_result.hasQualify()) - addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute); + addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets); const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) { @@ -1703,7 +1670,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { const auto & sort_analysis_result = expression_analysis_result.getSort(); - addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute); + addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } else @@ -1756,7 +1723,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy()) { const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); - addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute); + addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1788,7 +1755,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isToAggregationState()) { const auto & projection_analysis_result = expression_analysis_result.getProjection(); - addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute); + addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets); } // For additional_result_filter setting @@ -1796,7 +1763,7 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, useful_sets); query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode(); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 604d3366484..94054588d40 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1181,13 +1181,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); - left_join_tree_query_plan.actions_dags.emplace_back(left_join_expressions_actions_step->getExpression().get()); + appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); left_plan.addStep(std::move(left_join_expressions_actions_step)); join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); - right_join_tree_query_plan.actions_dags.emplace_back(right_join_expressions_actions_step->getExpression().get()); + appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1387,7 +1387,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ mixed_join_expression = std::make_shared( std::move(join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); - left_join_tree_query_plan.actions_dags.push_back(&mixed_join_expression->getActionsDAG()); + + appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); } } else if (join_node.isUsingJoinExpression()) @@ -1585,16 +1586,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); - /// Collect all required actions dags in `left_join_tree_query_plan.actions_dags` + /// Collect all required actions sets in `left_join_tree_query_plan.useful_sets` if (!is_filled_join) - for (const auto * action_dag : right_join_tree_query_plan.actions_dags) - left_join_tree_query_plan.actions_dags.emplace_back(action_dag); - // if (join_clauses_and_actions.left_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get()); - // if (join_clauses_and_actions.right_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get()); - // if (join_clauses_and_actions.mixed_join_expressions_actions) - // left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get()); + for (const auto & useful_set : right_join_tree_query_plan.useful_sets) + left_join_tree_query_plan.useful_sets.insert(useful_set); auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping); auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping; @@ -1604,7 +1599,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ .query_plan = std::move(result_plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(left_join_tree_query_plan.used_row_policies), - .actions_dags = std::move(left_join_tree_query_plan.actions_dags), + .useful_sets = std::move(left_join_tree_query_plan.useful_sets), .query_node_to_plan_step_mapping = std::move(mapping), }; } @@ -1649,7 +1644,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); - join_tree_query_plan.actions_dags.push_back(array_join_actions->getExpression().get()); + appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets); plan.addStep(std::move(array_join_actions)); auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); @@ -1690,7 +1685,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ .query_plan = std::move(plan), .from_stage = QueryProcessingStage::FetchColumns, .used_row_policies = std::move(join_tree_query_plan.used_row_policies), - .actions_dags = std::move(join_tree_query_plan.actions_dags), + .useful_sets = std::move(join_tree_query_plan.useful_sets), .query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping), }; } diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index 675079427eb..bc58e802a09 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -11,12 +11,14 @@ namespace DB { +using UsefulSets = std::unordered_set; + struct JoinTreeQueryPlan { QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - std::vector actions_dags{}; + UsefulSets useful_sets; std::unordered_map query_node_to_plan_step_mapping{}; }; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 493ecf5ef53..7ac53e0f8c1 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -11,10 +11,12 @@ #include #include +#include #include #include +#include #include @@ -475,4 +477,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings) return additional_result_filter_ast; } +void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) +{ + for (const auto & node : dag.getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); + + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData()); + } + + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets); + } + } + } + } +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 3172847f053..ae60976a8d6 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, ASTPtr parseAdditionalResultFilter(const Settings & settings); +using UsefulSets = std::unordered_set; +void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets); + } From 5d16ba57aa84ef82ccf7e34a4635ad1d14e7859d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Jul 2024 14:46:49 +0000 Subject: [PATCH 0100/1170] Update version_date.tsv and changelogs after v24.3.5.46-lts --- docs/changelogs/v24.3.5.46-lts.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 41 insertions(+) create mode 100644 docs/changelogs/v24.3.5.46-lts.md diff --git a/docs/changelogs/v24.3.5.46-lts.md b/docs/changelogs/v24.3.5.46-lts.md new file mode 100644 index 00000000000..1f2b7c8b0b7 --- /dev/null +++ b/docs/changelogs/v24.3.5.46-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.5.46-lts (fe54cead6b6) FIXME as compared to v24.3.4.147-lts (31a7bdc346d) + +#### Improvement +* Backported in [#65463](https://github.com/ClickHouse/ClickHouse/issues/65463): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Backported in [#65882](https://github.com/ClickHouse/ClickHouse/issues/65882): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65302](https://github.com/ClickHouse/ClickHouse/issues/65302): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65892](https://github.com/ClickHouse/ClickHouse/issues/65892): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65283](https://github.com/ClickHouse/ClickHouse/issues/65283): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65370](https://github.com/ClickHouse/ClickHouse/issues/65370): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#65446](https://github.com/ClickHouse/ClickHouse/issues/65446): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65708](https://github.com/ClickHouse/ClickHouse/issues/65708): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65352](https://github.com/ClickHouse/ClickHouse/issues/65352): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65327](https://github.com/ClickHouse/ClickHouse/issues/65327): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#65538](https://github.com/ClickHouse/ClickHouse/issues/65538): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65576](https://github.com/ClickHouse/ClickHouse/issues/65576): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65159](https://github.com/ClickHouse/ClickHouse/issues/65159): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65615](https://github.com/ClickHouse/ClickHouse/issues/65615): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65728](https://github.com/ClickHouse/ClickHouse/issues/65728): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65261](https://github.com/ClickHouse/ClickHouse/issues/65261): Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#65667](https://github.com/ClickHouse/ClickHouse/issues/65667): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65784](https://github.com/ClickHouse/ClickHouse/issues/65784): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65929](https://github.com/ClickHouse/ClickHouse/issues/65929): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65824](https://github.com/ClickHouse/ClickHouse/issues/65824): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65223](https://github.com/ClickHouse/ClickHouse/issues/65223): Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#65901](https://github.com/ClickHouse/ClickHouse/issues/65901): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8112ed9083b..8e748a2c2ca 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.5.1.1763-stable 2024-06-01 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 From d0f36e09a964c1e8a3040d6cfd8b3edfec47474d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 04:58:27 +0200 Subject: [PATCH 0101/1170] Fix error --- src/Databases/DatabaseAtomic.cpp | 3 +++ src/Databases/DatabaseAtomic.h | 2 ++ src/Databases/DatabaseOnDisk.cpp | 2 ++ src/Databases/DatabaseOnDisk.h | 1 + 4 files changed, 8 insertions(+) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index c06fc98d0b9..7f198042e44 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -58,6 +58,9 @@ 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(); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 26ab7657354..9df300daa20 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -75,7 +75,9 @@ 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/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index faac4b23701..0a0ad589a9f 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -175,6 +175,8 @@ DatabaseOnDisk::DatabaseOnDisk( void DatabaseOnDisk::createDirectories() { + if (directories_created.test_and_set()) + return; fs::create_directories(std::filesystem::path(getContext()->getPath()) / data_path); fs::create_directories(metadata_path); } diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 00e7a2850b8..a8be674a4e2 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -99,6 +99,7 @@ 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; From b52937e8580055a126f3d263cad893f212ce07b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 05:00:18 +0200 Subject: [PATCH 0102/1170] Better test --- tests/queries/0_stateless/01191_rename_dictionary.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index c5012dabc81..be95e5a7d4b 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -27,6 +27,7 @@ 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; From ee0985c5b4dbb15d28c9a034b60e438099b5c5bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 05:01:35 +0200 Subject: [PATCH 0103/1170] Fix test --- .../02141_clickhouse_local_interactive_table.reference | 4 ++-- .../0_stateless/02141_clickhouse_local_interactive_table.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) 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 0bb8966cbe4..0e74c0a083e 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 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\') +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\') 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 934d87616ac..3a95e59416a 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 table table' -$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' +$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' From bf312f200cb69267741778b2af04bf7a6854a2f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 05:24:42 +0200 Subject: [PATCH 0104/1170] DatabaseOverlay: add support for rename --- programs/local/LocalServer.cpp | 8 ++++---- src/Databases/DatabasesOverlay.cpp | 33 ++++++++++++++++++++++++++++++ src/Databases/DatabasesOverlay.h | 8 ++++++++ 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index cda276c8407..41bb5604a52 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -201,10 +201,10 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context) { - auto databaseCombiner = std::make_shared(name_, context); - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context)); - databaseCombiner->registerNextDatabase(std::make_shared(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context)); - return databaseCombiner; + 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; } /// If path is specified and not empty, will try to setup server environment and load existing metadata diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 02a0aab8230..e1a457920cf 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -124,6 +124,39 @@ 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; diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 5f6d4e601d3..40c653e5cb5 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -35,6 +35,14 @@ 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; From a70710e3f19aea4434aebbd07233b8e681e4e9e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 05:27:56 +0200 Subject: [PATCH 0105/1170] Add a test --- .../03199_atomic_clickhouse_local.reference | 6 +++++ .../03199_atomic_clickhouse_local.sh | 24 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/03199_atomic_clickhouse_local.reference create mode 100755 tests/queries/0_stateless/03199_atomic_clickhouse_local.sh diff --git a/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference b/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference new file mode 100644 index 00000000000..1975397394b --- /dev/null +++ b/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference @@ -0,0 +1,6 @@ +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 new file mode 100755 index 00000000000..edaa83b8f95 --- /dev/null +++ b/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh @@ -0,0 +1,24 @@ +#!/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 cb5d5863467a14cebdbc2dc1c6e4d72afe256515 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Jul 2024 05:49:55 +0200 Subject: [PATCH 0106/1170] Fix style --- src/Databases/DatabasesOverlay.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index e1a457920cf..495733e15fd 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -14,6 +14,8 @@ 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_) From a99c803ddf137e91f43c8f26f549f74f71eab102 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 17:26:06 +0800 Subject: [PATCH 0107/1170] 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 0108/1170] 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 0109/1170] 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 0110/1170] 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 07c0a72e5f06283d0d70f076eb8f33ee1339e9c8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jul 2024 18:30:55 +0000 Subject: [PATCH 0111/1170] Style --- src/Storages/System/IStorageSystemOneBlock.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/IStorageSystemOneBlock.cpp b/src/Storages/System/IStorageSystemOneBlock.cpp index 7cde31905aa..308b34510ea 100644 --- a/src/Storages/System/IStorageSystemOneBlock.cpp +++ b/src/Storages/System/IStorageSystemOneBlock.cpp @@ -80,9 +80,9 @@ void IStorageSystemOneBlock::read( void ReadFromSystemOneBlock::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - const auto & sample_block = getOutputStream().header; + const Block & sample_block = getOutputStream().header; MutableColumns res_columns = sample_block.cloneEmptyColumns(); - auto predicate = filter ? filter->getOutputs().at(0) : nullptr; + const ActionsDAG::Node * predicate = filter ? filter->getOutputs().at(0) : nullptr; storage->fillData(res_columns, context, predicate, std::move(columns_mask)); UInt64 num_rows = res_columns.at(0)->size(); From 90df83438f9866363690239ee9ec386a303dc3ba Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 5 Jul 2024 04:42:58 +0000 Subject: [PATCH 0112/1170] more shellcheck --- docker/test/stateless/attach_gdb.lib | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index a3616ac1a04..a0d1b8af6d8 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -11,7 +11,8 @@ function attach_gdb_to_clickhouse() # explicitly ignore non-fatal signals that are used by server. # Number of SIGRTMIN can be determined only in runtime. RTMIN=$(kill -l SIGRTMIN) - echo " + # shellcheck disable=SC2016 + echo " set follow-fork-mode parent handle SIGHUP nostop noprint pass handle SIGINT nostop noprint pass From 7843313f8e09eb018a481b4ba70fcf5fc147105e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 18:20:50 +0200 Subject: [PATCH 0113/1170] Update PlannerJoinTree.h --- src/Planner/PlannerJoinTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index bc58e802a09..259622b1d50 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -18,7 +18,7 @@ struct JoinTreeQueryPlan QueryPlan query_plan; QueryProcessingStage::Enum from_stage; std::set used_row_policies{}; - UsefulSets useful_sets; + UsefulSets useful_sets{}; std::unordered_map query_node_to_plan_step_mapping{}; }; From 2257f9a2aee5e8a5c5e178e5f7ccaf269018756a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 17:49:50 +0000 Subject: [PATCH 0114/1170] Change ActionsDAGPtr to ActionsDAG where possible. --- src/Core/InterpolateDescription.cpp | 6 +- src/Core/InterpolateDescription.h | 7 +- src/Functions/indexHint.h | 6 +- src/Interpreters/ActionsDAG.cpp | 148 +++++++++--------- src/Interpreters/ActionsDAG.h | 46 +++--- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 19 ++- src/Interpreters/ExpressionActions.h | 8 +- src/Interpreters/ExpressionAnalyzer.cpp | 51 +++--- src/Interpreters/ExpressionAnalyzer.h | 6 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- .../IInterpreterUnionOrSelectQuery.cpp | 12 +- src/Interpreters/InterpreterSelectQuery.cpp | 49 +++--- src/Interpreters/InterpreterSelectQuery.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 14 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/Interpreters/TableJoin.cpp | 49 +++--- src/Interpreters/TableJoin.h | 10 +- src/Interpreters/addMissingDefaults.cpp | 24 +-- src/Interpreters/addMissingDefaults.h | 2 +- .../evaluateConstantExpression.cpp | 2 +- src/Interpreters/inplaceBlockConversions.cpp | 16 +- src/Interpreters/inplaceBlockConversions.h | 6 +- src/Planner/Planner.cpp | 46 +++--- src/Planner/PlannerActionsVisitor.cpp | 14 +- src/Planner/PlannerContext.h | 2 +- src/Planner/PlannerJoinTree.cpp | 81 +++++----- src/Planner/PlannerJoins.cpp | 56 +++---- src/Planner/PlannerJoins.h | 10 +- src/Planner/Utils.cpp | 10 +- src/Processors/QueryPlan/AggregatingStep.cpp | 16 +- src/Processors/QueryPlan/CubeStep.cpp | 12 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 22 +-- src/Processors/QueryPlan/ExpressionStep.h | 11 +- src/Processors/QueryPlan/FilterStep.cpp | 20 +-- src/Processors/QueryPlan/FilterStep.h | 12 +- .../convertOuterJoinToInnerJoin.cpp | 4 +- .../Optimizations/distinctReadInOrder.cpp | 4 +- .../Optimizations/filterPushDown.cpp | 50 +++--- .../Optimizations/liftUpArrayJoin.cpp | 8 +- .../Optimizations/liftUpFunctions.cpp | 6 +- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../Optimizations/mergeExpressions.cpp | 45 +++--- .../Optimizations/optimizePrewhere.cpp | 19 +-- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +- .../Optimizations/optimizeReadInOrder.cpp | 14 +- .../optimizeUseAggregateProjection.cpp | 30 ++-- .../optimizeUseNormalProjection.cpp | 22 +-- .../Optimizations/projectionsCommon.cpp | 14 +- .../Optimizations/projectionsCommon.h | 4 +- .../Optimizations/removeRedundantDistinct.cpp | 12 +- .../Optimizations/removeRedundantSorting.cpp | 4 +- .../QueryPlan/Optimizations/splitFilter.cpp | 14 +- .../useDataParallelAggregation.cpp | 6 +- src/Processors/QueryPlan/PartsSplitter.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 39 ++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 8 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 2 +- .../QueryPlan/SourceStepWithFilter.cpp | 8 +- .../QueryPlan/SourceStepWithFilter.h | 6 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 14 +- src/Processors/QueryPlan/TotalsHavingStep.h | 7 +- src/Processors/SourceWithKeyCondition.h | 8 +- .../Transforms/AddingDefaultsTransform.cpp | 2 +- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 24 +-- src/Storages/Hive/StorageHive.h | 10 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 4 +- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/KVStorageUtils.h | 2 +- src/Storages/KeyDescription.cpp | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 38 ++--- src/Storages/MergeTree/MergeTreeIndexSet.h | 6 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeSequentialSource.cpp | 8 +- .../MergeTree/MergeTreeSequentialSource.h | 2 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 4 +- .../MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 4 +- .../StorageObjectStorageSource.cpp | 4 +- .../StorageObjectStorageSource.h | 2 +- .../ReadFinalForExternalReplicaStorage.cpp | 2 +- src/Storages/SelectQueryInfo.h | 12 +- src/Storages/StorageBuffer.cpp | 12 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMerge.cpp | 30 ++-- src/Storages/StorageMerge.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageTableFunction.h | 2 +- src/Storages/StorageURL.h | 4 +- src/Storages/StorageValues.cpp | 6 +- src/Storages/StorageView.cpp | 6 +- .../System/StorageSystemStackTrace.cpp | 4 +- src/Storages/TTLDescription.cpp | 8 +- src/Storages/VirtualColumnUtils.cpp | 12 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 110 files changed, 718 insertions(+), 721 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 76bbefdcfd7..86681fdb591 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -13,10 +13,10 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) + InterpolateDescription::InterpolateDescription(ActionsDAG actions_, const Aliases & aliases) : actions(std::move(actions_)) { - for (const auto & name_type : actions->getRequiredColumns()) + for (const auto & name_type : actions.getRequiredColumns()) { if (const auto & p = aliases.find(name_type.name); p != aliases.end()) required_columns_map[p->second->getColumnName()] = name_type; @@ -24,7 +24,7 @@ namespace DB required_columns_map[name_type.name] = name_type; } - for (const ColumnWithTypeAndName & column : actions->getResultColumns()) + for (const ColumnWithTypeAndName & column : actions.getResultColumns()) { std::string name = column.name; if (const auto & p = aliases.find(name); p != aliases.end()) diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 73579aebee4..eeead71d780 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -5,21 +5,20 @@ #include #include #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; using Aliases = std::unordered_map; /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases); + explicit InterpolateDescription(ActionsDAG actions, const Aliases & aliases); - ActionsDAGPtr actions; + ActionsDAG actions; std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_set; /// result block columns diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 8fd7b751760..3ab8a021ae1 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -58,11 +58,11 @@ public: return DataTypeUInt8().createColumnConst(input_rows_count, 1u); } - void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); } - const ActionsDAGPtr & getActions() const { return actions; } + void setActions(ActionsDAG actions_) { actions = std::move(actions_); } + const ActionsDAG & getActions() const { return actions; } private: - ActionsDAGPtr actions; + ActionsDAG actions; }; } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index c2626285235..04be9d23c32 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -961,9 +961,9 @@ NameSet ActionsDAG::foldActionsByProjection( } -ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) +ActionsDAG ActionsDAG::foldActionsByProjection(const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs) { - auto dag = std::make_unique(); + ActionsDAG dag; std::unordered_map inputs_mapping; std::unordered_map mapping; struct Frame @@ -1003,9 +1003,9 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name != rename->result_name; const auto & input_name = should_rename ? rename->result_name : new_input->result_name; - mapped_input = &dag->addInput(input_name, new_input->result_type); + mapped_input = &dag.addInput(input_name, new_input->result_type); if (should_rename) - mapped_input = &dag->addAlias(*mapped_input, new_input->result_name); + mapped_input = &dag.addAlias(*mapped_input, new_input->result_name); } node = mapped_input; @@ -1034,7 +1034,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name, frame.node->result_name); - auto & node = dag->nodes.emplace_back(*frame.node); + auto & node = dag.nodes.emplace_back(*frame.node); for (auto & child : node.children) child = mapping[child]; @@ -1049,8 +1049,8 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_mapresult_name != mapped_output->result_name) - mapped_output = &dag->addAlias(*mapped_output, output->result_name); - dag->outputs.push_back(mapped_output); + mapped_output = &dag.addAlias(*mapped_output, output->result_name); + dag.outputs.push_back(mapped_output); } return dag; @@ -1411,7 +1411,7 @@ const ActionsDAG::Node & ActionsDAG::materializeNode(const Node & node) return addAlias(*func, name); } -ActionsDAGPtr ActionsDAG::makeConvertingActions( +ActionsDAG ActionsDAG::makeConvertingActions( const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & result, MatchColumnsMode mode, @@ -1428,7 +1428,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (add_casted_columns && mode != MatchColumnsMode::Name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); - auto actions_dag = std::make_unique(source); + ActionsDAG actions_dag(source); NodeRawConstPtrs projection(num_result_columns); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); @@ -1436,9 +1436,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( std::unordered_map> inputs; if (mode == MatchColumnsMode::Name) { - size_t input_nodes_size = actions_dag->inputs.size(); + size_t input_nodes_size = actions_dag.inputs.size(); for (size_t pos = 0; pos < input_nodes_size; ++pos) - inputs[actions_dag->inputs[pos]->result_name].push_back(pos); + inputs[actions_dag.inputs[pos]->result_name].push_back(pos); } for (size_t result_col_num = 0; result_col_num < num_result_columns; ++result_col_num) @@ -1451,7 +1451,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( { case MatchColumnsMode::Position: { - src_node = dst_node = actions_dag->inputs[result_col_num]; + src_node = dst_node = actions_dag.inputs[result_col_num]; break; } @@ -1462,7 +1462,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( { const auto * res_const = typeid_cast(res_elem.column.get()); if (ignore_constant_values && res_const) - src_node = dst_node = &actions_dag->addColumn(res_elem); + src_node = dst_node = &actions_dag.addColumn(res_elem); else throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Cannot find column `{}` in source stream, there are only columns: [{}]", @@ -1470,7 +1470,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } else { - src_node = dst_node = actions_dag->inputs[input.front()]; + src_node = dst_node = actions_dag.inputs[input.front()]; input.pop_front(); } break; @@ -1483,7 +1483,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( if (const auto * src_const = typeid_cast(dst_node->column.get())) { if (ignore_constant_values) - dst_node = &actions_dag->addColumn(res_elem); + dst_node = &actions_dag.addColumn(res_elem); else if (res_const->getField() != src_const->getField()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -1505,7 +1505,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( column.column = DataTypeString().createColumnConst(0, column.name); column.type = std::make_shared(); - const auto * right_arg = &actions_dag->addColumn(std::move(column)); + const auto * right_arg = &actions_dag.addColumn(std::move(column)); const auto * left_arg = dst_node; CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; @@ -1513,13 +1513,13 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( = createInternalCastOverloadResolver(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_builder_cast, std::move(children), {}); } if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column))) { NodeRawConstPtrs children = {dst_node}; - dst_node = &actions_dag->addFunction(func_builder_materialize, std::move(children), {}); + dst_node = &actions_dag.addFunction(func_builder_materialize, std::move(children), {}); } if (dst_node->result_name != res_elem.name) @@ -1538,7 +1538,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } else { - dst_node = &actions_dag->addAlias(*dst_node, res_elem.name); + dst_node = &actions_dag.addAlias(*dst_node, res_elem.name); projection[result_col_num] = dst_node; } } @@ -1548,36 +1548,36 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } } - actions_dag->outputs.swap(projection); - actions_dag->removeUnusedActions(false); + actions_dag.outputs.swap(projection); + actions_dag.removeUnusedActions(false); return actions_dag; } -ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) +ActionsDAG ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) { - auto adding_column_action = std::make_unique(); + ActionsDAG adding_column_action; FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); auto column_name = column.name; - const auto * column_node = &adding_column_action->addColumn(std::move(column)); + const auto * column_node = &adding_column_action.addColumn(std::move(column)); NodeRawConstPtrs inputs = {column_node}; - const auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {}); - const auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name)); + const auto & function_node = adding_column_action.addFunction(func_builder_materialize, std::move(inputs), {}); + const auto & alias_node = adding_column_action.addAlias(function_node, std::move(column_name)); - adding_column_action->outputs.push_back(&alias_node); + adding_column_action.outputs.push_back(&alias_node); return adding_column_action; } -ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) +ActionsDAG ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) { first.mergeInplace(std::move(second)); /// Some actions could become unused. Do not drop inputs to preserve the header. first.removeUnusedActions(false); - return std::make_unique(std::move(first)); + return std::move(first); } void ActionsDAG::mergeInplace(ActionsDAG && second) @@ -1970,15 +1970,15 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split second_inputs.push_back(cur.to_second); } - auto first_actions = std::make_unique(); - first_actions->nodes.swap(first_nodes); - first_actions->outputs.swap(first_outputs); - first_actions->inputs.swap(first_inputs); + ActionsDAG first_actions; + first_actions.nodes.swap(first_nodes); + first_actions.outputs.swap(first_outputs); + first_actions.inputs.swap(first_inputs); - auto second_actions = std::make_unique(); - second_actions->nodes.swap(second_nodes); - second_actions->outputs.swap(second_outputs); - second_actions->inputs.swap(second_inputs); + ActionsDAG second_actions; + second_actions.nodes.swap(second_nodes); + second_actions.outputs.swap(second_outputs); + second_actions.inputs.swap(second_inputs); std::unordered_map split_nodes_mapping; if (create_split_nodes_mapping) @@ -2098,7 +2098,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS return res; } -bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) +bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const { const auto * filter_node = tryFindInOutputs(filter_name); if (!filter_node) @@ -2122,7 +2122,7 @@ bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & fi input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name)); } - ActionsDAGPtr filter_with_default_value_inputs; + std::optional filter_with_default_value_inputs; try { @@ -2304,12 +2304,12 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt /// /// Result actions add single column with conjunction result (it is always first in outputs). /// No other columns are added or removed. -ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) +std::optional ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) { if (conjunction.empty()) - return nullptr; + return {}; - auto actions = std::make_unique(); + ActionsDAG actions; FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); @@ -2350,7 +2350,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti if (cur.next_child_to_visit == cur.node->children.size()) { - auto & node = actions->nodes.emplace_back(*cur.node); + auto & node = actions.nodes.emplace_back(*cur.node); nodes_mapping[cur.node] = &node; for (auto & child : node.children) @@ -2373,33 +2373,33 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti for (const auto * predicate : conjunction) args.emplace_back(nodes_mapping[predicate]); - result_predicate = &actions->addFunction(func_builder_and, std::move(args), {}); + result_predicate = &actions.addFunction(func_builder_and, std::move(args), {}); } - actions->outputs.push_back(result_predicate); + actions.outputs.push_back(result_predicate); for (const auto & col : all_inputs) { const Node * input; auto & list = required_inputs[col.name]; if (list.empty()) - input = &actions->addInput(col); + input = &actions.addInput(col); else { input = list.front(); list.pop_front(); - actions->inputs.push_back(input); + actions.inputs.push_back(input); } /// We should not add result_predicate into the outputs for the second time. if (input->result_name != result_predicate->result_name) - actions->outputs.push_back(input); + actions.outputs.push_back(input); } return actions; } -ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( +std::optional ActionsDAG::splitActionsForFilterPushDown( const std::string & filter_name, bool removes_filter, const Names & available_inputs, @@ -2415,7 +2415,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( /// If condition is constant let's do nothing. /// It means there is nothing to push down or optimization was already applied. if (predicate->type == ActionType::COLUMN) - return nullptr; + return {}; std::unordered_set allowed_nodes; @@ -2439,7 +2439,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( auto conjunction = getConjunctionNodes(predicate, allowed_nodes); if (conjunction.allowed.empty()) - return nullptr; + return {}; chassert(predicate->result_type); @@ -2451,13 +2451,13 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( && !conjunction.rejected.front()->result_type->equals(*predicate->result_type)) { /// No further optimization can be done - return nullptr; + return {}; } } auto actions = createActionsForConjunction(conjunction.allowed, all_inputs); if (!actions) - return nullptr; + return {}; /// Now, when actions are created, update the current DAG. removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter); @@ -2562,11 +2562,11 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName()); auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName()); - auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter, + auto replace_equivalent_columns_in_filter = [](const ActionsDAG & filter, const Block & stream_header, const std::unordered_map & columns_to_replace) { - auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace); + auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter.getOutputs()[0]}, columns_to_replace); chassert(updated_filter->getOutputs().size() == 1); /** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from @@ -2587,7 +2587,7 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu for (const auto & input : updated_filter->getInputs()) updated_filter_inputs[input->result_name].push_back(input); - for (const auto & input : filter->getInputs()) + for (const auto & input : filter.getInputs()) { if (updated_filter_inputs.contains(input->result_name)) continue; @@ -2625,12 +2625,12 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu }; if (left_stream_filter_to_push_down) - left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down, + left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*left_stream_filter_to_push_down, left_stream_header, equivalent_right_stream_column_to_left_stream_column); if (right_stream_filter_to_push_down) - right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down, + right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*right_stream_filter_to_push_down, right_stream_header, equivalent_left_stream_column_to_right_stream_column); @@ -2859,13 +2859,13 @@ bool ActionsDAG::isSortingPreserved( return true; } -ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( +std::optional ActionsDAG::buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column, bool single_output_condition_node) { if (filter_nodes.empty()) - return nullptr; + return {}; struct Frame { @@ -2873,7 +2873,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( bool visited_children = false; }; - auto result_dag = std::make_unique(); + ActionsDAG result_dag; std::unordered_map result_inputs; std::unordered_map node_to_result_node; @@ -2904,7 +2904,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { auto & result_input = result_inputs[input_node_it->second.name]; if (!result_input) - result_input = &result_dag->addInput(input_node_it->second); + result_input = &result_dag.addInput(input_node_it->second); node_to_result_node.emplace(node, result_input); nodes_to_process.pop_back(); @@ -2931,25 +2931,25 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { auto & result_input = result_inputs[node->result_name]; if (!result_input) - result_input = &result_dag->addInput({node->column, node->result_type, node->result_name}); + result_input = &result_dag.addInput({node->column, node->result_type, node->result_name}); result_node = result_input; break; } case ActionsDAG::ActionType::COLUMN: { - result_node = &result_dag->addColumn({node->column, node->result_type, node->result_name}); + result_node = &result_dag.addColumn({node->column, node->result_type, node->result_name}); break; } case ActionsDAG::ActionType::ALIAS: { const auto * child = node->children.front(); - result_node = &result_dag->addAlias(*(node_to_result_node.find(child)->second), node->result_name); + result_node = &result_dag.addAlias(*(node_to_result_node.find(child)->second), node->result_name); break; } case ActionsDAG::ActionType::ARRAY_JOIN: { const auto * child = node->children.front(); - result_node = &result_dag->addArrayJoin(*(node_to_result_node.find(child)->second), {}); + result_node = &result_dag.addArrayJoin(*(node_to_result_node.find(child)->second), {}); break; } case ActionsDAG::ActionType::FUNCTION: @@ -2967,13 +2967,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - ActionsDAGPtr index_hint_filter_dag; - const auto & index_hint_args = index_hint->getActions()->getOutputs(); + ActionsDAG index_hint_filter_dag; + const auto & index_hint_args = index_hint->getActions().getOutputs(); - if (index_hint_args.empty()) - index_hint_filter_dag = std::make_unique(); - else - index_hint_filter_dag = buildFilterActionsDAG(index_hint_args, + if (!index_hint_args.empty()) + index_hint_filter_dag = *buildFilterActionsDAG(index_hint_args, node_name_to_input_node_column, false /*single_output_condition_node*/); @@ -2995,7 +2993,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( auto [arguments, all_const] = getFunctionArguments(function_children); auto function_base = function_overload_resolver ? function_overload_resolver->build(arguments) : node->function_base; - result_node = &result_dag->addFunctionImpl( + result_node = &result_dag.addFunctionImpl( function_base, std::move(function_children), std::move(arguments), @@ -3010,7 +3008,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( nodes_to_process.pop_back(); } - auto & result_dag_outputs = result_dag->getOutputs(); + auto & result_dag_outputs = result_dag.getOutputs(); result_dag_outputs.reserve(filter_nodes_size); for (const auto & node : filter_nodes) @@ -3019,7 +3017,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( if (result_dag_outputs.size() > 1 && single_output_condition_node) { FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - result_dag_outputs = { &result_dag->addFunction(func_builder_and, result_dag_outputs, {}) }; + result_dag_outputs = { &result_dag.addFunction(func_builder_and, result_dag_outputs, {}) }; } return result_dag; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 7ca3d1c1b0d..cf6a91b9fe7 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -247,7 +247,7 @@ public: /// c * d e /// \ / /// c * d - e - static ActionsDAGPtr foldActionsByProjection( + static ActionsDAG foldActionsByProjection( const std::unordered_map & new_inputs, const NodeRawConstPtrs & required_outputs); @@ -303,7 +303,7 @@ public: /// @param ignore_constant_values - Do not check that constants are same. Use value from result_header. /// @param add_casted_columns - Create new columns with converted values instead of replacing original. /// @param new_names - Output parameter for new column names when add_casted_columns is used. - static ActionsDAGPtr makeConvertingActions( + static ActionsDAG makeConvertingActions( const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & result, MatchColumnsMode mode, @@ -312,13 +312,13 @@ public: NameToNameMap * new_names = nullptr); /// Create expression which add const column and then materialize it. - static ActionsDAGPtr makeAddingColumnActions(ColumnWithTypeAndName column); + static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column); /// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently. /// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression. /// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`. /// Otherwise, any two actions may be combined. - static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second); + static ActionsDAG merge(ActionsDAG && first, ActionsDAG && second); /// The result is similar to merge(*this, second); /// Invariant : no nodes are removed from the first (this) DAG. @@ -329,12 +329,7 @@ public: /// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs(). void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr); - struct SplitResult - { - ActionsDAGPtr first; - ActionsDAGPtr second; - std::unordered_map split_nodes_mapping; - }; + struct SplitResult; /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. /// Execution of first then second parts on block is equivalent to execution of initial DAG. @@ -362,7 +357,7 @@ public: * @param filter_name - name of filter node in current DAG. * @param input_stream_header - input stream header. */ - bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header); + bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const; /// Create actions which may calculate part of filter using only available_inputs. /// If nothing may be calculated, returns nullptr. @@ -381,19 +376,13 @@ public: /// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`. /// To avoid it, add inputs from `all_inputs` list, /// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order. - ActionsDAGPtr splitActionsForFilterPushDown( + std::optional splitActionsForFilterPushDown( const std::string & filter_name, bool removes_filter, const Names & available_inputs, const ColumnsWithTypeAndName & all_inputs); - struct ActionsForJOINFilterPushDown - { - ActionsDAGPtr left_stream_filter_to_push_down; - bool left_stream_filter_removes_filter; - ActionsDAGPtr right_stream_filter_to_push_down; - bool right_stream_filter_removes_filter; - }; + struct ActionsForJOINFilterPushDown; /** Split actions for JOIN filter push down. * @@ -440,7 +429,7 @@ public: * * If single_output_condition_node = false, result dag has multiple output nodes. */ - static ActionsDAGPtr buildFilterActionsDAG( + static std::optional buildFilterActionsDAG( const NodeRawConstPtrs & filter_nodes, const std::unordered_map & node_name_to_input_node_column = {}, bool single_output_condition_node = true); @@ -472,11 +461,26 @@ private: void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); + static std::optional createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter); }; +struct ActionsDAG::SplitResult +{ + ActionsDAG first; + ActionsDAG second; + std::unordered_map split_nodes_mapping; +}; + +struct ActionsDAG::ActionsForJOINFilterPushDown +{ + std::optional left_stream_filter_to_push_down; + bool left_stream_filter_removes_filter; + std::optional right_stream_filter_to_push_down; + bool right_stream_filter_removes_filter; +}; + class FindOriginalNodeForOutputName { using NameToNodeIndex = std::unordered_map; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 036b5ba9be0..c2dcdcd34e7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1022,7 +1022,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & dag.project(args); auto index_hint = std::make_shared(); - index_hint->setActions(std::make_unique(std::move(dag))); + index_hint->setActions(std::move(dag)); // Arguments are removed. We add function instead of constant column to avoid constant folding. data.addFunction(std::make_unique(index_hint), {}, column_name); @@ -1285,7 +1285,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - std::make_unique(std::move(lambda_dag)), + std::move(lambda_dag), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 2eca31fc75e..399f4f2ff4f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -49,14 +49,13 @@ namespace ErrorCodes static std::unordered_set processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation); -ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) - : project_inputs(project_inputs_) +ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) + : actions_dag(std::move(actions_dag_)) + , project_inputs(project_inputs_) , settings(settings_) { - actions_dag = ActionsDAG::clone(actions_dag_); - /// It's important to determine lazy executed nodes before compiling expressions. - std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation); + std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation); #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) @@ -68,7 +67,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns) throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, "Too many temporary columns: {}. Maximum: {}", - actions_dag->dumpNames(), settings.max_temporary_columns); + actions_dag.dumpNames(), settings.max_temporary_columns); } ExpressionActionsPtr ExpressionActions::clone() const @@ -76,12 +75,12 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = ActionsDAG::clone(actions_dag.get(), copy_map); + copy->actions_dag = std::move(*ActionsDAG::clone(&actions_dag, copy_map)); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; - for (const auto * input : copy->actions_dag->getInputs()) + for (const auto * input : copy->actions_dag.getInputs()) copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name)); copy->num_columns = num_columns; @@ -357,8 +356,8 @@ void ExpressionActions::linearizeActions(const std::unordered_setgetOutputs(); - const auto & inputs = actions_dag->getInputs(); + const auto & outputs = actions_dag.getOutputs(); + const auto & inputs = actions_dag.getInputs(); auto reverse_info = getActionsDAGReverseInfo(nodes, outputs); std::vector data; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 63ea989bd5e..6ff39ee07f7 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -70,7 +70,7 @@ public: using NameToInputMap = std::unordered_map>; private: - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; Actions actions; size_t num_columns = 0; @@ -84,13 +84,13 @@ private: ExpressionActionsSettings settings; public: - explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); + explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); ExpressionActions(ExpressionActions &&) = default; ExpressionActions & operator=(ExpressionActions &&) = default; const Actions & getActions() const { return actions; } - const std::list & getNodes() const { return actions_dag->getNodes(); } - const ActionsDAG & getActionsDAG() const { return *actions_dag; } + const std::list & getNodes() const { return actions_dag.getNodes(); } + const ActionsDAG & getActionsDAG() const { return actions_dag; } const ColumnNumbers & getResultPositions() const { return result_positions; } const ExpressionActionsSettings & getSettings() const { return settings; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6b49365b492..068b6f290fa 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -928,7 +928,7 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); - ActionsDAGPtr converting_actions; + std::optional converting_actions; JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions); if (converting_actions) @@ -1039,7 +1039,7 @@ static std::unique_ptr buildJoinedPlan( /// Actions which need to be calculated on joined block. auto joined_block_actions = analyzed_join.createJoinedBlockActions(context); NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); Names original_right_column_names; for (auto & pr : required_columns_with_aliases) @@ -1060,17 +1060,17 @@ static std::unique_ptr buildJoinedPlan( interpreter->buildQueryPlan(*joined_plan); { Block original_right_columns = interpreter->getSampleBlock(); - auto rename_dag = std::make_unique(original_right_columns.getColumnsWithTypeAndName()); + ActionsDAG rename_dag(original_right_columns.getColumnsWithTypeAndName()); for (const auto & name_with_alias : required_columns_with_aliases) { if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first)) { auto pos = original_right_columns.getPositionByName(name_with_alias.first); - const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second); - rename_dag->getOutputs()[pos] = &alias; + const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], name_with_alias.second); + rename_dag.getOutputs()[pos] = &alias; } } - rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); + rename_dag.appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); auto rename_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(rename_dag)); rename_step->setStepDescription("Rename joined columns"); joined_plan->addStep(std::move(rename_step)); @@ -1130,14 +1130,14 @@ std::shared_ptr tryKeyValueJoin(std::shared_ptr a JoinPtr SelectQueryExpressionAnalyzer::makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, - ActionsDAGPtr & left_convert_actions) + std::optional & left_convert_actions) { /// Two JOINs are not supported with the same subquery, but different USINGs. if (joined_plan) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query"); - ActionsDAGPtr right_convert_actions = nullptr; + std::optional right_convert_actions; const auto & analyzed_join = syntax->analyzed_join; @@ -1145,7 +1145,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( { auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext()); NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); Names original_right_column_names; for (auto & pr : required_columns_with_aliases) @@ -1162,7 +1162,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); if (right_convert_actions) { - auto converting_step = std::make_unique(joined_plan->getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); joined_plan->addStep(std::move(converting_step)); } @@ -1354,8 +1354,8 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - auto actions_dag = std::make_unique(columns_after_join); - getRootActions(child, only_types, *actions_dag); + ActionsDAG actions_dag(columns_after_join); + getRootActions(child, only_types, actions_dag); group_by_elements_actions.emplace_back( std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1471,7 +1471,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList()); step.required_output = std::move(required_output); - step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); + step.actions()->dag = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position); } void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) @@ -1607,8 +1607,8 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr { for (const auto & child : select_query->orderBy()->children) { - auto actions_dag = std::make_unique(columns_after_join); - getRootActions(child, only_types, *actions_dag); + ActionsDAG actions_dag(columns_after_join); + getRootActions(child, only_types, actions_dag); order_by_elements_actions.emplace_back( std::make_shared(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1737,7 +1737,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const step.addRequiredOutput(expr->getColumnName()); } -ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) +ActionsDAG ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) { ActionsDAG actions_dag(aggregated_columns); NamesWithAliases result_columns; @@ -1789,7 +1789,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un actions_dag.removeUnusedActions(name_set); } - return std::make_unique(std::move(actions_dag)); + return actions_dag; } ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions) @@ -1798,10 +1798,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result); } -ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) +ActionsDAG ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) { - auto actions = std::make_unique(constant_inputs); - getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); + ActionsDAG actions(constant_inputs); + getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */); return actions; } @@ -1879,8 +1879,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (prewhere_dag_and_flags) { - auto dag = std::make_unique(std::move(prewhere_dag_and_flags->dag)); - prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); + prewhere_info = std::make_shared(std::move(prewhere_dag_and_flags->dag), query.prewhere()->getColumnName()); prewhere_dag_and_flags.reset(); } @@ -1944,7 +1943,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - auto dag = ActionsDAG::clone(&prewhere_dag_and_flags->dag); + ActionsDAG dag = std::move(*ActionsDAG::clone(&prewhere_dag_and_flags->dag)); ExpressionActions( std::move(dag), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); @@ -1980,7 +1979,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - ActionsDAG::clone(&before_where->dag), + std::move(*ActionsDAG::clone(&before_where->dag)), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem @@ -2054,7 +2053,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( auto & step = chain.lastStep(query_analyzer.aggregated_columns); auto & actions = step.actions()->dag; - actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting))); + actions = ActionsDAG::merge(std::move(actions), std::move(converting)); } } @@ -2235,7 +2234,7 @@ void ExpressionAnalysisResult::checkActions() const /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (hasPrewhere()) { - auto check_actions = [](const ActionsDAGPtr & actions) + auto check_actions = [](const std::optional & actions) { if (actions) for (const auto & node : actions->getNodes()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index e44a5891e77..737d36eb504 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -117,12 +117,12 @@ public: /// If add_aliases, only the calculated values in the desired order and add aliases. /// If also remove_unused_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. - ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true); + ActionsDAG getActionsDAG(bool add_aliases, bool remove_unused_result = true); ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no); /// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants. /// Does not execute subqueries. - ActionsDAGPtr getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {}); + ActionsDAG getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {}); ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {}); /** Sets that require a subquery to be create. @@ -367,7 +367,7 @@ private: JoinPtr makeJoin( const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_columns, - ActionsDAGPtr & left_convert_actions); + std::optional & left_convert_actions); const ASTSelectQuery * getAggregatingQuery() const; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 64b6eb5dce9..fcf0d591918 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -295,7 +295,7 @@ private: { auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext()); NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns( - Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames()); for (auto & pr : required_columns_with_aliases) required_columns.push_back(pr.first); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index fed29b410db..288d06d2220 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -99,16 +99,16 @@ static ASTPtr parseAdditionalPostFilter(const Context & context) "additional filter", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks); } -static ActionsDAGPtr makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) +static ActionsDAG makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) { auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList()); String result_column_name = ast->getColumnName(); auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false); - const ActionsDAG::Node * result_node = &dag->findInOutputs(result_column_name); - auto & outputs = dag->getOutputs(); + const ActionsDAG::Node * result_node = &dag.findInOutputs(result_column_name); + auto & outputs = dag.getOutputs(); outputs.clear(); - outputs.reserve(dag->getInputs().size() + 1); - for (const auto * node : dag->getInputs()) + outputs.reserve(dag.getInputs().size() + 1); + for (const auto * node : dag.getInputs()) outputs.push_back(node); outputs.push_back(result_node); @@ -126,7 +126,7 @@ void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) c return; auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header); - std::string filter_name = dag->getOutputs().back()->result_name; + std::string filter_name = dag.getOutputs().back()->result_name; auto filter_step = std::make_unique( plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true); filter_step->setStepDescription("Additional result filter"); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 09a7e440f31..cde6e305005 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -175,7 +175,7 @@ FilterDAGInfoPtr generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); - filter_info->actions = std::make_unique(std::move(analyzer.simpleSelectActions()->dag)); + filter_info->actions = std::move(analyzer.simpleSelectActions()->dag); filter_info->column_name = expr_list->children.at(0)->getColumnName(); filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); @@ -938,7 +938,8 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() } } - query_info_copy.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + if (auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes)) + query_info_copy.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy); /// Note that we treat an estimation of 0 rows as a real estimation size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; @@ -973,7 +974,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) ActionsDAG::MatchColumnsMode::Name, true); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); query_plan.addStep(std::move(converting)); } @@ -1297,10 +1298,10 @@ static InterpolateDescriptionPtr getInterpolateDescription( auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns); ExpressionAnalyzer analyzer(exprs, syntax_result, context); - ActionsDAGPtr actions = analyzer.getActionsDAG(true); - ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), + ActionsDAG actions = analyzer.getActionsDAG(true); + ActionsDAG conv_dag = ActionsDAG::makeConvertingActions(actions.getResultColumns(), result_columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(* ActionsDAG::clone(actions)), std::move(*conv_dag)); + ActionsDAG merge_dag = ActionsDAG::merge(std::move(actions), std::move(conv_dag)); interpolate_descr = std::make_shared(std::move(merge_dag), aliases); } @@ -1485,7 +1486,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.filter_info->actions, + std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1499,7 +1500,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->row_level_filter, + std::move(*ActionsDAG::clone(&*expressions.prewhere_info->row_level_filter)), expressions.prewhere_info->row_level_column_name, true); @@ -1509,7 +1510,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->prewhere_actions, + std::move(*ActionsDAG::clone(&*expressions.prewhere_info->prewhere_actions)), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -1611,7 +1612,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.filter_info->actions, + std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1623,7 +1624,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - new_filter_info->actions, + std::move(*ActionsDAG::clone(&*new_filter_info->actions)), new_filter_info->column_name, new_filter_info->do_remove_column); @@ -2045,7 +2046,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(ActionsDAG::clone(prewhere_info.row_level_filter)), + std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.row_level_filter))), prewhere_info.row_level_column_name, true); }); } @@ -2053,7 +2054,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(ActionsDAG::clone(prewhere_info.prewhere_actions)), + header, std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.prewhere_actions))), prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2106,7 +2107,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = std::move(*analysis.filter_info->actions); analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } @@ -2323,7 +2324,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle if (!filter_actions_dag) return {}; - return storage->totalRowsByPartitionPredicate(filter_actions_dag, context); + return storage->totalRowsByPartitionPredicate(*filter_actions_dag, context); } } @@ -2573,7 +2574,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), alias_actions); + auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&*alias_actions))); table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } @@ -2581,9 +2582,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = ActionsDAG::clone(&expression->dag); + auto dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto where_step = std::make_unique( query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter); @@ -2755,9 +2756,9 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = ActionsDAG::clone(&expression->dag); + auto dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto having_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter); @@ -2770,10 +2771,10 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final) { - ActionsDAGPtr dag; + std::optional dag; if (expression) { - dag = ActionsDAG::clone(&expression->dag); + dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2822,9 +2823,9 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - auto dag = ActionsDAG::clone(&expression->dag); + ActionsDAG dag = std::move(*ActionsDAG::clone(&expression->dag)); if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index d4ed19d45ea..ed6dd8af3b2 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -240,7 +240,7 @@ private: Block source_header; /// Actions to calculate ALIAS if required. - ActionsDAGPtr alias_actions; + std::optional alias_actions; /// The subquery interpreter, if the subquery std::unique_ptr interpreter_subquery; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 704c5ce7d8b..2372d26e83f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1197,7 +1197,7 @@ void MutationsInterpreter::Source::read( const auto & names = first_stage.filter_column_names; size_t num_filters = names.size(); - ActionsDAGPtr filter; + std::optional filter; if (!first_stage.filter_column_names.empty()) { ActionsDAG::NodeRawConstPtrs nodes(num_filters); @@ -1278,19 +1278,19 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = ActionsDAG::clone(&step->actions()->dag); + auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); if (step->actions()->project_input) - dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), std::move(dag), stage.filter_column_names[i], false)); } else { - auto dag = ActionsDAG::clone(&step->actions()->dag); + auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); if (step->actions()->project_input) - dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), std::move(dag))); } } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4821d607d0e..c1d7acf0775 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -182,7 +182,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, ASTPtr temp_ast = expr.clone(); auto syntax = TreeRewriter(context).analyze(temp_ast, columns); - auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false)->getRequiredColumns(); + auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false).getRequiredColumns(); return required_columns; } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index baf3a743f40..c8c926db13c 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -462,19 +462,19 @@ static void makeColumnNameUnique(const ColumnsWithTypeAndName & source_columns, } } -static ActionsDAGPtr createWrapWithTupleActions( +static std::optional createWrapWithTupleActions( const ColumnsWithTypeAndName & source_columns, std::unordered_set && column_names_to_wrap, NameToNameMap & new_names) { if (column_names_to_wrap.empty()) - return nullptr; + return {}; - auto actions_dag = std::make_unique(source_columns); + ActionsDAG actions_dag(source_columns); FunctionOverloadResolverPtr func_builder = std::make_unique(std::make_shared()); - for (const auto * input_node : actions_dag->getInputs()) + for (const auto * input_node : actions_dag.getInputs()) { const auto & column_name = input_node->result_name; auto it = column_names_to_wrap.find(column_name); @@ -485,9 +485,9 @@ static ActionsDAGPtr createWrapWithTupleActions( String node_name = "__wrapNullsafe(" + column_name + ")"; makeColumnNameUnique(source_columns, node_name); - const auto & dst_node = actions_dag->addFunction(func_builder, {input_node}, node_name); + const auto & dst_node = actions_dag.addFunction(func_builder, {input_node}, node_name); new_names[column_name] = dst_node.result_name; - actions_dag->addOrReplaceInOutputs(dst_node); + actions_dag.addOrReplaceInOutputs(dst_node); } if (!column_names_to_wrap.empty()) @@ -537,21 +537,23 @@ std::pair TableJoin::getKeysForNullSafeComparion(const Columns return {left_keys_to_wrap, right_keys_to_wrap}; } -static void mergeDags(ActionsDAGPtr & result_dag, ActionsDAGPtr && new_dag) +static void mergeDags(std::optional & result_dag, std::optional && new_dag) { + if (!new_dag) + return; if (result_dag) result_dag->mergeInplace(std::move(*new_dag)); else result_dag = std::move(new_dag); } -std::pair +std::pair, std::optional> TableJoin::createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) { - ActionsDAGPtr left_dag = nullptr; - ActionsDAGPtr right_dag = nullptr; + std::optional left_dag; + std::optional right_dag; /** If the types are not equal, we need to convert them to a common type. * Example: * SELECT * FROM t1 JOIN t2 ON t1.a = t2.b @@ -693,7 +695,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } } -static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, +static std::optional changeKeyTypes(const ColumnsWithTypeAndName & cols_src, const TableJoin::NameToTypeMap & type_mapping, bool add_new_cols, NameToNameMap & key_column_rename) @@ -710,7 +712,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, } } if (!has_some_to_do) - return nullptr; + return {}; return ActionsDAG::makeConvertingActions( /* source= */ cols_src, @@ -721,7 +723,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, /* new_names= */ &key_column_rename); } -static ActionsDAGPtr changeTypesToNullable( +static std::optional changeTypesToNullable( const ColumnsWithTypeAndName & cols_src, const NameSet & exception_cols) { @@ -737,7 +739,7 @@ static ActionsDAGPtr changeTypesToNullable( } if (!has_some_to_do) - return nullptr; + return {}; return ActionsDAG::makeConvertingActions( /* source= */ cols_src, @@ -748,29 +750,29 @@ static ActionsDAGPtr changeTypesToNullable( /* new_names= */ nullptr); } -ActionsDAGPtr TableJoin::applyKeyConvertToTable( +std::optional TableJoin::applyKeyConvertToTable( const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, JoinTableSide table_side, NameToNameMap & key_column_rename) { if (type_mapping.empty()) - return nullptr; + return {}; /// Create DAG to convert key columns - ActionsDAGPtr convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename); + auto convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename); applyRename(table_side, key_column_rename); return convert_dag; } -ActionsDAGPtr TableJoin::applyNullsafeWrapper( +std::optional TableJoin::applyNullsafeWrapper( const ColumnsWithTypeAndName & cols_src, const NameSet & columns_for_nullsafe_comparison, JoinTableSide table_side, NameToNameMap & key_column_rename) { if (columns_for_nullsafe_comparison.empty()) - return nullptr; + return {}; std::unordered_set column_names_to_wrap; for (const auto & name : columns_for_nullsafe_comparison) @@ -784,7 +786,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper( } /// Create DAG to wrap keys with tuple for null-safe comparison - ActionsDAGPtr null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename); + auto null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename); for (auto & clause : clauses) { for (size_t i : clause.nullsafe_compare_key_indexes) @@ -799,7 +801,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper( return null_safe_wrap_dag; } -ActionsDAGPtr TableJoin::applyJoinUseNullsConversion( +std::optional TableJoin::applyJoinUseNullsConversion( const ColumnsWithTypeAndName & cols_src, const NameToNameMap & key_column_rename) { @@ -809,8 +811,7 @@ ActionsDAGPtr TableJoin::applyJoinUseNullsConversion( exclude_columns.insert(it.second); /// Create DAG to make columns nullable if needed - ActionsDAGPtr add_nullable_dag = changeTypesToNullable(cols_src, exclude_columns); - return add_nullable_dag; + return changeTypesToNullable(cols_src, exclude_columns); } void TableJoin::setStorageJoin(std::shared_ptr storage) @@ -957,7 +958,7 @@ bool TableJoin::allowParallelHashJoin() const return true; } -ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const +ActionsDAG TableJoin::createJoinedBlockActions(ContextPtr context) const { ASTPtr expression_list = rightKeysList(); auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable()); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 8e83233e54c..a057d46b94d 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -202,19 +202,19 @@ private: Names requiredJoinedNames() const; /// Create converting actions and change key column names if required - ActionsDAGPtr applyKeyConvertToTable( + std::optional applyKeyConvertToTable( const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, JoinTableSide table_side, NameToNameMap & key_column_rename); - ActionsDAGPtr applyNullsafeWrapper( + std::optional applyNullsafeWrapper( const ColumnsWithTypeAndName & cols_src, const NameSet & columns_for_nullsafe_comparison, JoinTableSide table_side, NameToNameMap & key_column_rename); - ActionsDAGPtr applyJoinUseNullsConversion( + std::optional applyJoinUseNullsConversion( const ColumnsWithTypeAndName & cols_src, const NameToNameMap & key_column_rename); @@ -264,7 +264,7 @@ public: TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } - ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const; + ActionsDAG createJoinedBlockActions(ContextPtr context) const; const std::vector & getEnabledJoinAlgorithms() const { return join_algorithm; } @@ -379,7 +379,7 @@ public: /// Calculate converting actions, rename key columns in required /// For `USING` join we will convert key columns inplace and affect into types in the result table /// For `JOIN ON` we will create new columns with converted keys to join by. - std::pair + std::pair, std::optional> createConvertingActions( const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns); diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index 929999c8c37..27d79e86622 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -14,15 +14,15 @@ namespace DB { -ActionsDAGPtr addMissingDefaults( +ActionsDAG addMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, ContextPtr context, bool null_as_default) { - auto actions = std::make_unique(header.getColumnsWithTypeAndName()); - auto & index = actions->getOutputs(); + ActionsDAG actions(header.getColumnsWithTypeAndName()); + auto & index = actions.getOutputs(); /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. /// First, remember the offset columns for all arrays in the block. @@ -40,7 +40,7 @@ ActionsDAGPtr addMissingDefaults( if (group.empty()) group.push_back(nullptr); - group.push_back(actions->getInputs()[i]); + group.push_back(actions.getInputs()[i]); } } @@ -62,11 +62,11 @@ ActionsDAGPtr addMissingDefaults( { const auto & nested_type = array_type->getNestedType(); ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0); - const auto & constant = actions->addColumn({nested_column, nested_type, column.name}); + const auto & constant = actions.addColumn({nested_column, nested_type, column.name}); auto & group = nested_groups[offsets_name]; group[0] = &constant; - index.push_back(&actions->addFunction(func_builder_replicate, group, constant.result_name)); + index.push_back(&actions.addFunction(func_builder_replicate, group, constant.result_name)); continue; } @@ -75,17 +75,17 @@ ActionsDAGPtr addMissingDefaults( * it can be full (or the interpreter may decide that it is constant everywhere). */ auto new_column = column.type->createColumnConstWithDefaultValue(0); - const auto * col = &actions->addColumn({new_column, column.type, column.name}); - index.push_back(&actions->materializeNode(*col)); + const auto * col = &actions.addColumn({new_column, column.type, column.name}); + index.push_back(&actions.materializeNode(*col)); } /// Computes explicitly specified values by default and materialized columns. - if (auto dag = evaluateMissingDefaults(actions->getResultColumns(), required_columns, columns, context, true, null_as_default)) - actions = ActionsDAG::merge(std::move(*actions), std::move(*dag)); + if (auto dag = evaluateMissingDefaults(actions.getResultColumns(), required_columns, columns, context, true, null_as_default)) + actions = ActionsDAG::merge(std::move(actions), std::move(*dag)); /// Removes unused columns and reorders result. - actions->removeUnusedActions(required_columns.getNames(), false); - actions->addMaterializingOutputActions(); + actions.removeUnusedActions(required_columns.getNames(), false); + actions.addMaterializingOutputActions(); return actions; } diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 94afd806dfd..5299bae9745 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -24,7 +24,7 @@ using ActionsDAGPtr = std::unique_ptr; * Also can substitute NULL with DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. * All three types of columns are materialized (not constants). */ -ActionsDAGPtr addMissingDefaults( +ActionsDAG addMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false); } diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 4e1a2bcf5ee..d5d9fce0dbd 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -89,7 +89,7 @@ std::optional evaluateConstantExpressionImpl(c ColumnPtr result_column; DataTypePtr result_type; String result_name = ast->getColumnName(); - for (const auto & action_node : actions->getOutputs()) + for (const auto & action_node : actions.getOutputs()) { if ((action_node->result_name == result_name) && action_node->column) { diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index b000264ae33..62f8aea86d1 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -152,22 +152,20 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi return conversion_expr_list; } -ActionsDAGPtr createExpressions( +std::optional createExpressions( const Block & header, ASTPtr expr_list, bool save_unneeded_columns, ContextPtr context) { if (!expr_list) - return nullptr; + return {}; auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList()); auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context}; - auto dag = std::make_unique(header.getNamesAndTypesList()); + ActionsDAG dag(header.getNamesAndTypesList()); auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns); - dag = ActionsDAG::merge(std::move(*dag), std::move(*actions)); - - return dag; + return ActionsDAG::merge(std::move(dag), std::move(actions)); } } @@ -180,7 +178,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require if (auto dag = createExpressions(block, conversion_expr_list, true, context)) { - auto expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context)); + auto expression = std::make_shared(std::move(*dag), ExpressionActionsSettings::fromContext(context)); expression->execute(block); } } @@ -195,7 +193,7 @@ bool needConvertAnyNullToDefault(const Block & header, const NamesAndTypesList & return false; } -ActionsDAGPtr evaluateMissingDefaults( +std::optional evaluateMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, @@ -204,7 +202,7 @@ ActionsDAGPtr evaluateMissingDefaults( bool null_as_default) { if (!columns.hasDefaults() && (!null_as_default || !needConvertAnyNullToDefault(header, required_columns, columns))) - return nullptr; + return {}; ASTPtr expr_list = defaultRequiredExpressions(header, required_columns, columns, null_as_default); return createExpressions(header, expr_list, save_unneeded_columns, context); diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index ffc77561e79..570eb75dd4a 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -5,9 +5,6 @@ #include #include -#include -#include - namespace DB { @@ -24,12 +21,11 @@ struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; /// Create actions which adds missing defaults to block according to required_columns using columns description /// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. /// Return nullptr if no actions required. -ActionsDAGPtr evaluateMissingDefaults( +std::optional evaluateMissingDefaults( const Block & header, const NamesAndTypesList & required_columns, const ColumnsDescription & columns, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 16ee6de73c4..48e42099ce8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -332,12 +332,12 @@ void addExpressionStep(QueryPlan & query_plan, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = ActionsDAG::clone(&expression_actions->dag); + auto actions = std::move(*ActionsDAG::clone(&expression_actions->dag)); if (expression_actions->project_input) - actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); - auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); - appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets); + auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions)); + appendSetsFromActionsDAG(expression_step->getExpression(), useful_sets); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -347,15 +347,15 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); + auto actions = std::move(*ActionsDAG::clone(&filter_analysis_result.filter_actions->dag)); if (filter_analysis_result.filter_actions->project_input) - actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), - actions, + std::move(actions), filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); - appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets); + appendSetsFromActionsDAG(where_step->getExpression(), useful_sets); where_step->setStepDescription(step_description); query_plan.addStep(std::move(where_step)); } @@ -552,10 +552,10 @@ void addTotalsHavingStep(QueryPlan & query_plan, const auto & having_analysis_result = expression_analysis_result.getHaving(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); - ActionsDAGPtr actions; + std::optional actions; if (having_analysis_result.filter_actions) { - actions = ActionsDAG::clone(&having_analysis_result.filter_actions->dag); + actions = std::move(*ActionsDAG::clone(&having_analysis_result.filter_actions->dag)); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -564,7 +564,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, query_plan.getCurrentDataStream(), aggregation_analysis_result.aggregate_descriptions, query_analysis_result.aggregate_overflow_row, - actions, + std::move(actions), having_analysis_result.filter_column_name, having_analysis_result.remove_filter_column, settings.totals_mode, @@ -715,13 +715,13 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (query_node.hasInterpolate()) { - auto interpolate_actions_dag = std::make_unique(); + ActionsDAG interpolate_actions_dag; auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) { /// INTERPOLATE actions dag input columns must be non constant query_plan_column.column = nullptr; - interpolate_actions_dag->addInput(query_plan_column); + interpolate_actions_dag.addInput(query_plan_column); } auto & interpolate_list_node = query_node.getInterpolate()->as(); @@ -729,12 +729,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (interpolate_list_nodes.empty()) { - for (const auto * input_node : interpolate_actions_dag->getInputs()) + for (const auto * input_node : interpolate_actions_dag.getInputs()) { if (column_names_with_fill.contains(input_node->result_name)) continue; - interpolate_actions_dag->getOutputs().push_back(input_node); + interpolate_actions_dag.getOutputs().push_back(input_node); } } else @@ -744,12 +744,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, auto & interpolate_node_typed = interpolate_node->as(); PlannerActionsVisitor planner_actions_visitor(planner_context); - auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, + auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); - auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, + auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); if (interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); @@ -760,16 +760,16 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, const auto * interpolate_expression = interpolate_expression_nodes[0]; if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type)) { - interpolate_expression = &interpolate_actions_dag->addCast(*interpolate_expression, + interpolate_expression = &interpolate_actions_dag.addCast(*interpolate_expression, expression_to_interpolate->result_type, interpolate_expression->result_name); } - const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name); - interpolate_actions_dag->getOutputs().push_back(alias_node); + const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name); + interpolate_actions_dag.getOutputs().push_back(alias_node); } - interpolate_actions_dag->removeUnusedActions(); + interpolate_actions_dag.removeUnusedActions(); } Aliases empty_aliases; @@ -1130,7 +1130,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, return; auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - filter_info.actions, + std::move(*filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription("additional result filter"); @@ -1418,7 +1418,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(ActionsDAG::clone(filters.filter_actions)); + table_expression_data.setFilterActions(ActionsDAG::clone(&*filters.filter_actions)); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 59ec7778e21..4c0c9bc7937 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -757,12 +757,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } - auto lambda_actions_dag = std::make_unique(); - actions_stack.emplace_back(*lambda_actions_dag, node); + ActionsDAG lambda_actions_dag; + actions_stack.emplace_back(lambda_actions_dag, node); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); - lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); - lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); + lambda_actions_dag.getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag.removeUnusedActions(Names(1, lambda_expression_node_name)); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); @@ -879,14 +879,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi const auto & function_node = node->as(); auto function_node_name = action_node_name_helper.calculateActionNodeName(node); - auto index_hint_actions_dag = std::make_unique(); - auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); + ActionsDAG index_hint_actions_dag; + auto & index_hint_actions_dag_outputs = index_hint_actions_dag.getOutputs(); std::unordered_set index_hint_actions_dag_output_node_names; PlannerActionsVisitor actions_visitor(planner_context); for (const auto & argument : function_node.getArguments()) { - auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument); + auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument); for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes) { diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 418240fa34e..f35772ef7c0 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -25,7 +25,7 @@ class TableNode; struct FiltersForTableExpression { - ActionsDAGPtr filter_actions; + std::optional filter_actions; PrewhereInfoPtr prewhere_info; }; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 94054588d40..fa3a3483a8e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,19 +591,19 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) std::unique_ptr createComputeAliasColumnsStep( const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { - ActionsDAGPtr merged_alias_columns_actions_dag = std::make_unique(current_data_stream.header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); + ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs(); for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) { const auto & current_outputs = alias_column_actions_dag->getOutputs(); action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); - merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag)); + merged_alias_columns_actions_dag.mergeNodes(std::move(*alias_column_actions_dag)); } for (const auto * output_node : action_dag_outputs) - merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node); - merged_alias_columns_actions_dag->removeUnusedActions(false); + merged_alias_columns_actions_dag.addOrReplaceInOutputs(*output_node); + merged_alias_columns_actions_dag.removeUnusedActions(false); auto alias_column_step = std::make_unique(current_data_stream, std::move(merged_alias_columns_actions_dag)); alias_column_step->setStepDescription("Compute alias columns"); @@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); + prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(prewhere_actions)); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -805,14 +805,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = std::move(filter_info.actions); + prewhere_info->prewhere_actions = std::move(*filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = std::move(filter_info.actions); + prewhere_info->row_level_filter = std::move(*filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } @@ -831,7 +831,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(row_policy_filter_info.actions)); + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&*row_policy_filter_info.actions)); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) @@ -964,15 +964,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan.addStep(std::move(alias_column_step)); } - for (const auto & filter_info_and_description : where_filters) + for (auto && [filter_info, description] : where_filters) { - const auto & [filter_info, description] = filter_info_and_description; if (query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns && filter_info.actions) { auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - filter_info.actions, + std::move(*filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription(description); @@ -1063,19 +1062,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (from_stage == QueryProcessingStage::FetchColumns) { - auto rename_actions_dag = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; - for (auto & output_node : rename_actions_dag->getOutputs()) + for (auto & output_node : rename_actions_dag.getOutputs()) { const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); if (!column_identifier) continue; - updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier)); + updated_actions_dag_outputs.push_back(&rename_actions_dag.addAlias(*output_node, *column_identifier)); } - rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); + rename_actions_dag.getOutputs() = std::move(updated_actions_dag_outputs); auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); rename_step->setStepDescription("Change column names to column identifiers"); @@ -1117,9 +1116,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { - auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (auto & output_node : cast_actions_dag->getOutputs()) + for (auto & output_node : cast_actions_dag.getOutputs()) { if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) { @@ -1128,11 +1127,11 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP type_to_check = type_to_check_low_cardinality->getDictionaryType(); if (type_to_check->canBeInsideNullable()) - output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + output_node = &cast_actions_dag.addFunction(to_nullable_function, {output_node}, output_node->result_name); } } - cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); + cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); plan_to_add_cast.addStep(std::move(cast_join_columns_step)); @@ -1178,16 +1177,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); - join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); - auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); + join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); + auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions)); left_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); left_plan.addStep(std::move(left_join_expressions_actions_step)); - join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); - auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); + join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); + auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions)); right_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1225,19 +1224,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) { - auto cast_actions_dag = std::make_unique(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (auto & output_node : cast_actions_dag->getOutputs()) + for (auto & output_node : cast_actions_dag.getOutputs()) { auto it = plan_column_name_to_cast_type.find(output_node->result_name); if (it == plan_column_name_to_cast_type.end()) continue; const auto & cast_type = it->second; - output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name); + output_node = &cast_actions_dag.addCast(*output_node, cast_type, output_node->result_name); } - cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); + cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); @@ -1385,7 +1384,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ { ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( - std::move(join_clauses_and_actions.mixed_join_expressions_actions), + std::move(*join_clauses_and_actions.mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); @@ -1542,12 +1541,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - auto drop_unused_columns_after_join_actions_dag = std::make_unique(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; std::optional first_skipped_column_node_index; - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag->getOutputs(); + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) @@ -1619,7 +1618,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ auto plan = std::move(join_tree_query_plan.query_plan); auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr array_join_action_dag = std::make_unique(plan_output_columns); + ActionsDAG array_join_action_dag(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); std::unordered_set array_join_expressions_output_nodes; @@ -1630,28 +1629,28 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_column_names.insert(array_join_column_identifier); auto & array_join_expression_column = array_join_expression->as(); - auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); + auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); for (auto & expression_dag_index_node : expression_dag_index_nodes) { - const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier); - array_join_action_dag->getOutputs().push_back(array_join_column_node); + const auto * array_join_column_node = &array_join_action_dag.addAlias(*expression_dag_index_node, array_join_column_identifier); + array_join_action_dag.getOutputs().push_back(array_join_column_node); array_join_expressions_output_nodes.insert(array_join_column_node->result_name); } } - array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); + array_join_action_dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action_dag)); array_join_actions->setStepDescription("ARRAY JOIN actions"); - appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(array_join_actions->getExpression(), join_tree_query_plan.useful_sets); plan.addStep(std::move(array_join_actions)); - auto drop_unused_columns_before_array_join_actions_dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG drop_unused_columns_before_array_join_actions_dag(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_before_array_join_actions_dag_updated_outputs_names; - auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag->getOutputs(); + auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag.getOutputs(); size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size(); for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 23b6a805ab9..db9678d91a6 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -177,13 +177,13 @@ std::set extractJoinTableSidesFromExpression(//const ActionsDAG:: } const ActionsDAG::Node * appendExpression( - ActionsDAGPtr & dag, + ActionsDAG & dag, const QueryTreeNodePtr & expression, const PlannerContextPtr & planner_context, const JoinNode & join_node) { PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", @@ -193,9 +193,9 @@ const ActionsDAG::Node * appendExpression( } void buildJoinClause( - ActionsDAGPtr & left_dag, - ActionsDAGPtr & right_dag, - ActionsDAGPtr & mixed_dag, + ActionsDAG & left_dag, + ActionsDAG & right_dag, + ActionsDAG & mixed_dag, const PlannerContextPtr & planner_context, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, @@ -376,8 +376,8 @@ JoinClausesAndActions buildJoinClausesAndActions( const JoinNode & join_node, const PlannerContextPtr & planner_context) { - ActionsDAGPtr left_join_actions = std::make_unique(left_table_expression_columns); - ActionsDAGPtr right_join_actions = std::make_unique(right_table_expression_columns); + ActionsDAG left_join_actions(left_table_expression_columns); + ActionsDAG right_join_actions(right_table_expression_columns); ColumnsWithTypeAndName mixed_table_expression_columns; for (const auto & left_column : left_table_expression_columns) { @@ -387,7 +387,7 @@ JoinClausesAndActions buildJoinClausesAndActions( { mixed_table_expression_columns.push_back(right_column); } - ActionsDAGPtr mixed_join_actions = std::make_unique(mixed_table_expression_columns); + ActionsDAG mixed_join_actions(mixed_table_expression_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -498,12 +498,12 @@ JoinClausesAndActions buildJoinClausesAndActions( { const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (left_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {}); + dag_filter_condition_node = &left_join_actions.addFunction(and_function, left_filter_condition_nodes, {}); else dag_filter_condition_node = left_filter_condition_nodes[0]; join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node}; - left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + left_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); } @@ -514,12 +514,12 @@ JoinClausesAndActions buildJoinClausesAndActions( const ActionsDAG::Node * dag_filter_condition_node = nullptr; if (right_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {}); + dag_filter_condition_node = &right_join_actions.addFunction(and_function, right_filter_condition_nodes, {}); else dag_filter_condition_node = right_filter_condition_nodes[0]; join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node}; - right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + right_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); } @@ -556,10 +556,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } if (!left_key_node->result_type->equals(*common_type)) - left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); + left_key_node = &left_join_actions.addCast(*left_key_node, common_type, {}); if (!right_key_node->result_type->equals(*common_type)) - right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); + right_key_node = &right_join_actions.addCast(*right_key_node, common_type, {}); } if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable()) @@ -576,24 +576,24 @@ JoinClausesAndActions buildJoinClausesAndActions( * SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b) */ auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext()); - left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {}); - right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {}); + left_key_node = &left_join_actions.addFunction(wrap_nullsafe_function, {left_key_node}, {}); + right_key_node = &right_join_actions.addFunction(wrap_nullsafe_function, {right_key_node}, {}); } - left_join_actions->addOrReplaceInOutputs(*left_key_node); - right_join_actions->addOrReplaceInOutputs(*right_key_node); + left_join_actions.addOrReplaceInOutputs(*left_key_node); + right_join_actions.addOrReplaceInOutputs(*right_key_node); add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); } } - result.left_join_expressions_actions = ActionsDAG::clone(left_join_actions); - result.left_join_tmp_expression_actions = std::move(left_join_actions); - result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = ActionsDAG::clone(right_join_actions); - result.right_join_tmp_expression_actions = std::move(right_join_actions); - result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + result.left_join_expressions_actions = std::move(left_join_actions); + //result.left_join_tmp_expression_actions = std::move(left_join_actions); + result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); + result.right_join_expressions_actions = std::move(right_join_actions); + //result.right_join_tmp_expression_actions = std::move(right_join_actions); + result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); if (is_inequal_join) { @@ -601,16 +601,16 @@ JoinClausesAndActions buildJoinClausesAndActions( /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - auto mixed_join_expressions_actions = std::make_unique(mixed_table_expression_columns); + ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception( ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); - mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); + mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; - mixed_join_expressions_actions->removeUnusedActions(required_names); + mixed_join_expressions_actions.removeUnusedActions(required_names); result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); } else diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 8adf6edd7ea..3735c373acc 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -182,15 +182,15 @@ struct JoinClausesAndActions /// Join clauses. Actions dag nodes point into join_expression_actions. JoinClauses join_clauses; /// Whole JOIN ON section expressions - ActionsDAGPtr left_join_tmp_expression_actions; - ActionsDAGPtr right_join_tmp_expression_actions; + // ActionsDAGPtr left_join_tmp_expression_actions; + // ActionsDAGPtr right_join_tmp_expression_actions; /// Left join expressions actions - ActionsDAGPtr left_join_expressions_actions; + ActionsDAG left_join_expressions_actions; /// Right join expressions actions - ActionsDAGPtr right_join_expressions_actions; + ActionsDAG right_join_expressions_actions; /// Originally used for inequal join. it's the total join expression. /// If there is no inequal join conditions, it's null. - ActionsDAGPtr mixed_join_expressions_actions; + std::optional mixed_join_expressions_actions; }; /** Calculate join clauses and actions for JOIN ON section. diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 7ac53e0f8c1..e9f9c51d338 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -442,22 +442,22 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSets(filter_query_tree, *planner_context); - auto filter_actions_dag = std::make_unique(); + ActionsDAG filter_actions_dag; PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); + auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filter actions must return single output node. Actual {}", expression_nodes.size()); - auto & filter_actions_outputs = filter_actions_dag->getOutputs(); + auto & filter_actions_outputs = filter_actions_dag.getOutputs(); filter_actions_outputs = std::move(expression_nodes); std::string filter_node_name = filter_actions_outputs[0]->result_name; bool remove_filter_column = true; - for (const auto & filter_input_node : filter_actions_dag->getInputs()) + for (const auto & filter_input_node : filter_actions_dag.getInputs()) if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) filter_actions_outputs.push_back(filter_input_node); @@ -498,7 +498,7 @@ void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets); + appendSetsFromActionsDAG(index_hint->getActions(), useful_sets); } } } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 64ba7f7cd2a..f31de80b22d 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -303,15 +303,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B const auto & header = ports[set_counter]->getHeader(); /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - auto dag = std::make_unique(header.getColumnsWithTypeAndName()); + ActionsDAG dag(header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(output_header.columns() + 1); auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0); - const auto * grouping_node = &dag->addColumn( + const auto * grouping_node = &dag.addColumn( {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - grouping_node = &dag->materializeNode(*grouping_node); + grouping_node = &dag.materializeNode(*grouping_node); outputs.push_back(grouping_node); const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; @@ -332,21 +332,21 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B column_with_default->finalize(); auto column = ColumnConst::create(std::move(column_with_default), 0); - const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name}); - node = &dag->materializeNode(*node); + const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); + node = &dag.materializeNode(*node); outputs.push_back(node); } else { - const auto * column_node = dag->getOutputs()[header.getPositionByName(col.name)]; + const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)]; if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) - outputs.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name)); + outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); else outputs.push_back(column_node); } } - dag->getOutputs().swap(outputs); + dag.getOutputs().swap(outputs); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index b6c70061987..3a98f8e4612 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -36,27 +36,27 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_, ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) { - auto dag = std::make_unique(header.getColumnsWithTypeAndName()); - auto & outputs = dag->getOutputs(); + ActionsDAG dag(header.getColumnsWithTypeAndName()); + auto & outputs = dag.getOutputs(); if (use_nulls) { auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); for (const auto & key : keys) { - const auto * node = dag->getOutputs()[header.getPositionByName(key)]; + const auto * node = dag.getOutputs()[header.getPositionByName(key)]; if (node->result_type->canBeInsideNullable()) { - dag->addOrReplaceInOutputs(dag->addFunction(to_nullable, { node }, node->result_name)); + dag.addOrReplaceInOutputs(dag.addFunction(to_nullable, { node }, node->result_name)); } } } auto grouping_col = ColumnUInt64::create(1, grouping_set_number); - const auto * grouping_node = &dag->addColumn( + const auto * grouping_node = &dag.addColumn( {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - grouping_node = &dag->materializeNode(*grouping_node); + grouping_node = &dag.materializeNode(*grouping_node); outputs.insert(outputs.begin(), grouping_node); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 1f4f271fa6e..1c199ebedb3 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -32,7 +32,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi }; auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); plan.addStep(std::move(converting)); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 50bc2e1533e..94098f443d9 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -10,33 +10,33 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions, const Block & header, const SortDescription & sort_description) +static ITransformingStep::Traits getTraits(const ActionsDAG & actions, const Block & header, const SortDescription & sort_description) { return ITransformingStep::Traits { { .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = actions->isSortingPreserved(header, sort_description), + .preserves_sorting = actions.isSortingPreserved(header, sort_description), }, { - .preserves_number_of_rows = !actions->hasArrayJoin(), + .preserves_number_of_rows = !actions.hasArrayJoin(), } }; } -ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_) +ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_) : ITransformingStep( input_stream_, - ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), + ExpressionTransform::transformHeader(input_stream_.header, actions_dag_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) - , actions_dag(ActionsDAG::clone(actions_dag_)) + , actions_dag(std::move(actions_dag_)) { } void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression = std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()); + auto expression = std::make_shared(std::move(actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { @@ -61,25 +61,25 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); map.add("Expression", expression->toTree()); } void ExpressionStep::updateOutputStream() { output_stream = createOutputStream( - input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits()); + input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, actions_dag), getDataStreamTraits()); if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(*actions_dag); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index ebbac8217cb..f2926318cbc 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -1,12 +1,10 @@ #pragma once #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class ExpressionTransform; class JoiningTransform; @@ -15,21 +13,22 @@ class ExpressionStep : public ITransformingStep { public: - explicit ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_); + explicit ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_); String getName() const override { return "Expression"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions_dag; } + ActionsDAG & getExpression() { return actions_dag; } + const ActionsDAG & getExpression() const { return actions_dag; } void describeActions(JSONBuilder::JSONMap & map) const override; private: void updateOutputStream() override; - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; }; } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 7883461f45a..5f15c5defac 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -9,9 +9,9 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) +static ITransformingStep::Traits getTraits(const ActionsDAG & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) { - bool preserves_sorting = expression->isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); + bool preserves_sorting = expression.isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); if (remove_filter_column) { preserves_sorting &= std::find_if( @@ -35,22 +35,22 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, con FilterStep::FilterStep( const DataStream & input_stream_, - const ActionsDAGPtr & actions_dag_, + ActionsDAG actions_dag_, String filter_column_name_, bool remove_filter_column_) : ITransformingStep( input_stream_, FilterTransform::transformHeader( input_stream_.header, - actions_dag_.get(), + &actions_dag_, filter_column_name_, remove_filter_column_), getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_)) + , actions_dag(std::move(actions_dag_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { - actions_dag = ActionsDAG::clone(actions_dag_); - actions_dag->removeAliasesForFilter(filter_column_name); + actions_dag.removeAliasesForFilter(filter_column_name); } void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); map.add("Expression", expression->toTree()); } @@ -104,13 +104,13 @@ void FilterStep::updateOutputStream() { output_stream = createOutputStream( input_streams.front(), - FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column), + FilterTransform::transformHeader(input_streams.front().header, &actions_dag, filter_column_name, remove_filter_column), getDataStreamTraits()); if (!getDataStreamTraits().preserves_sorting) return; - FindAliasForInputName alias_finder(*actions_dag); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index 0f894a570b7..b5a31bef5fc 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -1,19 +1,17 @@ #pragma once #include +#include namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - /// Implements WHERE, HAVING operations. See FilterTransform. class FilterStep : public ITransformingStep { public: FilterStep( const DataStream & input_stream_, - const ActionsDAGPtr & actions_dag_, + ActionsDAG actions_dag_, String filter_column_name_, bool remove_filter_column_); @@ -23,15 +21,15 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions_dag; } - ActionsDAGPtr & getExpression() { return actions_dag; } + const ActionsDAG & getExpression() const { return actions_dag; } + ActionsDAG & getExpression() { return actions_dag; } const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } private: void updateOutputStream() override; - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; String filter_column_name; bool remove_filter_column; }; diff --git a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp index d90f0e152e7..be468419cfb 100644 --- a/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp @@ -45,10 +45,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan:: bool right_stream_safe = true; if (check_left_stream) - left_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header); + left_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header); if (check_right_stream) - right_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header); + right_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header); if (!left_stream_safe || !right_stream_safe) return 0; diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 6cdc3cb4eb0..8666912514e 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) steps_to_update.push_back(step); if (const auto * const expr = typeid_cast(step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index f26cd79dd97..411b20b1a32 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -101,7 +101,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node) return res; } -static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) +static std::optional splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) { QueryPlan::Node * child_node = parent_node->children.front(); checkChildrenSize(child_node, child_idx + 1); @@ -110,16 +110,16 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & av auto & child = child_node->step; auto * filter = assert_cast(parent.get()); - const auto & expression = filter->getExpression(); + auto & expression = filter->getExpression(); const auto & filter_column_name = filter->getFilterColumnName(); bool removes_filter = filter->removesFilterColumn(); const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); - return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); + return expression.splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); } static size_t -addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter, +addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, ActionsDAG split_filter, bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true) { QueryPlan::Node * child_node = parent_node->children.front(); @@ -129,14 +129,14 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, auto & child = child_node->step; auto * filter = assert_cast(parent.get()); - const auto & expression = filter->getExpression(); + auto & expression = filter->getExpression(); const auto & filter_column_name = filter->getFilterColumnName(); - const auto * filter_node = expression->tryFindInOutputs(filter_column_name); + const auto * filter_node = expression.tryFindInOutputs(filter_column_name); if (update_parent_filter && !filter_node && !filter->removesFilterColumn()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, expression->dumpDAG()); + filter_column_name, expression.dumpDAG()); /// Add new Filter step before Child. /// Expression/Filter -> Child -> Something @@ -147,10 +147,10 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, /// Expression/Filter -> Child -> Filter -> Something /// New filter column is the first one. - String split_filter_column_name = split_filter->getOutputs().front()->result_name; + String split_filter_column_name = split_filter.getOutputs().front()->result_name; node.step = std::make_unique( - node.children.at(0)->step->getOutputStream(), split_filter, std::move(split_filter_column_name), can_remove_filter); + node.children.at(0)->step->getOutputStream(), std::move(split_filter), std::move(split_filter_column_name), can_remove_filter); if (auto * transforming_step = dynamic_cast(child.get())) { @@ -176,7 +176,7 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, { /// This means that all predicates of filter were pushed down. /// Replace current actions to expression, as we don't need to filter anything. - parent = std::make_unique(child->getOutputStream(), expression); + parent = std::make_unique(child->getOutputStream(), std::move(expression)); } else { @@ -192,7 +192,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con bool can_remove_filter = true, size_t child_idx = 0) { if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) - return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx); + return addNewFilterStepOrThrow(parent_node, nodes, std::move(*split_filter), can_remove_filter, child_idx); return 0; } @@ -332,7 +332,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available); Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available); - auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), + auto join_filter_push_down_actions = filter->getExpression().splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), filter->removesFilterColumn(), left_stream_available_columns_to_push_down, left_stream_input_header, @@ -346,42 +346,44 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: if (join_filter_push_down_actions.left_stream_filter_to_push_down) { + const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name; updated_steps += addNewFilterStepOrThrow(parent_node, nodes, - join_filter_push_down_actions.left_stream_filter_to_push_down, + std::move(*join_filter_push_down_actions.left_stream_filter_to_push_down), join_filter_push_down_actions.left_stream_filter_removes_filter, 0 /*child_idx*/, false /*update_parent_filter*/); LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", - join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name, + result_name, JoinKind::Left); } if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right) { + const auto & result_name = join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name; updated_steps += addNewFilterStepOrThrow(parent_node, nodes, - join_filter_push_down_actions.right_stream_filter_to_push_down, + std::move(*join_filter_push_down_actions.right_stream_filter_to_push_down), join_filter_push_down_actions.right_stream_filter_removes_filter, 1 /*child_idx*/, false /*update_parent_filter*/); LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", - join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name, + result_name, JoinKind::Right); } if (updated_steps > 0) { const auto & filter_column_name = filter->getFilterColumnName(); - const auto & filter_expression = filter->getExpression(); + auto & filter_expression = filter->getExpression(); - const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name); + const auto * filter_node = filter_expression.tryFindInOutputs(filter_column_name); if (!filter_node && !filter->removesFilterColumn()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, filter_expression->dumpDAG()); + filter_column_name, filter_expression.dumpDAG()); /// Filter column was replaced to constant. @@ -391,7 +393,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan:: { /// This means that all predicates of filter were pushed down. /// Replace current actions to expression, as we don't need to filter anything. - parent = std::make_unique(child->getOutputStream(), filter_expression); + parent = std::make_unique(child->getOutputStream(), std::move(filter_expression)); } else { @@ -416,7 +418,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (!filter) return 0; - if (filter->getExpression()->hasStatefulFunctions()) + if (filter->getExpression().hasStatefulFunctions()) return 0; if (auto * aggregating = typeid_cast(child.get())) @@ -430,7 +432,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 0; const auto & actions = filter->getExpression(); - const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName()); + const auto & filter_node = actions.findInOutputs(filter->getFilterColumnName()); auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node); @@ -597,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - ActionsDAG::clone(filter->getExpression()), + std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -611,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp index 36aab41df49..0d4f2330119 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp @@ -28,10 +28,10 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node const auto & expression = expression_step ? expression_step->getExpression() : filter_step->getExpression(); - auto split_actions = expression->splitActionsBeforeArrayJoin(array_join->columns); + auto split_actions = expression.splitActionsBeforeArrayJoin(array_join->columns); /// No actions can be moved before ARRAY JOIN. - if (split_actions.first->trivial()) + if (split_actions.first.trivial()) return 0; auto description = parent->getStepDescription(); @@ -49,9 +49,9 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node array_join_step->updateInputStream(node.step->getOutputStream()); if (expression_step) - parent = std::make_unique(array_join_step->getOutputStream(), split_actions.second); + parent = std::make_unique(array_join_step->getOutputStream(), std::move(split_actions.second)); else - parent = std::make_unique(array_join_step->getOutputStream(), split_actions.second, + parent = std::make_unique(array_join_step->getOutputStream(), std::move(split_actions.second), filter_step->getFilterColumnName(), filter_step->removesFilterColumn()); parent->setStepDescription(description + " [split]"); diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index b280e2d3cc6..7794ddae8fa 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -66,13 +66,13 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: NameSet sort_columns; for (const auto & col : sorting_step->getSortDescription()) sort_columns.insert(col.column_name); - auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns); + auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression().splitActionsBySortingDescription(sort_columns); // No calculations can be postponed. - if (unneeded_for_sorting->trivial()) + if (unneeded_for_sorting.trivial()) return 0; - if (!areNodesConvertableToBlock(needed_for_sorting->getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting->getInputs())) + if (!areNodesConvertableToBlock(needed_for_sorting.getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting.getInputs())) return 0; // Sorting (parent_node) -> Expression (child_node) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 4629bc0af53..53f59198d0f 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - ActionsDAG::clone(expression->getExpression())); + std::move(*ActionsDAG::clone(&expression->getExpression()))); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 97de69b1134..d7ca96e4c64 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -38,18 +38,18 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (parent_expr && child_expr) { - const auto & child_actions = child_expr->getExpression(); - const auto & parent_actions = parent_expr->getExpression(); + auto & child_actions = child_expr->getExpression(); + auto & parent_actions = parent_expr->getExpression(); /// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them. /// Example: select rowNumberInBlock() from (select arrayJoin([1, 2])) /// Such a query will return two zeroes if we combine actions together. - if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions()) return 0; - auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); + auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions)); - auto expr = std::make_unique(child_expr->getInputStreams().front(), merged); + auto expr = std::make_unique(child_expr->getInputStreams().front(), std::move(merged)); expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); parent_node->step = std::move(expr); @@ -58,16 +58,16 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) } else if (parent_filter && child_expr) { - const auto & child_actions = child_expr->getExpression(); - const auto & parent_actions = parent_filter->getExpression(); + auto & child_actions = child_expr->getExpression(); + auto & parent_actions = parent_filter->getExpression(); - if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions()) return 0; - auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); + auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions)); auto filter = std::make_unique(child_expr->getInputStreams().front(), - merged, + std::move(merged), parent_filter->getFilterColumnName(), parent_filter->removesFilterColumn()); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); @@ -78,32 +78,31 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) } else if (parent_filter && child_filter) { - const auto & child_actions = child_filter->getExpression(); - const auto & parent_actions = parent_filter->getExpression(); + auto & child_actions = child_filter->getExpression(); + auto & parent_actions = parent_filter->getExpression(); - if (child_actions->hasArrayJoin()) + if (child_actions.hasArrayJoin()) return 0; - auto actions = ActionsDAG::clone(child_actions); - const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName()); + const auto & child_filter_node = child_actions.findInOutputs(child_filter->getFilterColumnName()); if (child_filter->removesFilterColumn()) - removeFromOutputs(*actions, child_filter_node); + removeFromOutputs(child_actions, child_filter_node); - actions->mergeInplace(std::move(*ActionsDAG::clone(parent_actions))); + child_actions.mergeInplace(std::move(parent_actions)); - const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); + const auto & parent_filter_node = child_actions.findInOutputs(parent_filter->getFilterColumnName()); if (parent_filter->removesFilterColumn()) - removeFromOutputs(*actions, parent_filter_node); + removeFromOutputs(child_actions, parent_filter_node); FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto & condition = actions->addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); - auto & outputs = actions->getOutputs(); + const auto & condition = child_actions.addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); + auto & outputs = child_actions.getOutputs(); outputs.insert(outputs.begin(), &condition); - actions->removeUnusedActions(false); + child_actions.removeUnusedActions(false); auto filter = std::make_unique(child_filter->getInputStreams().front(), - actions, + std::move(child_actions), condition.result_name, true); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")"); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index f203d831750..0d9e050d6cb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -83,10 +83,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) Names queried_columns = source_step_with_filter->requiredSourceColumns(); + const auto & source_filter_actions_dag = source_step_with_filter->getFilterActionsDAG(); MergeTreeWhereOptimizer where_optimizer{ std::move(column_compressed_sizes), storage_metadata, - storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context), + storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_filter_actions_dag ? &*source_filter_actions_dag : nullptr, context), queried_columns, storage.supportedPrewhereColumns(), getLogger("QueryPlanOptimizePrewhere")}; @@ -113,15 +114,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (prewhere_info->remove_prewhere_column) { - removeFromOutput(*filter_expression, filter_column_name); - auto & outputs = filter_expression->getOutputs(); + removeFromOutput(filter_expression, filter_column_name); + auto & outputs = filter_expression.getOutputs(); size_t size = outputs.size(); outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end()); - filter_expression->removeUnusedActions(false); + filter_expression.removeUnusedActions(false); outputs.resize(size); } - auto split_result = filter_expression->split(optimize_result.prewhere_nodes, true, true); + auto split_result = filter_expression.split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. @@ -137,15 +138,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// So, here we restore removed inputs for PREWHERE actions { std::unordered_set first_outputs( - split_result.first->getOutputs().begin(), split_result.first->getOutputs().end()); - for (const auto * input : split_result.first->getInputs()) + split_result.first.getOutputs().begin(), split_result.first.getOutputs().end()); + for (const auto * input : split_result.first.getInputs()) { if (!first_outputs.contains(input)) { - split_result.first->getOutputs().push_back(input); + split_result.first.getOutputs().push_back(input); /// Add column to second actions as input. /// Do not add it to result, so it would be removed. - split_result.second->addInput(input->result_name, input->result_type); + split_result.second.addInput(input->result_name, input->result_type); } } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 0afddede708..71a7ca327b1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(ActionsDAG::clone(filter_step->getExpression()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(ActionsDAG::clone(&filter_step->getExpression()), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index a8bd98d7460..b3747b81215 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -170,12 +170,12 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt } } -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +static void appendExpression(ActionsDAGPtr & dag, const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); + dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); else - dag = ActionsDAG::clone(expression); + dag = ActionsDAG::clone(&expression); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. @@ -193,7 +193,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & if (prewhere_info->prewhere_actions) { //std::cerr << "====== Adding prewhere " << std::endl; - appendExpression(dag, prewhere_info->prewhere_actions); + appendExpression(dag, *prewhere_info->prewhere_actions); if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); } @@ -211,7 +211,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & const auto & actions = expression->getExpression(); /// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array. - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) limit = 0; appendExpression(dag, actions); @@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index da057bd25c2..34e9c8aac0e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -273,7 +273,7 @@ static void appendAggregateFunctions( } } -ActionsDAGPtr analyzeAggregateProjection( +std::optional analyzeAggregateProjection( const AggregateProjectionInfo & info, const QueryDAG & query, const DAGIndex & query_index, @@ -393,7 +393,7 @@ ActionsDAGPtr analyzeAggregateProjection( // LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection"); auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes); - appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates); + appendAggregateFunctions(proj_dag, aggregates, *matched_aggregates); return proj_dag; } @@ -405,7 +405,7 @@ struct AggregateProjectionCandidate : public ProjectionCandidate /// Actions which need to be applied to columns from projection /// in order to get all the columns required for aggregation. - ActionsDAGPtr dag; + ActionsDAG dag; }; struct MinMaxProjectionCandidate @@ -480,13 +480,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)}; // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( metadata, - candidate.dag->getRequiredColumnsNames(), - (dag.filter_node ? dag.dag.get() : nullptr), + candidate.dag.getRequiredColumnsNames(), + (dag.filter_node ? &*dag.dag : nullptr), parts, max_added_blocks.get(), context); @@ -536,7 +536,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) { // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); - AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; + AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)}; candidate.projection = projection; candidates.real.emplace_back(std::move(candidate)); } @@ -664,7 +664,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu /// Selecting best candidate. for (auto & candidate : candidates.real) { - auto required_column_names = candidate.dag->getRequiredColumnsNames(); + auto required_column_names = candidate.dag.getRequiredColumnsNames(); bool analyzed = analyzeProjectionCandidate( candidate, @@ -675,7 +675,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu query_info, context, max_added_blocks, - candidate.dag.get()); + &candidate.dag); if (!analyzed) continue; @@ -765,7 +765,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = reader.readFromParts( /* parts = */ {}, /* alter_conversions = */ {}, - best_candidate->dag->getRequiredColumnsNames(), + best_candidate->dag.getRequiredColumnsNames(), proj_snapshot, projection_query_info, context, @@ -777,7 +777,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu if (!projection_reading) { - auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); + auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag.getRequiredColumnsNames()); Pipe pipe(std::make_shared(std::move(header))); projection_reading = std::make_unique(std::move(pipe)); } @@ -808,17 +808,19 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu if (best_candidate) { aggregate_projection_node = &nodes.emplace_back(); + if (candidates.has_filter) { + const auto & result_name = best_candidate->dag.getOutputs().front()->result_name; aggregate_projection_node->step = std::make_unique( projection_reading_node.step->getOutputStream(), - best_candidate->dag, - best_candidate->dag->getOutputs().front()->result_name, + std::move(best_candidate->dag), + result_name, true); } else aggregate_projection_node->step - = std::make_unique(projection_reading_node.step->getOutputStream(), best_candidate->dag); + = std::make_unique(projection_reading_node.step->getOutputStream(), std::move(best_candidate->dag)); aggregate_projection_node->children.push_back(&projection_reading_node); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index c7e96d66817..c0af178f08e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -23,7 +23,7 @@ struct NormalProjectionCandidate : public ProjectionCandidate { }; -static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_header) +static std::optional makeMaterializingDAG(const Block & proj_header, const Block main_header) { /// Materialize constants in case we don't have it in output header. /// This may happen e.g. if we have PREWHERE. @@ -31,7 +31,7 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block size_t num_columns = main_header.columns(); /// This is a error; will have block structure mismatch later. if (proj_header.columns() != num_columns) - return nullptr; + return {}; std::vector const_positions; for (size_t i = 0; i < num_columns; ++i) @@ -45,17 +45,17 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block } if (const_positions.empty()) - return nullptr; + return {}; - ActionsDAGPtr dag = std::make_unique(); - auto & outputs = dag->getOutputs(); + ActionsDAG dag; + auto & outputs = dag.getOutputs(); for (const auto & col : proj_header.getColumnsWithTypeAndName()) - outputs.push_back(&dag->addInput(col)); + outputs.push_back(&dag.addInput(col)); for (auto pos : const_positions) { auto & output = outputs[pos]; - output = &dag->materializeNode(*output); + output = &dag.materializeNode(*output); } return dag; @@ -172,7 +172,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod query_info, context, max_added_blocks, - query.filter_node ? query.dag.get() : nullptr); + query.filter_node ? &*query.dag : nullptr); if (!analyzed) continue; @@ -242,14 +242,14 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod { expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - query.dag, + std::move(*query.dag), query.filter_node->result_name, true); } else expr_or_filter_node.step = std::make_unique( projection_reading_node.step->getOutputStream(), - query.dag); + std::move(*query.dag)); expr_or_filter_node.children.push_back(&projection_reading_node); next_node = &expr_or_filter_node; @@ -267,7 +267,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) { - auto converting = std::make_unique(*proj_stream, materializing); + auto converting = std::make_unique(*proj_stream, std::move(*materializing)); proj_stream = &converting->getOutputStream(); auto & expr_node = nodes.emplace_back(); expr_node.step = std::move(converting); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 0e2ad96a419..fb2e6c2096e 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -64,12 +64,12 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea return {}; } -void QueryDAG::appendExpression(const ActionsDAGPtr & expression) +void QueryDAG::appendExpression(const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); + dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); else - dag = ActionsDAG::clone(expression); + dag = std::move(*ActionsDAG::clone(&expression)); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -120,7 +120,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & { if (prewhere_info->row_level_filter) { - appendExpression(prewhere_info->row_level_filter); + appendExpression(*prewhere_info->row_level_filter); if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false)) filter_nodes.push_back(filter_expression); else @@ -129,7 +129,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (prewhere_info->prewhere_actions) { - appendExpression(prewhere_info->prewhere_actions); + appendExpression(*prewhere_info->prewhere_actions); if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) filter_nodes.push_back(filter_expression); @@ -149,7 +149,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (auto * expression = typeid_cast(step)) { const auto & actions = expression->getExpression(); - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) return false; appendExpression(actions); @@ -159,7 +159,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (auto * filter = typeid_cast(step)) { const auto & actions = filter->getExpression(); - if (actions->hasArrayJoin()) + if (actions.hasArrayJoin()) return false; appendExpression(actions); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 59ad3a43b97..ee0dfddc326 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -25,14 +25,14 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. struct QueryDAG { - ActionsDAGPtr dag; + std::optional dag; const ActionsDAG::Node * filter_node = nullptr; bool build(QueryPlan::Node & node); private: bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes); - void appendExpression(const ActionsDAGPtr & expression); + void appendExpression(const ActionsDAG & expression); }; struct ProjectionCandidate diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index 81a8a537830..d0acd8221d4 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -132,10 +132,10 @@ namespace return true; if (const auto * const expr = typeid_cast(step); expr) - return !expr->getExpression()->hasArrayJoin(); + return !expr->getExpression().hasArrayJoin(); if (const auto * const filter = typeid_cast(step); filter) - return !filter->getExpression()->hasArrayJoin(); + return !filter->getExpression().hasArrayJoin(); if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) @@ -183,9 +183,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); if (inner_distinct_step = typeid_cast(node->step.get()); inner_distinct_step) @@ -236,9 +236,9 @@ namespace } if (const auto * const expr = typeid_cast(current_step); expr) - dag_stack.push_back(expr->getExpression().get()); + dag_stack.push_back(&expr->getExpression()); else if (const auto * const filter = typeid_cast(current_step); filter) - dag_stack.push_back(filter->getExpression().get()); + dag_stack.push_back(&filter->getExpression()); node = node->children.front(); inner_distinct_step = typeid_cast(node->step.get()); diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 632eba6ab5f..7cac7bee6ec 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -213,12 +213,12 @@ private: logStep("checking for stateful function", node); if (const auto * expr = typeid_cast(step); expr) { - if (expr->getExpression()->hasStatefulFunctions()) + if (expr->getExpression().hasStatefulFunctions()) return false; } else if (const auto * filter = typeid_cast(step); filter) { - if (filter->getExpression()->hasStatefulFunctions()) + if (filter->getExpression().hasStatefulFunctions()) return false; } else diff --git a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp index 561ad7302c6..6aed57634b0 100644 --- a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp +++ b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp @@ -17,13 +17,13 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) const std::string & filter_column_name = filter_step->getFilterColumnName(); /// Do not split if there are function like runningDifference. - if (expr->hasStatefulFunctions()) + if (expr.hasStatefulFunctions()) return 0; bool filter_name_clashs_with_input = false; if (filter_step->removesFilterColumn()) { - for (const auto * input : expr->getInputs()) + for (const auto * input : expr.getInputs()) { if (input->result_name == filter_column_name) { @@ -33,14 +33,14 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) } } - auto split = expr->splitActionsForFilter(filter_column_name); + auto split = expr.splitActionsForFilter(filter_column_name); - if (split.second->trivial()) + if (split.second.trivial()) return 0; bool remove_filter = false; if (filter_step->removesFilterColumn()) - remove_filter = split.second->removeUnusedResult(filter_column_name); + remove_filter = split.second.removeUnusedResult(filter_column_name); auto description = filter_step->getStepDescription(); @@ -53,11 +53,11 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) { split_filter_name = "__split_filter"; - for (auto & filter_output : split.first->getOutputs()) + for (auto & filter_output : split.first.getOutputs()) { if (filter_output->result_name == filter_column_name) { - filter_output = &split.first->addAlias(*filter_output, split_filter_name); + filter_output = &split.first.addAlias(*filter_output, split_filter_name); break; } } diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 124cb735d5a..7e0260c0040 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -146,16 +146,16 @@ bool allOutputsDependsOnlyOnAllowedNodes( /// 3. We match partition key actions with group by key actions to find col1', ..., coln' in partition key actions. /// 4. We check that partition key is indeed a deterministic function of col1', ..., coln'. bool isPartitionKeySuitsGroupByKey( - const ReadFromMergeTree & reading, const ActionsDAGPtr & group_by_actions, const AggregatingStep & aggregating) + const ReadFromMergeTree & reading, const ActionsDAG & group_by_actions, const AggregatingStep & aggregating) { if (aggregating.isGroupingSets()) return false; - if (group_by_actions->hasArrayJoin() || group_by_actions->hasStatefulFunctions() || group_by_actions->hasNonDeterministic()) + if (group_by_actions.hasArrayJoin() || group_by_actions.hasStatefulFunctions() || group_by_actions.hasNonDeterministic()) return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - auto key_nodes = group_by_actions->findInOutpus(aggregating.getParams().keys); + auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ed4b1906635..a12fce95b10 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -943,7 +943,7 @@ SplitPartsWithRangesByPrimaryKeyResult splitPartsWithRangesByPrimaryKey( auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes()); auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false); - reorderColumns(*actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName()); + reorderColumns(actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName()); ExpressionActionsPtr expression_actions = std::make_shared(std::move(actions)); auto description = fmt::format( "filter values in ({}, {}]", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf"); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index eca3cc54ce9..bc878e7ee49 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -799,7 +799,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); + auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -848,16 +848,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); } -static ActionsDAGPtr createProjection(const Block & header) +static ActionsDAG createProjection(const Block & header) { - return std::make_unique(header.getNamesAndTypesList()); + return ActionsDAG(header.getNamesAndTypesList()); } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names, - ActionsDAGPtr & out_projection, + std::optional & out_projection, const InputOrderInfoPtr & input_order_info) { const auto & settings = context->getSettingsRef(); @@ -1171,7 +1171,7 @@ bool ReadFromMergeTree::doNotMergePartsAcrossPartitionsFinal() const } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection) + RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional & out_projection) { const auto & settings = context->getSettingsRef(); const auto & data_settings = data.getSettings(); @@ -1212,7 +1212,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); + auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); if (prewhere_info) { @@ -1333,7 +1333,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (!merging_pipes.empty() && !no_merging_pipes.empty()) { - out_projection = nullptr; /// We do projection here + out_projection = {}; /// We do projection here Pipes pipes; pipes.resize(2); pipes[0] = Pipe::unitePipes(std::move(merging_pipes)); @@ -1519,7 +1519,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. - query_info.filter_actions_dag = std::move(filter_actions_dag); + if (filter_actions_dag) + query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); buildIndexes( indexes, @@ -1833,7 +1834,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const } Pipe ReadFromMergeTree::spreadMarkRanges( - RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) + RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional & result_projection) { const bool final = isQueryWithFinal(); Names column_names_to_read = result.column_names_to_read; @@ -1894,7 +1895,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( } } -Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection) +Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, std::optional & result_projection) { auto && parts_with_ranges = std::move(result.parts_with_ranges); @@ -1983,7 +1984,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. /// NOTE: It may lead to double computation of expressions. - ActionsDAGPtr result_projection; + std::optional result_projection; Pipe pipe = output_each_partition_through_separate_port ? groupStreamsByPartition(result, result_projection) @@ -2000,7 +2001,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(ActionsDAG::clone(result.sampling.filter_expression.get())); + auto sampling_actions = std::make_shared(std::move(*ActionsDAG::clone(result.sampling.filter_expression.get()))); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2013,12 +2014,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons Block cur_header = pipe.getHeader(); - auto append_actions = [&result_projection](ActionsDAGPtr actions) + auto append_actions = [&result_projection](ActionsDAG actions) { if (!result_projection) result_projection = std::move(actions); else - result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); + result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(actions)); }; if (result_projection) @@ -2038,7 +2039,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - auto projection_actions = std::make_shared(ActionsDAG::clone(result_projection)); + auto projection_actions = std::make_shared(std::move(*result_projection)); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, projection_actions); @@ -2133,7 +2134,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); expression->describeActions(format_settings.out, prefix); } @@ -2142,7 +2143,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); expression->describeActions(format_settings.out, prefix); } } @@ -2168,7 +2169,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2178,7 +2179,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index e32507e1f22..a12f53924c3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -243,9 +243,9 @@ private: Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings); Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit); - Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection); + Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional & result_projection); - Pipe groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection); + Pipe groupStreamsByPartition(AnalysisResult & result, std::optional & result_projection); Pipe spreadMarkRangesAmongStreams(RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names); @@ -253,13 +253,13 @@ private: RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names, - ActionsDAGPtr & out_projection, + std::optional & out_projection, const InputOrderInfoPtr & input_order_info); bool doNotMergePartsAcrossPartitionsFinal() const; Pipe spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); + RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional & out_projection); ReadFromMergeTree::AnalysisResult getAnalysisResult() const; diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 90fe609a17d..ca98f7c2110 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -441,7 +441,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() chassert(numbers_storage.step != UInt64{0}); /// Build rpn of query filters - KeyCondition condition(filter_actions_dag.get(), context, column_names, key_expression); + KeyCondition condition(filter_actions_dag ? &*filter_actions_dag : nullptr, context, column_names, key_expression); if (condition.extractPlainRanges(ranges)) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index 79b225e7f93..55c9b5e442e 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 91b62efa860..f7a030c0628 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -33,8 +33,8 @@ public: { } - const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; } - ActionsDAGPtr detachFilterActionsDAG() { return std::move(filter_actions_dag); } + const std::optional & getFilterActionsDAG() const { return filter_actions_dag; } + std::optional detachFilterActionsDAG() { return std::move(filter_actions_dag); } const SelectQueryInfo & getQueryInfo() const { return query_info; } const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } @@ -81,7 +81,7 @@ protected: ContextPtr context; std::optional limit; - ActionsDAGPtr filter_actions_dag; + std::optional filter_actions_dag; private: /// Will be cleared after applyFilters() is called. diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 19632b1862f..4aa4f10ac86 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -28,7 +28,7 @@ TotalsHavingStep::TotalsHavingStep( const DataStream & input_stream_, const AggregateDescriptions & aggregates_, bool overflow_row_, - const ActionsDAGPtr & actions_dag_, + std::optional actions_dag_, const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, @@ -38,7 +38,7 @@ TotalsHavingStep::TotalsHavingStep( input_stream_, TotalsHavingTransform::transformHeader( input_stream_.header, - actions_dag_.get(), + actions_dag_ ? &*actions_dag_ : nullptr, filter_column_, remove_filter_, final_, @@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep( getTraits(!filter_column_.empty())) , aggregates(aggregates_) , overflow_row(overflow_row_) - , actions_dag(ActionsDAG::clone(actions_dag_)) + , actions_dag(std::move(actions_dag_)) , filter_column_name(filter_column_) , remove_filter(remove_filter_) , totals_mode(totals_mode_) @@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - auto expression_actions = actions_dag ? std::make_shared(ActionsDAG::clone(actions_dag), settings.getActionsSettings()) : nullptr; + auto expression_actions = actions_dag ? std::make_shared(std::move(*actions_dag), settings.getActionsSettings()) : nullptr; auto totals_having = std::make_shared( pipeline.getHeader(), @@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(ActionsDAG::clone(actions_dag)); + auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); map.add("Expression", expression->toTree()); } } @@ -128,7 +128,7 @@ void TotalsHavingStep::updateOutputStream() input_streams.front(), TotalsHavingTransform::transformHeader( input_streams.front().header, - actions_dag.get(), + getActions(), filter_column_name, remove_filter, final, diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 52ef5437701..927b8d99de3 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -18,7 +19,7 @@ public: const DataStream & input_stream_, const AggregateDescriptions & aggregates_, bool overflow_row_, - const ActionsDAGPtr & actions_dag_, + std::optional actions_dag_, const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, @@ -32,7 +33,7 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getActions() const { return actions_dag; } + const ActionsDAG * getActions() const { return actions_dag ? &*actions_dag : nullptr; } private: void updateOutputStream() override; @@ -40,7 +41,7 @@ private: const AggregateDescriptions aggregates; bool overflow_row; - ActionsDAGPtr actions_dag; + std::optional actions_dag; String filter_column_name; bool remove_filter; TotalsMode totals_mode; diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h index fcf576637ff..cfd3eb236b7 100644 --- a/src/Processors/SourceWithKeyCondition.h +++ b/src/Processors/SourceWithKeyCondition.h @@ -16,13 +16,13 @@ protected: /// Represents pushed down filters in source std::shared_ptr key_condition; - void setKeyConditionImpl(const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & keys) + void setKeyConditionImpl(const std::optional & filter_actions_dag, ContextPtr context, const Block & keys) { key_condition = std::make_shared( - filter_actions_dag, + filter_actions_dag ? &*filter_actions_dag : nullptr, context, keys.getNames(), - std::make_shared(std::make_unique(keys.getColumnsWithTypeAndName()))); + std::make_shared(ActionsDAG(keys.getColumnsWithTypeAndName()))); } public: @@ -33,6 +33,6 @@ public: virtual void setKeyCondition(const std::shared_ptr & key_condition_) { key_condition = key_condition_; } /// Set key_condition created by filter_actions_dag and context. - virtual void setKeyCondition(const ActionsDAGPtr & /*filter_actions_dag*/, ContextPtr /*context*/) { } + virtual void setKeyCondition(const std::optional & /*filter_actions_dag*/, ContextPtr /*context*/) { } }; } diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp index 7945b3999c1..da4d3a0041b 100644 --- a/src/Processors/Transforms/AddingDefaultsTransform.cpp +++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp @@ -178,7 +178,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk) auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false); if (dag) { - auto actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); + auto actions = std::make_shared(std::move(*dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); actions->execute(evaluate_block); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index bbe57fc6441..36ffc515f43 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(ActionsDAG::clone(interpolate_description->actions)); + interpolate_actions = std::make_shared(std::move(*ActionsDAG::clone(&interpolate_description->actions))); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 2cd51259549..da5a45f36d5 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -516,7 +516,7 @@ void StorageHive::initMinMaxIndexExpression() partition_names = partition_name_types.getNames(); partition_types = partition_name_types.getTypes(); partition_minmax_idx_expr = std::make_shared( - std::make_unique(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); + ActionsDAG(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); } NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); @@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression() hivefile_name_types.push_back(column); } hivefile_minmax_idx_expr = std::make_shared( - std::make_unique(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); + ActionsDAG(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); } ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) @@ -583,7 +583,7 @@ static HiveFilePtr createHiveFile( HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -647,7 +647,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(filter_actions_dag.get(), getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -681,7 +681,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv HiveFilePtr StorageHive::getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level) const @@ -715,7 +715,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(filter_actions_dag.get(), getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply @@ -828,7 +828,7 @@ void ReadFromHive::createFiles() if (hive_files) return; - hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context); + hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag ? &*filter_actions_dag : nullptr, hive_table_metadata, fs, context); LOG_INFO(log, "Collect {} hive files to read", hive_files->size()); } @@ -950,7 +950,7 @@ void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const Bui HiveFiles StorageHive::collectHiveFiles( size_t max_threads, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -1023,12 +1023,12 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad std::optional StorageHive::totalRows(const Settings & settings) const { /// query_info is not used when prune_level == PruneLevel::None - return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None); + return totalRowsImpl(settings, {}, getContext(), PruneLevel::None); } -std::optional StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const +std::optional StorageHive::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const { - return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition); + return totalRowsImpl(context_->getSettingsRef(), &filter_actions_dag, context_, PruneLevel::Partition); } void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const @@ -1043,7 +1043,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt } std::optional -StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const +StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const { /// Row-based format like Text doesn't support totalRowsByPartitionPredicate if (!supportsSubsetOfColumns()) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 8a457dd6e01..e16df22e138 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -57,7 +57,7 @@ public: bool supportsSubsetOfColumns() const; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; protected: @@ -90,7 +90,7 @@ private: HiveFiles collectHiveFiles( size_t max_threads, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -98,7 +98,7 @@ private: HiveFiles collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, const ContextPtr & context_, @@ -107,7 +107,7 @@ private: HiveFilePtr getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, - const ActionsDAGPtr & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const HiveTableMetadataPtr & hive_table_metadata, const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; @@ -115,7 +115,7 @@ private: void lazyInitialize(); std::optional - totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; + totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; String hive_metastore_url; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 1f7ac23eb82..57f79a2cd7f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -236,7 +236,7 @@ StorageID IStorage::getStorageID() const return storage_id; } -ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const +ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const { return {}; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 98afd844046..c86f18d5d3b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -135,7 +135,7 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } - virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const; + virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const; /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// This is needed for engines whose aggregates data from multiple tables, like Merge. @@ -682,7 +682,7 @@ public: virtual std::optional totalRows(const Settings &) const { return {}; } /// Same as above but also take partition predicate into account. - virtual std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; } + virtual std::optional totalRowsByPartitionPredicate(const ActionsDAG &, ContextPtr) const { return {}; } /// If it is possible to quickly determine exact number of bytes for the table on storage: /// - memory (approximated, resident) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 94319aef3b8..88783246e10 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -231,7 +231,7 @@ bool traverseDAGFilter( } std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context) + const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context) { if (!filter_actions_dag) return {{}, true}; diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index e20a1ce4f37..64108290270 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -22,7 +22,7 @@ std::pair getFilterKeys( const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context); + const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context); template void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index e03ecc05064..7e43966556e 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -160,7 +160,7 @@ KeyDescription KeyDescription::buildEmptyKey() { KeyDescription result; result.expression_list_ast = std::make_shared(); - result.expression = std::make_shared(std::make_unique(), ExpressionActionsSettings{}); + result.expression = std::make_shared(ActionsDAG(), ExpressionActionsSettings{}); return result; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4ad7f6ef991..264b2b397f4 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -163,8 +163,8 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns if (dag) { dag->addMaterializingOutputActions(); - auto actions = std::make_shared< - ExpressionActions>(std::move(dag), + auto actions = std::make_shared( + std::move(*dag), ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef())); actions->execute(additional_columns); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 1efded3b064..d781345d834 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -628,7 +628,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { const auto & index_hint_dag = index_hint->getActions(); - children = index_hint_dag->getOutputs(); + children = index_hint_dag.getOutputs(); for (auto & arg : children) arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); @@ -729,7 +729,7 @@ Block KeyCondition::getBlockWithConstants( if (syntax_analyzer_result) { auto actions = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActionsDAG(); - for (const auto & action_node : actions->getOutputs()) + for (const auto & action_node : actions.getOutputs()) { if (action_node->column) result.insert(ColumnWithTypeAndName{action_node->column, action_node->result_type, action_node->result_name}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7b642c34f37..334c8c9c5ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -472,7 +472,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const } ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByPredicate( - const StorageSnapshotPtr & storage_snapshot, const ActionsDAGPtr & filter_dag, ContextPtr local_context) const + const StorageSnapshotPtr & storage_snapshot, const ActionsDAG * filter_dag, ContextPtr local_context) const { if (!local_context->getSettings().allow_statistics_optimize) return {}; @@ -487,7 +487,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP ASTPtr expression_ast; ConditionSelectivityEstimator result; - PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag.get(), local_context); + PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context); if (partition_pruner.isUseless()) { @@ -746,7 +746,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti if (!partition_key.column_names.empty()) partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); - return std::make_shared(std::make_unique(partition_key_columns), settings); + return std::make_shared(ActionsDAG(partition_key_columns), settings); } Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) @@ -1134,7 +1134,7 @@ Block MergeTreeData::getBlockWithVirtualsForFilter( std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( - const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const + const ActionsDAG & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const { if (parts.empty()) return 0; @@ -1142,7 +1142,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); if (!filter_dag) return {}; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 52916d85fef..e490e4b0bf9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -426,7 +426,7 @@ public: bool supportsPrewhere() const override { return true; } - ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const override; + ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const override; bool supportsFinal() const override; @@ -1227,7 +1227,7 @@ protected: boost::iterator_range range, const ColumnsDescription & storage_columns); std::optional totalRowsByPartitionPredicateImpl( - const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; + const ActionsDAG & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; static decltype(auto) getStateModifier(DataPartState state) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 61b8b6fdaa8..5a5b6d4a6e1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -428,7 +428,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( ASTPtr query = sampling.filter_function; auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); - sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); + sampling.filter_expression = std::make_shared(ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false)); } } @@ -466,7 +466,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( dag.get(), context, sample.getNames(), - std::make_shared(std::make_unique(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), + std::make_shared(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), {}}); } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 8d4ef69b1b9..ca31ffc9de5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,15 +265,15 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = ActionsDAG::clone(filter_dag); - const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); + auto filter_actions_dag = std::move(*ActionsDAG::clone(filter_dag)); + const auto * filter_actions_dag_node = filter_actions_dag.getOutputs().at(0); std::unordered_map node_to_result_node; - filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); + filter_actions_dag.getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); - filter_actions_dag->removeUnusedActions(); + filter_actions_dag.removeUnusedActions(); - actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; + actions_output_column_name = filter_actions_dag.getOutputs().at(0)->result_name; actions = std::make_shared(std::move(filter_actions_dag)); } @@ -306,7 +306,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx } -static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, const ActionsDAGPtr & result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage) +static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, ActionsDAG * result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage) { chassert(node.type == ActionsDAG::ActionType::FUNCTION); if (node.function_base->getName() != "indexHint") @@ -316,17 +316,17 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node const auto & adaptor = typeid_cast(*node.function_base); const auto & index_hint = typeid_cast(*adaptor.getFunction()); if (!result_dag_or_null) - return index_hint.getActions()->getOutputs(); + return index_hint.getActions().getOutputs(); /// Import the DAG and map argument pointers. - ActionsDAGPtr actions_clone = ActionsDAG::clone(index_hint.getActions()); + auto actions_clone = std::move(*ActionsDAG::clone(&index_hint.getActions())); chassert(storage); - result_dag_or_null->mergeNodes(std::move(*actions_clone), storage); + result_dag_or_null->mergeNodes(std::move(actions_clone), storage); return *storage; } const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const { @@ -348,7 +348,7 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA atom_node_ptr->type == ActionsDAG::ActionType::FUNCTION) { auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); - result_node = &result_dag->addFunction(bit_wrapper_function, {atom_node_ptr}, {}); + result_node = &result_dag.addFunction(bit_wrapper_function, {atom_node_ptr}, {}); } } else @@ -359,14 +359,14 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA unknown_field_column_with_type.type = std::make_shared(); unknown_field_column_with_type.column = unknown_field_column_with_type.type->createColumnConst(1, UNKNOWN_FIELD); - result_node = &result_dag->addColumn(unknown_field_column_with_type); + result_node = &result_dag.addColumn(unknown_field_column_with_type); } node_to_result_node.emplace(&node, result_node); return *result_node; } -const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAGPtr & result_dag, const ContextPtr & context) const +const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAG & result_dag, const ContextPtr & context) const { /// Function, literal or column @@ -386,7 +386,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA const auto * result_node = node_to_check; if (node.type != ActionsDAG::ActionType::INPUT) - result_node = &result_dag->addInput(column_name, node.result_type); + result_node = &result_dag.addInput(column_name, node.result_type); return result_node; } @@ -407,11 +407,11 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA return nullptr; } - return &result_dag->addFunction(node.function_base, children, {}); + return &result_dag.addFunction(node.function_base, children, {}); } const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const { @@ -429,7 +429,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio auto function_name = node_to_check->function->getName(); ActionsDAG::NodeRawConstPtrs temp_ptrs_to_argument; - const auto & arguments = getArguments(*node_to_check, result_dag, &temp_ptrs_to_argument); + const auto & arguments = getArguments(*node_to_check, &result_dag, &temp_ptrs_to_argument); size_t arguments_size = arguments.size(); if (function_name == "not") @@ -440,7 +440,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node); auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); - return &result_dag->addFunction(bit_swap_last_two_function, {argument}, {}); + return &result_dag.addFunction(bit_swap_last_two_function, {argument}, {}); } else if (function_name == "and" || function_name == "indexHint" || function_name == "or") { @@ -468,7 +468,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio const auto * before_last_argument = children.back(); children.pop_back(); - last_argument = &result_dag->addFunction(function, {before_last_argument, last_argument}, {}); + last_argument = &result_dag.addFunction(function, {before_last_argument, last_argument}, {}); } return last_argument; diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index abd40b3cf9d..03b02515e47 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -93,16 +93,16 @@ public: ~MergeTreeIndexConditionSet() override = default; private: const ActionsDAG::Node & traverseDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const; const ActionsDAG::Node * atomFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context) const; const ActionsDAG::Node * operatorFromDAG(const ActionsDAG::Node & node, - ActionsDAGPtr & result_dag, + ActionsDAG & result_dag, const ContextPtr & context, std::unordered_map & node_to_result_node) const; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index e924f853524..aec2f988e8d 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(ActionsDAG::clone(prewhere_info->row_level_filter), actions_settings), + .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(ActionsDAG::clone(prewhere_info->prewhere_actions), actions_settings), + .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 98b35a3ca2c..15917d59c9f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -349,7 +349,7 @@ public: MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, bool apply_deleted_mask_, - ActionsDAGPtr filter_, + std::optional filter_, ContextPtr context_, LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) @@ -376,7 +376,7 @@ public: { const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - KeyCondition key_condition(filter.get(), context, primary_key_column_names, primary_key.expression); + KeyCondition key_condition(&*filter, context, primary_key_column_names, primary_key.expression); LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); if (!key_condition.alwaysFalse()) @@ -417,7 +417,7 @@ private: MergeTreeData::DataPartPtr data_part; Names columns_to_read; bool apply_deleted_mask; - ActionsDAGPtr filter; + std::optional filter; ContextPtr context; LoggerPtr log; }; @@ -430,7 +430,7 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, Names columns_to_read, bool apply_deleted_mask, - ActionsDAGPtr filter, + std::optional filter, ContextPtr context, LoggerPtr log) { diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index e6f055f776c..1b05512b9a3 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -38,7 +38,7 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, Names columns_to_read, bool apply_deleted_mask, - ActionsDAGPtr filter, + std::optional filter, ContextPtr context, LoggerPtr log); diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 25596b42951..116edf5b9cb 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -349,7 +349,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction PrewhereExprStep new_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(step.actions), actions_settings), + .actions = std::make_shared(std::move(*step.actions), actions_settings), .filter_column_name = step.column_name, /// Don't remove if it's in the list of original outputs .remove_filter_column = diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index a9a5fddace4..8c389f00780 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -112,7 +112,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); } -MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, +MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAG & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final) @@ -126,7 +126,7 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize; RPNBuilderTreeContext tree_context(context); - RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); + RPNBuilderTreeNode node(&filter_dag.findInOutputs(filter_column_name), tree_context); auto optimize_result = optimizeImpl(node, where_optimizer_context); if (!optimize_result) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index ba6b4660924..a3d035675c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -52,7 +52,7 @@ public: bool fully_moved_to_prewhere = false; }; - FilterActionsOptimizeResult optimize(const ActionsDAGPtr & filter_dag, + FilterActionsOptimizeResult optimize(const ActionsDAG & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 4a18d606bb7..915a0e84902 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -398,7 +398,7 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const { const auto * adaptor = typeid_cast(dag_node->function_base.get()); const auto * index_hint = typeid_cast(adaptor->getFunction().get()); - return index_hint->getActions()->getOutputs().size(); + return index_hint->getActions().getOutputs().size(); } return dag_node->children.size(); @@ -426,7 +426,7 @@ RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const { const auto & adaptor = typeid_cast(*dag_node->function_base); const auto & index_hint = typeid_cast(*adaptor.getFunction()); - return RPNBuilderTreeNode(index_hint.getActions()->getOutputs()[index], tree_context); + return RPNBuilderTreeNode(index_hint.getActions().getOutputs()[index], tree_context); } return RPNBuilderTreeNode(dag_node->children[index], tree_context); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 503f542f2bd..d114608d8f1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -77,9 +77,9 @@ StorageObjectStorageSource::~StorageObjectStorageSource() create_reader_pool->wait(); } -void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +void StorageObjectStorageSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag.get(), context_, read_from_format_info.format_header); + setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); } std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index fd7c7aa7102..01ce980feaa 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -45,7 +45,7 @@ public: String getName() const override { return name; } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; Chunk generate() override; diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index e1d52eefc20..393d3f3fbb9 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -79,7 +79,7 @@ void readFinalFromNestedStorage( auto step = std::make_unique( query_plan.getCurrentDataStream(), - actions, + std::move(actions), filter_column_name, false); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 5f48d5e795e..5276870c037 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -46,9 +46,9 @@ struct PrewhereInfo { /// Actions for row level security filter. Applied separately before prewhere_actions. /// This actions are separate because prewhere condition should not be executed over filtered rows. - ActionsDAGPtr row_level_filter; + std::optional row_level_filter; /// Actions which are executed on block in order to get filter column for prewhere step. - ActionsDAGPtr prewhere_actions; + std::optional prewhere_actions; String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; @@ -56,7 +56,7 @@ struct PrewhereInfo bool generated_by_optimizer = false; PrewhereInfo() = default; - explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(std::optional prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} std::string dump() const; @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = ActionsDAG::clone(row_level_filter); + prewhere_info->row_level_filter = std::move(*ActionsDAG::clone(&*row_level_filter)); if (prewhere_actions) - prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); + prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(&*prewhere_actions)); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; @@ -93,7 +93,7 @@ struct FilterInfo /// Same as FilterInfo, but with ActionsDAG. struct FilterDAGInfo { - ActionsDAGPtr actions; + std::optional actions; String column_name; bool do_remove_column = false; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 695b31d0c80..fdddd84ab59 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -312,7 +312,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*ActionsDAG::clone(actions_dag)), + std::move(*ActionsDAG::clone(&actions_dag)), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -321,7 +321,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*ActionsDAG::clone(actions_dag)), + std::move(*ActionsDAG::clone(&actions_dag)), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -353,7 +353,7 @@ void StorageBuffer::read( header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag)); converting->setStepDescription("Convert destination table columns to Buffer table structure"); query_plan.addStep(std::move(converting)); @@ -432,7 +432,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(ActionsDAG::clone(query_info.prewhere_info->row_level_filter), actions_settings), + std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->row_level_filter)), actions_settings), query_info.prewhere_info->row_level_column_name, false); }); @@ -442,7 +442,7 @@ void StorageBuffer::read( { return std::make_shared( header, - std::make_shared(ActionsDAG::clone(query_info.prewhere_info->prewhere_actions), actions_settings), + std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->prewhere_actions)), actions_settings), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -472,7 +472,7 @@ void StorageBuffer::read( result_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); query_plan.addStep(std::move(converting)); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3d91da240cc..6f8a9189941 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1074,7 +1074,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu return pipeline; } -static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) +static std::optional getFilterFromQuery(const ASTPtr & ast, ContextPtr context) { QueryPlan plan; SelectQueryOptions options; @@ -1118,7 +1118,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) } if (!source) - return nullptr; + return {}; return source->detachFilterActionsDAG(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2422bcd700b..4611371a471 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1233,9 +1233,9 @@ StorageFileSource::~StorageFileSource() beforeDestroy(); } -void StorageFileSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +void StorageFileSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); + setKeyConditionImpl(filter_actions_dag, context_, block_for_format); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ac094aeb489..e9424527997 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -265,7 +265,7 @@ private: return storage->getName(); } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; bool tryGetCountFromCache(const struct stat & file_stat); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 316f398b476..8b6a9a4d4bb 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -273,8 +273,8 @@ void StorageMaterializedView::read( * They may be added in case of distributed query with JOIN. * In that case underlying table returns joined columns as well. */ - converting_actions->removeUnusedActions(); - auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); + converting_actions.removeUnusedActions(); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(converting_actions)); converting_step->setStepDescription("Convert target table structure to MaterializedView structure"); query_plan.addStep(std::move(converting_step)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c3fdad3a8f2..374abd0b0a5 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -407,7 +407,7 @@ void ReadFromMerge::addFilter(FilterDAGInfo filter) { output_stream->header = FilterTransform::transformHeader( output_stream->header, - filter.actions.get(), + filter.actions ? &*filter.actions : nullptr, filter.column_name, filter.do_remove_column); pushed_down_filters.push_back(std::move(filter)); @@ -628,7 +628,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); - column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); + column_names_as_aliases = alias_actions.getRequiredColumns().getNames(); if (column_names_as_aliases.empty()) column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - ActionsDAG::clone(filter_info.actions), + std::move(*ActionsDAG::clone(&*filter_info.actions)), filter_info.column_name, filter_info.do_remove_column); @@ -1060,7 +1060,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1074,7 +1074,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1089,7 +1089,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(database_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1102,7 +1102,7 @@ void ReadFromMerge::addVirtualColumns( column.column = column.type->createColumnConst(0, Field(table_name)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(adding_column_dag)); child.plan.addStep(std::move(expression_step)); plan_header = child.plan.getCurrentDataStream().header; } @@ -1240,7 +1240,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(ActionsDAG::clone(actions_dag), + filter_actions = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag)), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1278,12 +1278,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(ActionsDAG::clone(actions_dag), filter_column_name); + step->addFilter(ActionsDAG::clone(&actions_dag), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), ActionsDAG::clone(actions_dag), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&actions_dag)), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } @@ -1476,7 +1476,7 @@ void ReadFromMerge::convertAndFilterSourceStream( { pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); - auto actions_dag = std::make_unique(pipe_columns); + ActionsDAG actions_dag(pipe_columns); QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context); query_tree->setAlias(alias.name); @@ -1485,12 +1485,12 @@ void ReadFromMerge::convertAndFilterSourceStream( query_analysis_pass.run(query_tree, local_context); PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); - const auto & nodes = actions_visitor.visit(*actions_dag, query_tree); + const auto & nodes = actions_visitor.visit(actions_dag, query_tree); if (nodes.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); - actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); + actions_dag.addOrReplaceInOutputs(actions_dag.addAlias(*nodes.front(), alias.name)); auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } @@ -1506,7 +1506,7 @@ void ReadFromMerge::convertAndFilterSourceStream( auto dag = std::make_shared(pipe_columns); auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(actions_dag)); child.plan.addStep(std::move(expression_step)); } } @@ -1524,7 +1524,7 @@ void ReadFromMerge::convertAndFilterSourceStream( header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); - auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), convert_actions_dag); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), std::move(convert_actions_dag)); child.plan.addStep(std::move(expression_step)); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 94b34256d02..d6f2deca7fd 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -225,7 +225,7 @@ private: private: std::string filter_column_name; // complex filter, may contain logic operations - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; ExpressionActionsPtr filter_actions; StorageMetadataPtr storage_metadata_snapshot; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..b1a8a81914c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -245,7 +245,7 @@ std::optional StorageMergeTree::totalRows(const Settings &) const return getTotalActiveSizeInRows(); } -std::optional StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const +std::optional StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr local_context) const { auto parts = getVisibleDataPartsVector(local_context); return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4d819508934..56324449b34 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -65,7 +65,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings &) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr) const override; std::optional totalBytes(const Settings &) const override; std::optional totalBytesUncompressed(const Settings &) const override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..b472710b6d8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5556,7 +5556,7 @@ std::optional StorageReplicatedMergeTree::totalRows(const Settings & set return res; } -std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const +std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr local_context) const { DataPartsVector parts; foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f96206ce657..2e54f17d5d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -159,7 +159,7 @@ public: size_t num_streams) override; std::optional totalRows(const Settings & settings) const override; - std::optional totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override; + std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context) const override; std::optional totalBytes(const Settings & settings) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 9507eb6ed8a..345dd62c687 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -112,7 +112,7 @@ public: auto step = std::make_unique( query_plan.getCurrentDataStream(), - convert_actions_dag); + std::move(convert_actions_dag)); step->setStepDescription("Converting columns"); query_plan.addStep(std::move(step)); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index c336f597f41..ec1f803750e 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -185,9 +185,9 @@ public: String getName() const override { return name; } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override + void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override { - setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); + setKeyConditionImpl(filter_actions_dag, context_, block_for_format); } Chunk generate() override; diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index 4d73f8e5c87..c1ca6244866 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -48,13 +48,13 @@ Pipe StorageValues::read( if (!prepared_pipe.empty()) { - auto dag = std::make_unique(prepared_pipe.getHeader().getColumnsWithTypeAndName()); + ActionsDAG dag(prepared_pipe.getHeader().getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs outputs; outputs.reserve(column_names.size()); for (const auto & name : column_names) - outputs.push_back(dag->getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); + outputs.push_back(dag.getOutputs()[prepared_pipe.getHeader().getPositionByName(name)]); - dag->getOutputs().swap(outputs); + dag.getOutputs().swap(outputs); auto expression = std::make_shared(std::move(dag)); prepared_pipe.addSimpleTransform([&](const Block & header) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 2c0d5c5ca85..e2c4d67c8d1 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -177,8 +177,8 @@ void StorageView::read( /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - auto materializing_actions = std::make_unique(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - materializing_actions->addMaterializingOutputActions(); + ActionsDAG materializing_actions(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + materializing_actions.addMaterializingOutputActions(); auto materializing = std::make_unique(query_plan.getCurrentDataStream(), std::move(materializing_actions)); materializing->setStepDescription("Materialize constants after VIEW subquery"); @@ -203,7 +203,7 @@ void StorageView::read( expected_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting = std::make_unique(query_plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(query_plan.getCurrentDataStream(), std::move(convert_actions_dag)); converting->setStepDescription("Convert VIEW subquery result to VIEW table structure"); query_plan.addStep(std::move(converting)); } diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index ba7433fb9ae..f4e6fe3df5f 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -276,7 +276,7 @@ public: StackTraceSource( const Names & column_names, Block header_, - ActionsDAGPtr && filter_dag_, + std::optional filter_dag_, ContextPtr context_, UInt64 max_block_size_, LoggerPtr log_) @@ -422,7 +422,7 @@ protected: private: ContextPtr context; Block header; - const ActionsDAGPtr filter_dag; + const std::optional filter_dag; const ActionsDAG::Node * predicate; const size_t max_block_size; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 56f65b57367..a32eef20aed 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -176,12 +176,12 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); - const auto * col = &dag->findInOutputs(ast->getColumnName()); + const auto * col = &dag.findInOutputs(ast->getColumnName()); if (col->result_name != ttl_string) - col = &dag->addAlias(*col, ttl_string); + col = &dag.addAlias(*col, ttl_string); - dag->getOutputs() = {col}; - dag->removeUnusedActions(); + dag.getOutputs() = {col}; + dag.removeUnusedActions(); result.expression = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1bd5e80a4f9..7f54c6a6ee3 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,7 +80,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(ActionsDAG::clone(dag)); + auto actions = std::make_shared(std::move(*ActionsDAG::clone(dag))); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); @@ -318,9 +318,9 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = ActionsDAG::clone(index_hint->getActions()); + auto index_hint_dag = std::move(*ActionsDAG::clone(&index_hint->getActions())); ActionsDAG::NodeRawConstPtrs atoms; - for (const auto & output : index_hint_dag->getOutputs()) + for (const auto & output : index_hint_dag.getOutputs()) if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) atoms.push_back(child_copy); @@ -331,13 +331,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( if (atoms.size() > 1) { FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - res = &index_hint_dag->addFunction(func_builder_and, atoms, {}); + res = &index_hint_dag.addFunction(func_builder_and, atoms, {}); } if (!res->result_type->equals(*node->result_type)) - res = &index_hint_dag->addCast(*res, node->result_type, {}); + res = &index_hint_dag.addCast(*res, node->result_type, {}); - additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(*index_hint_dag))); + additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(index_hint_dag))); return res; } } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8f39f0da5af..30ae1f95593 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1130,7 +1130,7 @@ void StorageWindowView::read( { auto converting_actions = ActionsDAG::makeConvertingActions( target_header.getColumnsWithTypeAndName(), wv_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(converting_actions)); converting_step->setStepDescription("Convert Target table structure to WindowView structure"); query_plan.addStep(std::move(converting_step)); } From 17c6b97cbcb90e19c236708116f20ef3f88cc9c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Jul 2024 03:16:13 +0200 Subject: [PATCH 0115/1170] Fix error --- programs/local/LocalServer.cpp | 7 ++++++- src/Databases/IDatabase.h | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 41bb5604a52..269bffc2d56 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -737,7 +737,12 @@ void LocalServer::processConfig() DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); std::string default_database = server_settings.default_database; - DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context)); + { + DatabasePtr database = createClickHouseLocalDatabaseOverlay(default_database, global_context); + if (UUID uuid = database->getUUID(); uuid != UUIDHelpers::Nil) + DatabaseCatalog::instance().addUUIDMapping(uuid); + DatabaseCatalog::instance().attachDatabase(default_database, std::move(database)); + } global_context->setCurrentDatabase(default_database); if (getClientConfiguration().has("path")) diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b00f2fe4baf..3065c8ae6b5 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -358,6 +358,7 @@ public: std::lock_guard lock{mutex}; return database_name; } + /// Get UUID of database. virtual UUID getUUID() const { return UUIDHelpers::Nil; } From 1237f93182db21f00df9ca7913619ee63d75850b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Jul 2024 15:06:52 +0000 Subject: [PATCH 0116/1170] Fixing some crashes. --- src/Interpreters/ExpressionActions.cpp | 2 +- src/Planner/Planner.cpp | 8 +++++--- src/Planner/PlannerJoins.cpp | 8 ++++---- src/Planner/PlannerJoins.h | 4 ++-- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 399f4f2ff4f..1c6c3f2556b 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -59,7 +59,7 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) - actions_dag->compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); + actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); #endif linearizeActions(lazy_executed_nodes); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 48e42099ce8..0b10cef82ce 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -212,9 +212,11 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & if (!read_from_dummy) continue; - auto filter_actions = read_from_dummy->detachFilterActionsDAG(); - const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); - res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; + if (auto filter_actions = read_from_dummy->detachFilterActionsDAG()) + { + const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); + res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; + } } return res; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index db9678d91a6..7772336f7c0 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -588,11 +588,11 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = std::move(left_join_actions); - //result.left_join_tmp_expression_actions = std::move(left_join_actions); + result.left_join_expressions_actions = std::move(*ActionsDAG::clone(&left_join_actions)); + result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = std::move(right_join_actions); - //result.right_join_tmp_expression_actions = std::move(right_join_actions); + result.right_join_expressions_actions = std::move(*ActionsDAG::clone(&right_join_actions)); + result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); if (is_inequal_join) diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 3735c373acc..d8665ab7739 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -182,8 +182,8 @@ struct JoinClausesAndActions /// Join clauses. Actions dag nodes point into join_expression_actions. JoinClauses join_clauses; /// Whole JOIN ON section expressions - // ActionsDAGPtr left_join_tmp_expression_actions; - // ActionsDAGPtr right_join_tmp_expression_actions; + ActionsDAG left_join_tmp_expression_actions; + ActionsDAG right_join_tmp_expression_actions; /// Left join expressions actions ActionsDAG left_join_expressions_actions; /// Right join expressions actions diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bc878e7ee49..9ca79fde26f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1520,7 +1520,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); + query_info.filter_actions_dag = std::make_shared(std::move(*ActionsDAG::clone(&*filter_actions_dag))); buildIndexes( indexes, From a2b17b01f9561fd1853851932a2ae77513c49e26 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Jul 2024 17:47:56 +0000 Subject: [PATCH 0117/1170] fix filling of missed subcolumns --- src/Interpreters/inplaceBlockConversions.cpp | 38 ++++++++---- src/Storages/MergeTree/IMergeTreeReader.cpp | 48 +++++++++++---- .../MergeTree/MergeTreeReaderCompact.cpp | 60 ++++++++++++------- .../MergeTree/MergeTreeReaderCompact.h | 1 + .../03203_fill_missed_subcolumns.reference | 31 ++++++++++ .../03203_fill_missed_subcolumns.sql | 47 +++++++++++++++ 6 files changed, 181 insertions(+), 44 deletions(-) create mode 100644 tests/queries/0_stateless/03203_fill_missed_subcolumns.reference create mode 100644 tests/queries/0_stateless/03203_fill_missed_subcolumns.sql diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 239cce5b427..f7d8a2a2daf 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -35,8 +36,13 @@ namespace /// Add all required expressions for missing columns calculation void addDefaultRequiredExpressionsRecursively( - const Block & block, const String & required_column_name, DataTypePtr required_column_type, - const ColumnsDescription & columns, ASTPtr default_expr_list_accum, NameSet & added_columns, bool null_as_default) + const Block & block, + const String & required_column_name, + DataTypePtr required_column_type, + const ColumnsDescription & columns, + ASTPtr default_expr_list_accum, + NameSet & added_columns, + bool null_as_default) { checkStackSize(); @@ -273,6 +279,20 @@ static std::unordered_map collectOffsetsColumns( return offsets_columns; } +static ColumnPtr createColumnWithDefaultValue(const IDataType & data_type, const String & subcolumn_name, size_t num_rows) +{ + auto column = data_type.createColumnConstWithDefaultValue(num_rows); + + if (subcolumn_name.empty()) + return column->convertToFullColumnIfConst(); + + /// Firstly get subcolumn from const column and then replicate. + column = assert_cast(*column).getDataColumnPtr(); + column = data_type.getSubcolumn(subcolumn_name, column); + + return ColumnConst::create(std::move(column), num_rows)->convertToFullColumnIfConst(); +} + void fillMissingColumns( Columns & res_columns, size_t num_rows, @@ -298,21 +318,19 @@ void fillMissingColumns( auto requested_column = requested_columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++requested_column) { - const auto & [name, type] = *requested_column; - - if (res_columns[i] && partially_read_columns.contains(name)) + if (res_columns[i] && partially_read_columns.contains(requested_column->name)) res_columns[i] = nullptr; if (res_columns[i]) continue; - if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(name)) + if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(requested_column->getNameInStorage())) continue; std::vector current_offsets; size_t num_dimensions = 0; - const auto * array_type = typeid_cast(type.get()); + const auto * array_type = typeid_cast(requested_column->type.get()); if (array_type && !offsets_columns.empty()) { num_dimensions = getNumberOfDimensions(*array_type); @@ -348,10 +366,10 @@ void fillMissingColumns( if (!current_offsets.empty()) { size_t num_empty_dimensions = num_dimensions - current_offsets.size(); - auto scalar_type = createArrayOfType(getBaseTypeOfArray(type), num_empty_dimensions); + auto scalar_type = createArrayOfType(getBaseTypeOfArray(requested_column->getTypeInStorage()), num_empty_dimensions); size_t data_size = assert_cast(*current_offsets.back()).getData().back(); - res_columns[i] = scalar_type->createColumnConstWithDefaultValue(data_size)->convertToFullColumnIfConst(); + res_columns[i] = createColumnWithDefaultValue(*scalar_type, requested_column->getSubcolumnName(), data_size); for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) res_columns[i] = ColumnArray::create(res_columns[i], *it); @@ -360,7 +378,7 @@ void fillMissingColumns( { /// We must turn a constant column into a full column because the interpreter could infer /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. - res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst(); + res_columns[i] = createColumnWithDefaultValue(*requested_column->getTypeInStorage(), requested_column->getSubcolumnName(), num_rows); } } } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 4ad7f6ef991..aff1001163e 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -144,19 +145,26 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns throw Exception(ErrorCodes::LOGICAL_ERROR, "invalid number of columns passed to MergeTreeReader::fillMissingColumns. " "Expected {}, got {}", num_columns, res_columns.size()); - /// Convert columns list to block. - /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. - auto name_and_type = requested_columns.begin(); - for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) - { - if (res_columns[pos] == nullptr) - continue; + NameSet full_requested_columns_set; + NamesAndTypesList full_requested_columns; - additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name}); + /// Convert columns list to block. And convert subcolumns to full columns. + /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. + + auto it = requested_columns.begin(); + for (size_t pos = 0; pos < num_columns; ++pos, ++it) + { + auto name_in_storage = it->getNameInStorage(); + + if (full_requested_columns_set.emplace(name_in_storage).second) + full_requested_columns.emplace_back(name_in_storage, it->getTypeInStorage()); + + if (res_columns[pos]) + additional_columns.insert({res_columns[pos], it->type, it->name}); } auto dag = DB::evaluateMissingDefaults( - additional_columns, requested_columns, + additional_columns, full_requested_columns, storage_snapshot->metadata->getColumns(), data_part_info_for_read->getContext()); @@ -170,9 +178,18 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } /// Move columns from block. - name_and_type = requested_columns.begin(); - for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) - res_columns[pos] = std::move(additional_columns.getByName(name_and_type->name).column); + it = requested_columns.begin(); + for (size_t pos = 0; pos < num_columns; ++pos, ++it) + { + auto name_in_storage = it->getNameInStorage(); + res_columns[pos] = additional_columns.getByName(name_in_storage).column; + + if (it->isSubcolumn()) + { + const auto & type_in_storage = it->getTypeInStorage(); + res_columns[pos] = type_in_storage->getSubcolumn(it->getSubcolumnName(), res_columns[pos]); + } + } } catch (Exception & e) { @@ -192,7 +209,12 @@ bool IMergeTreeReader::isSubcolumnOffsetsOfNested(const String & name_in_storage if (!data_part_info_for_read->isWidePart() || subcolumn_name != "size0") return false; - return Nested::isSubcolumnOfNested(name_in_storage, part_columns); + auto split = Nested::splitName(name_in_storage); + if (split.second.empty()) + return false; + + auto nested_column = part_columns.tryGetColumn(GetColumnsOptions::All, split.first); + return nested_column && isNested(nested_column->type); } String IMergeTreeReader::getColumnNameInPart(const NameAndTypePair & required_column) const diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a2b8f0ad96f..fde9dafffb8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -60,39 +60,25 @@ void MergeTreeReaderCompact::fillColumnPositions() for (size_t i = 0; i < columns_num; ++i) { - const auto & column_to_read = columns_to_read[i]; - + auto & column_to_read = columns_to_read[i]; auto position = data_part_info_for_read->getColumnPosition(column_to_read.getNameInStorage()); - bool is_array = isArray(column_to_read.type); if (column_to_read.isSubcolumn()) { - auto storage_column_from_part = getColumnInPart( - {column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()}); + NameAndTypePair column_in_storage{column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()}; + auto storage_column_from_part = getColumnInPart(column_in_storage); auto subcolumn_name = column_to_read.getSubcolumnName(); if (!storage_column_from_part.type->hasSubcolumn(subcolumn_name)) position.reset(); } + column_positions[i] = std::move(position); + /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - if (!position && is_array) - { - auto column_to_read_with_subcolumns = getColumnConvertedToSubcolumnOfNested(column_to_read); - auto name_level_for_offsets = findColumnForOffsets(column_to_read_with_subcolumns); - - if (name_level_for_offsets.has_value()) - { - column_positions[i] = data_part_info_for_read->getColumnPosition(name_level_for_offsets->first); - columns_for_offsets[i] = name_level_for_offsets; - partially_read_columns.insert(column_to_read.name); - } - } - else - { - column_positions[i] = std::move(position); - } + if (!column_positions[i]) + findPositionForMissedNested(i); } } @@ -125,6 +111,38 @@ NameAndTypePair MergeTreeReaderCompact::getColumnConvertedToSubcolumnOfNested(co Nested::concatenateName(name_in_storage, subcolumn_name)); } +void MergeTreeReaderCompact::findPositionForMissedNested(size_t pos) +{ + auto & column = columns_to_read[pos]; + + bool is_array = isArray(column.type); + bool is_offsets_subcolumn = isArray(column.getTypeInStorage()) && column.getSubcolumnName() == "size0"; + + if (!is_array && !is_offsets_subcolumn) + return; + + NameAndTypePair column_in_storage{column.getNameInStorage(), column.getTypeInStorage()}; + + auto column_to_read_with_subcolumns = getColumnConvertedToSubcolumnOfNested(column_in_storage); + auto name_level_for_offsets = findColumnForOffsets(column_to_read_with_subcolumns); + + if (!name_level_for_offsets) + return; + + column_positions[pos] = data_part_info_for_read->getColumnPosition(name_level_for_offsets->first); + + if (is_offsets_subcolumn) + { + /// Read offsets from antoher array from the same Nested column. + column = {name_level_for_offsets->first, column.getSubcolumnName(), column.getTypeInStorage(), column.type}; + } + else + { + columns_for_offsets[pos] = std::move(name_level_for_offsets); + partially_read_columns.insert(column.name); + } +} + void MergeTreeReaderCompact::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index a783e595af5..22eabd47930 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -36,6 +36,7 @@ public: protected: void fillColumnPositions(); NameAndTypePair getColumnConvertedToSubcolumnOfNested(const NameAndTypePair & column); + void findPositionForMissedNested(size_t pos); using InputStreamGetter = ISerialization::InputStreamGetter; diff --git a/tests/queries/0_stateless/03203_fill_missed_subcolumns.reference b/tests/queries/0_stateless/03203_fill_missed_subcolumns.reference new file mode 100644 index 00000000000..872eb7da3c8 --- /dev/null +++ b/tests/queries/0_stateless/03203_fill_missed_subcolumns.reference @@ -0,0 +1,31 @@ +0 +2 +4 +6 +8 +0 +2 +4 +6 +8 +1 ['aaa',NULL] [NULL,NULL] +2 ['ccc'] [NULL] +3 [NULL] [NULL] +4 [NULL,'bbb'] ['ddd',NULL] +5 [NULL] [NULL] +1 2 2 +2 1 1 +3 1 1 +4 2 2 +5 1 1 +1 [0,1] [1,1] +2 [0] [1] +3 [1] [1] +4 [1,0] [0,1] +5 [1] [1] +1 ('foo','bar') [1,NULL,3] +2 ('aaa','bbb') [1,NULL,3] +3 ('ccc','ddd') [4,5,6] +1 foo bar 3 [0,1,0] +2 foo bar 3 [0,1,0] +3 ccc ddd 3 [0,0,0] diff --git a/tests/queries/0_stateless/03203_fill_missed_subcolumns.sql b/tests/queries/0_stateless/03203_fill_missed_subcolumns.sql new file mode 100644 index 00000000000..2789c9de35c --- /dev/null +++ b/tests/queries/0_stateless/03203_fill_missed_subcolumns.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS t_missed_subcolumns; + +-- .null subcolumn + +CREATE TABLE t_missed_subcolumns (x UInt32) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_missed_subcolumns SELECT * FROM numbers(10); + +ALTER TABLE t_missed_subcolumns ADD COLUMN `y` Nullable(UInt32); + +INSERT INTO t_missed_subcolumns SELECT number, if(number % 2, NULL, number) FROM numbers(10); + +SELECT x FROM t_missed_subcolumns WHERE y IS NOT NULL SETTINGS optimize_functions_to_subcolumns = 1; +SELECT x FROM t_missed_subcolumns WHERE y IS NOT NULL SETTINGS optimize_functions_to_subcolumns = 0; + +DROP TABLE IF EXISTS t_missed_subcolumns; + +-- .null and .size0 subcolumn in array + +CREATE TABLE t_missed_subcolumns (id UInt64, `n.a` Array(Nullable(String))) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_missed_subcolumns VALUES (1, ['aaa', NULL]) (2, ['ccc']) (3, [NULL]); +ALTER TABLE t_missed_subcolumns ADD COLUMN `n.b` Array(Nullable(String)); +INSERT INTO t_missed_subcolumns VALUES (4, [NULL, 'bbb'], ['ddd', NULL]), (5, [NULL], [NULL]); + +SELECT id, n.a, n.b FROM t_missed_subcolumns ORDER BY id; +SELECT id, n.a.size0, n.b.size0 FROM t_missed_subcolumns ORDER BY id; +SELECT id, n.a.null, n.b.null FROM t_missed_subcolumns ORDER BY id; + +DROP TABLE IF EXISTS t_missed_subcolumns; + +-- subcolumns and custom defaults + +CREATE TABLE t_missed_subcolumns (id UInt64) ENGINE = MergeTree ORDER BY id; +SYSTEM STOP MERGES t_missed_subcolumns; + +INSERT INTO t_missed_subcolumns VALUES (1); + +ALTER TABLE t_missed_subcolumns ADD COLUMN t Tuple(a String, b String) DEFAULT ('foo', 'bar'); +INSERT INTO t_missed_subcolumns VALUES (2, ('aaa', 'bbb')); + +ALTER TABLE t_missed_subcolumns ADD COLUMN arr Array(Nullable(UInt64)) DEFAULT [1, NULL, 3]; +INSERT INTO t_missed_subcolumns VALUES (3, ('ccc', 'ddd'), [4, 5, 6]); + +SELECT id, t, arr FROM t_missed_subcolumns ORDER BY id; +SELECT id, t.a, t.b, arr.size0, arr.null FROM t_missed_subcolumns ORDER BY id; + +DROP TABLE t_missed_subcolumns; From 1505cb2b788b1e7d1f5721117f33e061e7ee21e1 Mon Sep 17 00:00:00 2001 From: skyoct Date: Tue, 9 Jul 2024 13:33:00 +0800 Subject: [PATCH 0118/1170] 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 849fb83c9770fedb937dc59df73c0cc172e115bf Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Jul 2024 17:37:17 +0800 Subject: [PATCH 0119/1170] add function printf --- .../functions/string-replace-functions.md | 25 ++ src/Functions/printf.cpp | 308 ++++++++++++++++++ .../0_stateless/032010_printf.reference | 16 + tests/queries/0_stateless/032010_printf.sql | 24 ++ 4 files changed, 373 insertions(+) create mode 100644 src/Functions/printf.cpp create mode 100644 tests/queries/0_stateless/032010_printf.reference create mode 100644 tests/queries/0_stateless/032010_printf.sql diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 8793ebdd1a3..177790c983e 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -223,3 +223,28 @@ SELECT translateUTF8('Münchener Straße', 'üß', 'us') AS res; │ Munchener Strase │ └──────────────────┘ ``` + +## printf + +The `printf` function formats the given string with the values (strings, integers, floating-points etc.) listed in the arguments, similar to printf function in C++. The format string can contain format specifiers starting with `%` character. Anything not contained in `%` and the following format specifier is considered literal text and copied verbatim into the output. Literal `%` character can be escaped by `%%`. + +**Syntax** + +``` sql +printf(format, arg1, arg2, ...) +``` + +**Example** + +Query: + +``` sql +select printf('%%%s %s %d', 'Hello', 'World', 2024); +``` + + +``` response +┌─printf('%%%s %s %d', 'Hello', 'World', 2024)─┐ +│ %Hello World 2024 │ +└──────────────────────────────────────────────┘ +``` diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp new file mode 100644 index 00000000000..cb21d5e39ad --- /dev/null +++ b/src/Functions/printf.cpp @@ -0,0 +1,308 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +class FunctionPrintf : public IFunction +{ +private: + ContextPtr context; + FunctionOverloadResolverPtr function_concat; + + struct Instruction + { + std::string_view format; + size_t rows; + bool is_literal; /// format is literal string without any argument + ColumnWithTypeAndName input; /// Only used when is_literal is false + + ColumnWithTypeAndName execute() + { + if (is_literal) + return executeLiteral(format); + else if (isColumnConst(*input.column)) + return executeConstant(input); + else + return executeNonconstant(input); + } + + String toString() const + { + std::ostringstream oss; + oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() + << std::endl; + return oss.str(); + } + + private: + ColumnWithTypeAndName executeLiteral(std::string_view literal) + { + ColumnWithTypeAndName res; + auto str_col = ColumnString::create(); + str_col->insert(fmt::sprintf(literal)); + res.column = ColumnConst::create(std::move(str_col), rows); + res.type = std::make_shared(); + return res; + } + + ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) + { + ColumnWithTypeAndName tmp_arg = arg; + const auto & const_col = static_cast(*arg.column); + tmp_arg.column = const_col.getDataColumnPtr(); + + ColumnWithTypeAndName tmp_res = executeNonconstant(tmp_arg); + return ColumnWithTypeAndName{ColumnConst::create(tmp_res.column, arg.column->size()), tmp_res.type, tmp_res.name}; + } + + ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) + { + size_t size = arg.column->size(); + auto res_col = ColumnString::create(); + auto & res_str = static_cast(*res_col); + auto & res_offsets = res_str.getOffsets(); + auto & res_chars = res_str.getChars(); + res_offsets.reserve_exact(size); + res_chars.reserve(format.size() * size * 2); + + String s; + WhichDataType which(arg.type); + +#define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ + else if (which.IS_TYPE()) \ + { \ + for (size_t i = 0; i < size; ++i) \ + { \ + auto a = arg.column->GET_TYPE(i); \ + s = fmt::sprintf(format, a); \ + res_str.insertData(s.data(), s.size()); \ + } \ + } + + if (false) + ; + EXECUTE_BY_TYPE(isNativeInt, getInt) + EXECUTE_BY_TYPE(isNativeUInt, getUInt) + EXECUTE_BY_TYPE(isFloat32, getFloat32) + EXECUTE_BY_TYPE(isFloat64, getFloat64) + else if (which.isStringOrFixedString()) + { + for (size_t i = 0; i < size; ++i) + { + auto a = arg.column->getDataAt(i).toView(); + s = fmt::sprintf(format, a); + res_str.insertData(s.data(), s.size()); + } + } + else throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument type of function {} is {}, but native numeric or string type is expected", + FunctionPrintf::name, + arg.type->getName()); +#undef EXECUTE_BY_TYPE + + ColumnWithTypeAndName res; + res.name = arg.name; + res.type = std::make_shared(); + res.column = std::move(res_col); + return res; + } + }; + +public: + static constexpr auto name = "printf"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + explicit FunctionPrintf(ContextPtr context_) + : context(context_), function_concat(FunctionFactory::instance().get("concat", context)) { } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool useDefaultImplementationForConstants() const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be at least 1", + getName(), + arguments.size()); + + /// First pattern argument must have string type + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The first argument type of function {} is {}, but String type is expected", + getName(), + arguments[0]->getName()); + + for (size_t i = 1; i < arguments.size(); ++i) + { + if (!isNativeNumber(arguments[i]) && !isStringOrFixedString(arguments[i])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The {}-th argument type of function {} is {}, but native numeric or string type is expected", + i + 1, + getName(), + arguments[i]->getName()); + } + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const ColumnPtr & c0 = arguments[0].column; + const ColumnConst * c0_const_string = typeid_cast(&*c0); + if (!c0_const_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function {} must be constant string", getName()); + + String format = c0_const_string->getValue(); + auto instructions = buildInstructions(format, arguments, input_rows_count); + + ColumnsWithTypeAndName concat_args(instructions.size()); + for (size_t i = 0; i < instructions.size(); ++i) + { + std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + concat_args[i] = instructions[i].execute(); + std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + } + + auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); + return res; + } + +private: + std::vector buildInstructions(const String & format , const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + { + std::vector instructions; + instructions.reserve(arguments.size()); + + auto append_instruction = [&](const char * begin, const char * end, const ColumnWithTypeAndName & arg) + { + Instruction instr; + instr.rows = input_rows_count; + instr.format = std::string_view(begin, end - begin); + + size_t size = end - begin; + if (size > 1 && begin[0] == '%' and begin[1] != '%') + { + instr.is_literal = false; + instr.input = arg; + } + else + { + instr.is_literal = true; + } + instructions.emplace_back(std::move(instr)); + }; + + auto check_index_range = [&](size_t idx) + { + if (idx >= arguments.size()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, but format is {}", + getName(), + arguments.size(), + format); + }; + + const char * begin = format.data(); + const char * end = format.data() + format.size(); + const char * curr = begin; + size_t idx = 0; + while (curr < end) + { + const char * tmp = curr; + bool is_first = curr == begin; /// If current instruction is the first one + bool is_literal = false; /// If current instruction is literal string without any argument + if (is_first) + { + if (*curr != '%') + is_literal = true; + else if (curr + 1 < end && *(curr + 1) == '%') + is_literal = true; + else + ++idx; /// Skip first argument if first instruction is not literal + } + + if (!is_literal) + ++curr; + + while (curr < end) + { + if (*curr != '%') + ++curr; + else if (curr + 1 < end && *(curr + 1) == '%') + curr += 2; + else + { + check_index_range(idx); + append_instruction(tmp, curr, arguments[idx]); + ++idx; + break; + } + } + + if (curr == end) + { + check_index_range(idx); + append_instruction(tmp, curr, arguments[idx]); + ++idx; + } + } + + /// Check if all arguments are used + if (idx != arguments.size()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, but format is {}", + getName(), + arguments.size(), + format); + + return instructions; + } +}; + +} + +REGISTER_FUNCTION(Printf) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/032010_printf.reference b/tests/queries/0_stateless/032010_printf.reference new file mode 100644 index 00000000000..58501cbd0fc --- /dev/null +++ b/tests/queries/0_stateless/032010_printf.reference @@ -0,0 +1,16 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/032010_printf.sql b/tests/queries/0_stateless/032010_printf.sql new file mode 100644 index 00000000000..58fe081e499 --- /dev/null +++ b/tests/queries/0_stateless/032010_printf.sql @@ -0,0 +1,24 @@ +-- Testing integer formats +select printf('%%d: %d', 123) = '%d: 123'; +select printf('%%i: %i', 123) = '%i: 123'; +select printf('%%u: %u', 123) = '%u: 123'; +select printf('%%o: %o', 123) = '%o: 173'; +select printf('%%x: %x', 123) = '%x: 7b'; +select printf('%%X: %X', 123) = '%X: 7B'; + +-- Testing floating point formats +select printf('%%f: %f', 123.456) = '%f: 123.456000'; +select printf('%%F: %F', 123.456) = '%F: 123.456000'; +select printf('%%e: %e', 123.456) = '%e: 1.234560e+02'; +select printf('%%E: %E', 123.456) = '%E: 1.234560E+02'; +select printf('%%g: %g', 123.456) = '%g: 123.456'; +select printf('%%G: %G', 123.456) = '%G: 123.456'; +select printf('%%a: %a', 123.456) = '%a: 0x1.edd2f1a9fbe77p+6'; +select printf('%%A: %A', 123.456) = '%A: 0X1.EDD2F1A9FBE77P+6'; + +-- Testing character formats +select printf('%%s: %s', 'abc') = '%s: abc'; + + +-- Testing the %% specifier +select printf('%%%%: %%') = '%%: %'; \ No newline at end of file From 246f421f2402799fd11b22a608b4d0d497cb8438 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 9 Jul 2024 16:33:57 +0200 Subject: [PATCH 0120/1170] merge tree sink cancel delayed_chunk --- .../Transforms/ExceptionKeepingTransform.h | 2 +- src/Storages/MergeTree/MergeTreeSink.cpp | 9 +++++++++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 15 +++++++++++++++ src/Storages/MergeTree/ReplicatedMergeTreeSink.h | 2 ++ 4 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.h b/src/Processors/Transforms/ExceptionKeepingTransform.h index 000b5da798a..9aa33a8cbe5 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.h +++ b/src/Processors/Transforms/ExceptionKeepingTransform.h @@ -52,7 +52,7 @@ protected: virtual void onConsume(Chunk chunk) = 0; virtual GenerateResult onGenerate() = 0; virtual void onFinish() {} - virtual void onException(std::exception_ptr /* exception */) {} + virtual void onException(std::exception_ptr /* exception */) { } public: ExceptionKeepingTransform(const Block & in_header, const Block & out_header, bool ignore_on_start_and_finish_ = true); diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 05751e0fa6f..2e8f6db6868 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -56,6 +56,15 @@ void MergeTreeSink::onFinish() void MergeTreeSink::onCancel() { + if (!delayed_chunk) + return; + + for (auto & partition : delayed_chunk->partitions) + { + partition.temp_part.cancel(); + } + + delayed_chunk.reset(); } void MergeTreeSink::consume(Chunk chunk) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4b4f4c33e7d..93f82a5a789 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1155,6 +1155,21 @@ void ReplicatedMergeTreeSinkImpl::onFinish() finishDelayedChunk(std::make_shared(zookeeper)); } + +template +void ReplicatedMergeTreeSinkImpl::onCancel() +{ + if (!delayed_chunk) + return; + + for (auto & partition : delayed_chunk->partitions) + { + partition.temp_part.cancel(); + } + + delayed_chunk.reset(); +} + template void ReplicatedMergeTreeSinkImpl::waitForQuorum( const ZooKeeperWithFaultInjectionPtr & zookeeper, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 39623c20584..b1796a35ed2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -54,6 +54,8 @@ public: void consume(Chunk chunk) override; void onFinish() override; + void onCancel() override; + String getName() const override { return "ReplicatedMergeTreeSink"; } /// For ATTACHing existing data on filesystem. From e0e3842772ead940f53346cc087ea54e5e6aa8fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Jul 2024 10:15:33 +0800 Subject: [PATCH 0121/1170] support printf --- src/Functions/printf.cpp | 6 +++--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index cb21d5e39ad..b2a0143a3f8 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -52,7 +52,7 @@ private: String toString() const { - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() << std::endl; return oss.str(); @@ -195,9 +195,9 @@ public: ColumnsWithTypeAndName concat_args(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) { - std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; concat_args[i] = instructions[i].execute(); - std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; } auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 943caf918d6..21a9b759466 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2264,6 +2264,7 @@ prettyspacemonoblock prettyspacenoescapes prettyspacenoescapesmonoblock prewhere +printf privateKeyFile privateKeyPassphraseHandler prlimit From 8b1bc00e9a6462a6dce2946510d83a85bc69a139 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Jul 2024 10:52:53 +0800 Subject: [PATCH 0122/1170] fix style --- src/Functions/printf.cpp | 33 +++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index b2a0143a3f8..c7c6bd228a7 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -9,19 +9,19 @@ #include #include +#include #include #include #include #include -#include namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int ILLEGAL_COLUMN; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -54,7 +54,7 @@ private: { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() - << std::endl; + << std::endl; return oss.str(); } @@ -93,15 +93,15 @@ private: WhichDataType which(arg.type); #define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ - else if (which.IS_TYPE()) \ - { \ - for (size_t i = 0; i < size; ++i) \ - { \ - auto a = arg.column->GET_TYPE(i); \ - s = fmt::sprintf(format, a); \ - res_str.insertData(s.data(), s.size()); \ - } \ - } + else if (which.IS_TYPE()) \ + { \ + for (size_t i = 0; i < size; ++i) \ + { \ + auto a = arg.column->GET_TYPE(i); \ + s = fmt::sprintf(format, a); \ + res_str.insertData(s.data(), s.size()); \ + } \ + } if (false) ; @@ -205,7 +205,8 @@ public: } private: - std::vector buildInstructions(const String & format , const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const + std::vector + buildInstructions(const String & format, const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const { std::vector instructions; instructions.reserve(arguments.size()); @@ -248,7 +249,7 @@ private: { const char * tmp = curr; bool is_first = curr == begin; /// If current instruction is the first one - bool is_literal = false; /// If current instruction is literal string without any argument + bool is_literal = false; /// If current instruction is literal string without any argument if (is_first) { if (*curr != '%') From 6cd6319ba70945e7ae50447772c57d61e488e72e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 15:32:28 +0000 Subject: [PATCH 0123/1170] 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 0124/1170] 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 0125/1170] 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 0126/1170] 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 614e899e8d286b5b25f34f0cd94163905dceffeb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 10 Jul 2024 18:52:09 +0200 Subject: [PATCH 0127/1170] onCancel is concurrent call --- .../Executors/CompletedPipelineExecutor.cpp | 7 +++ .../Formats/Impl/ArrowBlockInputFormat.h | 2 +- .../Formats/Impl/DWARFBlockInputFormat.h | 2 +- .../Formats/Impl/NativeORCBlockInputFormat.h | 2 +- .../Formats/Impl/ORCBlockInputFormat.h | 2 +- .../Impl/ParallelFormattingOutputFormat.h | 2 +- .../Impl/ParallelParsingInputFormat.cpp | 2 +- .../Formats/Impl/ParallelParsingInputFormat.h | 2 +- .../Formats/Impl/ParquetBlockInputFormat.h | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 2 +- .../Formats/Impl/ParquetMetadataInputFormat.h | 2 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- src/Processors/IProcessor.cpp | 2 +- src/Processors/IProcessor.h | 2 +- src/Processors/Sources/RemoteSource.cpp | 2 +- src/Processors/Sources/RemoteSource.h | 2 +- .../Transforms/AggregatingTransform.cpp | 2 +- src/Server/TCPHandler.cpp | 3 ++ src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/LiveView/LiveViewEventsSource.h | 2 +- src/Storages/LiveView/LiveViewSource.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 ++ src/Storages/MergeTree/MergeTreeSink.cpp | 32 +++++++------ src/Storages/MergeTree/MergeTreeSink.h | 1 - src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/MergeTree/MergeTreeSource.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 48 ++++++++++++------- .../MergeTree/ReplicatedMergeTreeSink.h | 2 - src/Storages/MessageQueueSink.h | 6 ++- .../StorageObjectStorageSink.cpp | 28 ++++------- .../ObjectStorage/StorageObjectStorageSink.h | 8 +--- src/Storages/StorageFile.cpp | 37 ++++---------- src/Storages/StorageURL.cpp | 26 ++++------ src/Storages/StorageURL.h | 6 +-- 36 files changed, 123 insertions(+), 131 deletions(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 598a51bf0c7..909e742ffbf 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Common/Logger.h" #include #include #include @@ -97,7 +98,10 @@ void CompletedPipelineExecutor::execute() break; if (is_cancelled_callback()) + { + LOG_INFO(getLogger("CompletedPipelineExecutor"), "execute CancelCallback FULLY_CANCELLED"); data->executor->cancel(); + } } if (data->has_exception) @@ -116,7 +120,10 @@ CompletedPipelineExecutor::~CompletedPipelineExecutor() try { if (data && data->executor) + { + LOG_INFO(getLogger("CompletedPipelineExecutor"), "~CompletedPipelineExecutor"); data->executor->cancel(); + } } catch (...) { diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index cdbc5e57e4e..4fe01d0be12 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -32,7 +32,7 @@ public: private: Chunk read() override; - void onCancel() override + void onCancelX() override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h index d8f5fc3d896..6cab5d34994 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h @@ -32,7 +32,7 @@ public: protected: Chunk read() override; - void onCancel() override + void onCancelX() override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index a3ef9ed4b8f..de9925e3737 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -64,7 +64,7 @@ public: protected: Chunk read() override; - void onCancel() override { is_stopped = 1; } + void onCancelX() override { is_stopped = 1; } private: void prepareFileReader(); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 34630345849..167436ad4b9 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -34,7 +34,7 @@ public: protected: Chunk read() override; - void onCancel() override + void onCancelX() override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 341141dd633..40774fcfbfa 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -122,7 +122,7 @@ public: started_prefix = true; } - void onCancel() override + void onCancelX() override { finishAndWait(); } diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 447adb1ed48..d38a299cb6e 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -176,7 +176,7 @@ Chunk ParallelParsingInputFormat::read() if (background_exception) { lock.unlock(); - onCancel(); + onCancelX(); std::rethrow_exception(background_exception); } diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 963ccd88def..eed40dc43e5 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -137,7 +137,7 @@ private: Chunk read() final; - void onCancel() final + void onCancelX() final { /* * The format parsers themselves are not being cancelled here, so we'll diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 24735ee4371..0123329f026 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -68,7 +68,7 @@ public: private: Chunk read() override; - void onCancel() override + void onCancelX() override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index a5d334f4f1d..d08c91d286b 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -268,7 +268,7 @@ void ParquetBlockOutputFormat::resetFormatterImpl() staging_bytes = 0; } -void ParquetBlockOutputFormat::onCancel() +void ParquetBlockOutputFormat::onCancelX() { is_stopped = true; } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 422bae5c315..0704ba0ed90 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -112,7 +112,7 @@ private: void consume(Chunk) override; void finalizeImpl() override; void resetFormatterImpl() override; - void onCancel() override; + void onCancelX() override; void writeRowGroup(std::vector chunks); void writeUsingArrow(std::vector chunks); diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h index ff63d78fa44..35180d202d8 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h @@ -65,7 +65,7 @@ public: private: Chunk read() override; - void onCancel() override + void onCancelX() override { is_stopped = 1; } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 9cf609ed2d7..a245df8234d 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -29,7 +29,7 @@ public: void setRowsBeforeLimit(size_t rows_before_limit) override; - void onCancel() override + void onCancelX() override { queue.clearAndFinish(); } diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index f403aca2280..4d95bb5f3e0 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -16,7 +16,7 @@ void IProcessor::cancel() if (already_cancelled) return; - onCancel(); + onCancelX(); } String IProcessor::debug() const diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 6f779e7a8d4..9ef7d83eefa 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -363,7 +363,7 @@ public: virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} protected: - virtual void onCancel() {} + virtual void onCancelX() {} std::atomic is_cancelled{false}; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..44cf26e0b01 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -182,7 +182,7 @@ std::optional RemoteSource::tryGenerate() return chunk; } -void RemoteSource::onCancel() +void RemoteSource::onCancelX() { query_executor->cancel(); } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 052567bc261..880eb234bfb 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -36,7 +36,7 @@ public: protected: std::optional tryGenerate() override; - void onCancel() override; + void onCancelX() override; private: bool was_query_sent = false; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index cdbe194cfac..e42c1fd3a8d 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -375,7 +375,7 @@ public: return prepareTwoLevel(); } - void onCancel() override + void onCancelX() override { shared_data->is_cancelled.store(true, std::memory_order_seq_cst); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ac1423f87c1..9fffad26a72 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -556,7 +556,10 @@ void TCPHandler::runImpl() std::scoped_lock lock(out_mutex, task_callback_mutex); if (getQueryCancellationStatus() == CancellationStatus::FULLY_CANCELLED) + { + LOG_INFO(log, "CancelCallback FULLY_CANCELLED"); return true; + } sendProgress(); sendSelectProfileEvents(); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index e556bda2561..6283594e0d2 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -596,7 +596,7 @@ void DistributedSink::onFinish() } } -void DistributedSink::onCancel() +void DistributedSink::onCancelX() { std::lock_guard lock(execution_mutex); if (pool && !pool->finished()) diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index a4c95633595..1dac3eeba6d 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -53,7 +53,7 @@ public: void onFinish() override; private: - void onCancel() override; + void onCancelX() override; IColumn::Selector createSelector(const Block & source_block) const; diff --git a/src/Storages/LiveView/LiveViewEventsSource.h b/src/Storages/LiveView/LiveViewEventsSource.h index de10a98e1a2..d1ed222c185 100644 --- a/src/Storages/LiveView/LiveViewEventsSource.h +++ b/src/Storages/LiveView/LiveViewEventsSource.h @@ -54,7 +54,7 @@ public: String getName() const override { return "LiveViewEventsSource"; } - void onCancel() override + void onCancelX() override { if (storage->shutdown_called) return; diff --git a/src/Storages/LiveView/LiveViewSource.h b/src/Storages/LiveView/LiveViewSource.h index f8b428fc04d..83589067cf5 100644 --- a/src/Storages/LiveView/LiveViewSource.h +++ b/src/Storages/LiveView/LiveViewSource.h @@ -36,7 +36,7 @@ public: String getName() const override { return "LiveViewSource"; } - void onCancel() override + void onCancelX() override { if (storage->shutdown_called) return; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 5c8aa32949d..0aaa7909a0f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,6 +13,8 @@ #include #include #include +#include "Common/Logger.h" +#include "Common/logger_useful.h" #include #include #include @@ -182,6 +184,8 @@ void updateTTL( void MergeTreeDataWriter::TemporaryPart::cancel() { + LOG_INFO(getLogger("MergeTreeDataWriter"), "TemporaryPart cancel"); + try { /// An exception context is needed to proper delete write buffers without finalization diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 2e8f6db6868..d2e34665962 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -26,7 +26,23 @@ struct MergeTreeSink::DelayedChunk }; -MergeTreeSink::~MergeTreeSink() = default; +MergeTreeSink::~MergeTreeSink() +{ + size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; + LOG_INFO(storage.log, "~ReplicatedMergeTreeSinkImpl, delayed_chunk {}, called from {}", addr, StackTrace().toString()); + + if (!delayed_chunk) + return; + + for (auto & partition : delayed_chunk->partitions) + { + partition.temp_part.cancel(); + } + + delayed_chunk.reset(); + + LOG_INFO(storage.log, "~ReplicatedMergeTreeSinkImpl end"); +} MergeTreeSink::MergeTreeSink( StorageMergeTree & storage_, @@ -51,22 +67,10 @@ void MergeTreeSink::onStart() void MergeTreeSink::onFinish() { + chassert(!isCancelled()); finishDelayedChunk(); } -void MergeTreeSink::onCancel() -{ - if (!delayed_chunk) - return; - - for (auto & partition : delayed_chunk->partitions) - { - partition.temp_part.cancel(); - } - - delayed_chunk.reset(); -} - void MergeTreeSink::consume(Chunk chunk) { if (num_blocks_processed > 0) diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index cf6715a3415..07ab3850df2 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -28,7 +28,6 @@ public: void consume(Chunk chunk) override; void onStart() override; void onFinish() override; - void onCancel() override; private: StorageMergeTree & storage; diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index e323b9f9ee7..4070ccf4433 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -149,7 +149,7 @@ std::string MergeTreeSource::getName() const return processor->getName(); } -void MergeTreeSource::onCancel() +void MergeTreeSource::onCancelX() { processor->cancel(); } diff --git a/src/Storages/MergeTree/MergeTreeSource.h b/src/Storages/MergeTree/MergeTreeSource.h index fc39b4f9b09..c7092aa26b1 100644 --- a/src/Storages/MergeTree/MergeTreeSource.h +++ b/src/Storages/MergeTree/MergeTreeSource.h @@ -26,7 +26,7 @@ public: protected: std::optional tryGenerate() override; - void onCancel() override; + void onCancelX() override; private: MergeTreeSelectProcessorPtr processor; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 93f82a5a789..6c7ed9bdae0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -4,11 +4,13 @@ #include #include #include "Common/Exception.h" +#include "Common/StackTrace.h" #include #include #include #include #include +#include "base/defines.h" #include #include #include @@ -151,7 +153,23 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( } template -ReplicatedMergeTreeSinkImpl::~ReplicatedMergeTreeSinkImpl() = default; +ReplicatedMergeTreeSinkImpl::~ReplicatedMergeTreeSinkImpl() +{ + size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; + LOG_INFO(log, "~ReplicatedMergeTreeSinkImpl, delayed_chunk {}, called from {}", addr, StackTrace().toString()); + + if (!delayed_chunk) + return; + + for (auto & partition : delayed_chunk->partitions) + { + partition.temp_part.cancel(); + } + + delayed_chunk.reset(); + + LOG_INFO(log, "~ReplicatedMergeTreeSinkImpl end"); +} template size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) @@ -255,6 +273,8 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const template void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { + LOG_INFO(log, "consume"); + if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); @@ -428,6 +448,9 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { + size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; + LOG_INFO(log, "finishDelayedChunk {}", addr); + if (!delayed_chunk) return; @@ -457,16 +480,22 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF { auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus::fromCurrentException("", true)); + + size_t addr1 = delayed_chunk ? size_t(delayed_chunk.get()) : 0; + LOG_INFO(log, "finishDelayedChunk exception, delayed_chunk {}", addr1); throw; } } delayed_chunk.reset(); + + LOG_INFO(log, "finishDelayedChunk end, delayed_chunk {}", bool(delayed_chunk)); } template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { + if (!delayed_chunk) return; @@ -1151,25 +1180,12 @@ void ReplicatedMergeTreeSinkImpl::onStart() template void ReplicatedMergeTreeSinkImpl::onFinish() { + chassert(!isCancelled()); + auto zookeeper = storage.getZooKeeper(); finishDelayedChunk(std::make_shared(zookeeper)); } - -template -void ReplicatedMergeTreeSinkImpl::onCancel() -{ - if (!delayed_chunk) - return; - - for (auto & partition : delayed_chunk->partitions) - { - partition.temp_part.cancel(); - } - - delayed_chunk.reset(); -} - template void ReplicatedMergeTreeSinkImpl::waitForQuorum( const ZooKeeperWithFaultInjectionPtr & zookeeper, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index b1796a35ed2..39623c20584 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -54,8 +54,6 @@ public: void consume(Chunk chunk) override; void onFinish() override; - void onCancel() override; - String getName() const override { return "ReplicatedMergeTreeSink"; } /// For ATTACHing existing data on filesystem. diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index b3c1e61734f..38754e9475e 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -33,13 +33,17 @@ public: const String & storage_name_, const ContextPtr & context_); + ~MessageQueueSink() override + { + onFinish(); + } + String getName() const override { return storage_name + "Sink"; } void consume(Chunk chunk) override; void onStart() override; void onFinish() override; - void onCancel() override { onFinish(); } void onException(std::exception_ptr /* exception */) override { onFinish(); } protected: diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index f2f6eac333c..3bd0e88ecdb 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -2,6 +2,7 @@ #include #include #include +#include "base/defines.h" #include namespace DB @@ -41,31 +42,16 @@ StorageObjectStorageSink::StorageObjectStorageSink( void StorageObjectStorageSink::consume(Chunk chunk) { - std::lock_guard lock(cancel_mutex); - if (cancelled) + if (isCancelled()) return; writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } -void StorageObjectStorageSink::onCancel() -{ - std::lock_guard lock(cancel_mutex); - cancelBuffers(); - releaseBuffers(); - cancelled = true; -} - -void StorageObjectStorageSink::onException(std::exception_ptr) -{ - std::lock_guard lock(cancel_mutex); - cancelBuffers(); - releaseBuffers(); -} - void StorageObjectStorageSink::onFinish() { - std::lock_guard lock(cancel_mutex); + chassert(!isCancelled()); finalizeBuffers(); + releaseBuffers(); } void StorageObjectStorageSink::finalizeBuffers() @@ -119,6 +105,12 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( { } +StorageObjectStorageSink::~StorageObjectStorageSink() +{ + if (isCancelled()) + cancelBuffers(); +} + SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) { auto partition_bucket = replaceWildcards(configuration->getNamespace(), partition_id); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index e0081193686..578290a92a5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -18,22 +18,18 @@ public: ContextPtr context, const std::string & blob_path = ""); + ~StorageObjectStorageSink() override; + String getName() const override { return "StorageObjectStorageSink"; } void consume(Chunk chunk) override; - void onCancel() override; - - void onException(std::exception_ptr exception) override; - void onFinish() override; private: const Block sample_block; std::unique_ptr write_buf; OutputFormatPtr writer; - bool cancelled = false; - std::mutex cancel_mutex; void finalizeBuffers(); void releaseBuffers(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..855667b1cc6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include "base/defines.h" #include #include @@ -1747,6 +1748,12 @@ public: initialize(); } + ~StorageFileSink() override + { + if (isCancelled()) + cancelBuffers(); + } + void initialize() { std::unique_ptr naked_buffer; @@ -1780,37 +1787,14 @@ public: void consume(Chunk chunk) override { - std::lock_guard cancel_lock(cancel_mutex); - if (cancelled) + if (isCancelled()) return; writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } - void onCancel() override - { - std::lock_guard cancel_lock(cancel_mutex); - cancelBuffers(); - releaseBuffers(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard cancel_lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - releaseBuffers(); - } - } - void onFinish() override { - std::lock_guard cancel_lock(cancel_mutex); + chassert(!isCancelled()); finalizeBuffers(); } @@ -1865,9 +1849,6 @@ private: int flags; std::unique_lock lock; - - std::mutex cancel_mutex; - bool cancelled = false; }; class PartitionedStorageFileSink : public PartitionedSink diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 895da028fc2..6f600393263 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -567,31 +567,15 @@ StorageURLSink::StorageURLSink( void StorageURLSink::consume(Chunk chunk) { - std::lock_guard lock(cancel_mutex); - if (cancelled) + if (isCancelled()) return; writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } -void StorageURLSink::onCancel() -{ - std::lock_guard lock(cancel_mutex); - cancelBuffers(); - releaseBuffers(); - cancelled = true; -} - -void StorageURLSink::onException(std::exception_ptr) -{ - std::lock_guard lock(cancel_mutex); - cancelBuffers(); - releaseBuffers(); -} - void StorageURLSink::onFinish() { - std::lock_guard lock(cancel_mutex); finalizeBuffers(); + releaseBuffers(); } void StorageURLSink::finalizeBuffers() @@ -1396,6 +1380,11 @@ StorageURLWithFailover::StorageURLWithFailover( } } +StorageURLSink::~StorageURLSink() +{ + if (isCancelled()) + cancelBuffers(); +} FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Arguments & args) { @@ -1586,4 +1575,5 @@ void registerStorageURL(StorageFactory & factory) .source_access_type = AccessType::URL, }); } + } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fa7cc6eeeef..12a49d3dff5 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -250,10 +250,10 @@ public: const HTTPHeaderEntries & headers = {}, const String & method = Poco::Net::HTTPRequest::HTTP_POST); + ~StorageURLSink() override; + std::string getName() const override { return "StorageURLSink"; } void consume(Chunk chunk) override; - void onCancel() override; - void onException(std::exception_ptr exception) override; void onFinish() override; private: @@ -263,8 +263,6 @@ private: std::unique_ptr write_buf; OutputFormatPtr writer; - std::mutex cancel_mutex; - bool cancelled = false; }; class StorageURL : public IStorageURLBase From 96d063bcc39712c5a21a8e51244a9e216af8536a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 12:10:44 +0800 Subject: [PATCH 0128/1170] renmae ut files --- src/Functions/printf.cpp | 118 +++++++++++------- ...erence => 03203_function_printf.reference} | 0 ...0_printf.sql => 03203_function_printf.sql} | 0 3 files changed, 73 insertions(+), 45 deletions(-) rename tests/queries/0_stateless/{032010_printf.reference => 03203_function_printf.reference} (100%) rename tests/queries/0_stateless/{032010_printf.sql => 03203_function_printf.sql} (100%) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index c7c6bd228a7..247c4a65daf 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -40,7 +40,7 @@ private: bool is_literal; /// format is literal string without any argument ColumnWithTypeAndName input; /// Only used when is_literal is false - ColumnWithTypeAndName execute() + ColumnWithTypeAndName execute() const { if (is_literal) return executeLiteral(format); @@ -50,7 +50,7 @@ private: return executeNonconstant(input); } - String toString() const + [[maybe_unused]] String toString() const { std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() @@ -59,7 +59,7 @@ private: } private: - ColumnWithTypeAndName executeLiteral(std::string_view literal) + ColumnWithTypeAndName executeLiteral(std::string_view literal) const { ColumnWithTypeAndName res; auto str_col = ColumnString::create(); @@ -69,7 +69,7 @@ private: return res; } - ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) + ColumnWithTypeAndName executeConstant(const ColumnWithTypeAndName & arg) const { ColumnWithTypeAndName tmp_arg = arg; const auto & const_col = static_cast(*arg.column); @@ -79,57 +79,85 @@ private: return ColumnWithTypeAndName{ColumnConst::create(tmp_res.column, arg.column->size()), tmp_res.type, tmp_res.name}; } - ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) + template + bool executeNumber(const IColumn & column, ColumnString::Chars & res_chars, ColumnString::Offsets & res_offsets) const + { + const ColumnVector * concrete_column = checkAndGetColumn>(&column); + if (!concrete_column) + return false; + + String s; + size_t curr_offset = 0; + const auto & data = concrete_column->getData(); + for (size_t i = 0; i < data.size(); ++i) + { + T a = data[i]; + s = fmt::sprintf(format, static_cast>(a)); + memcpy(&res_chars[curr_offset], s.data(), s.size()); + res_chars[curr_offset + s.size()] = 0; + + curr_offset += s.size() + 1; + res_offsets[i] = curr_offset; + } + return true; + } + + template + bool executeString(const IColumn & column, ColumnString::Chars & res_chars, ColumnString::Offsets & res_offsets) const + { + const COLUMN * concrete_column = checkAndGetColumn(&column); + if (!concrete_column) + return false; + + String s; + size_t curr_offset = 0; + for (size_t i = 0; i < concrete_column->size(); ++i) + { + auto a = concrete_column->getDataAt(i).toView(); + s = fmt::sprintf(format, a); + memcpy(&res_chars[curr_offset], s.data(), s.size()); + res_chars[curr_offset + s.size()] = 0; + + curr_offset += s.size() + 1; + res_offsets[i] = curr_offset; + } + return true; + } + + ColumnWithTypeAndName executeNonconstant(const ColumnWithTypeAndName & arg) const { size_t size = arg.column->size(); auto res_col = ColumnString::create(); auto & res_str = static_cast(*res_col); auto & res_offsets = res_str.getOffsets(); auto & res_chars = res_str.getChars(); - res_offsets.reserve_exact(size); - res_chars.reserve(format.size() * size * 2); + res_offsets.resize_exact(size); + res_chars.reserve(format.size() * size); - String s; WhichDataType which(arg.type); - -#define EXECUTE_BY_TYPE(IS_TYPE, GET_TYPE) \ - else if (which.IS_TYPE()) \ - { \ - for (size_t i = 0; i < size; ++i) \ - { \ - auto a = arg.column->GET_TYPE(i); \ - s = fmt::sprintf(format, a); \ - res_str.insertData(s.data(), s.size()); \ - } \ - } - - if (false) - ; - EXECUTE_BY_TYPE(isNativeInt, getInt) - EXECUTE_BY_TYPE(isNativeUInt, getUInt) - EXECUTE_BY_TYPE(isFloat32, getFloat32) - EXECUTE_BY_TYPE(isFloat64, getFloat64) - else if (which.isStringOrFixedString()) + if (which.isNativeNumber() + && (executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets))) { - for (size_t i = 0; i < size; ++i) - { - auto a = arg.column->getDataAt(i).toView(); - s = fmt::sprintf(format, a); - res_str.insertData(s.data(), s.size()); - } + return {std::move(res_col), std::make_shared(), arg.name}; } - else throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The argument type of function {} is {}, but native numeric or string type is expected", - FunctionPrintf::name, - arg.type->getName()); -#undef EXECUTE_BY_TYPE - - ColumnWithTypeAndName res; - res.name = arg.name; - res.type = std::make_shared(); - res.column = std::move(res_col); - return res; + else if ( + which.isStringOrFixedString() + && (executeString(*arg.column, res_chars, res_offsets) + || executeString(*arg.column, res_chars, res_offsets))) + { + return {std::move(res_col), std::make_shared(), arg.name}; + } + else + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument type of function {} is {}, but native numeric or string type is expected", + FunctionPrintf::name, + arg.type->getName()); } }; diff --git a/tests/queries/0_stateless/032010_printf.reference b/tests/queries/0_stateless/03203_function_printf.reference similarity index 100% rename from tests/queries/0_stateless/032010_printf.reference rename to tests/queries/0_stateless/03203_function_printf.reference diff --git a/tests/queries/0_stateless/032010_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql similarity index 100% rename from tests/queries/0_stateless/032010_printf.sql rename to tests/queries/0_stateless/03203_function_printf.sql From 88851ddb569f9ae8c61420bde99d2ad5f3d76889 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 12:15:44 +0800 Subject: [PATCH 0129/1170] improve uts --- .../0_stateless/03203_function_printf.reference | 5 +++++ .../queries/0_stateless/03203_function_printf.sql | 14 ++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_function_printf.reference b/tests/queries/0_stateless/03203_function_printf.reference index 58501cbd0fc..338ecb0183d 100644 --- a/tests/queries/0_stateless/03203_function_printf.reference +++ b/tests/queries/0_stateless/03203_function_printf.reference @@ -14,3 +14,8 @@ 1 1 1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03203_function_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql index 58fe081e499..c41cbf0b5e9 100644 --- a/tests/queries/0_stateless/03203_function_printf.sql +++ b/tests/queries/0_stateless/03203_function_printf.sql @@ -19,6 +19,16 @@ select printf('%%A: %A', 123.456) = '%A: 0X1.EDD2F1A9FBE77P+6'; -- Testing character formats select printf('%%s: %s', 'abc') = '%s: abc'; - -- Testing the %% specifier -select printf('%%%%: %%') = '%%: %'; \ No newline at end of file +select printf('%%%%: %%') = '%%: %'; + +-- Testing integer formats with precision +select printf('%%.5d: %.5d', 123) = '%.5d: 00123'; + +-- Testing floating point formats with precision +select printf('%%.2f: %.2f', 123.456) = '%.2f: 123.46'; +select printf('%%.2e: %.2e', 123.456) = '%.2e: 1.23e+02'; +select printf('%%.2g: %.2g', 123.456) = '%.2g: 1.2e+02'; + +-- Testing character formats with precision +select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; \ No newline at end of file From d988399aa68ad8a6bc412bfc48d9fdefe63c1657 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 14:32:24 +0800 Subject: [PATCH 0130/1170] fix failed uts --- src/Functions/printf.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index 247c4a65daf..a890b886338 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -141,7 +141,9 @@ private: || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) || executeNumber(*arg.column, res_chars, res_offsets) - || executeNumber(*arg.column, res_chars, res_offsets))) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets) + || executeNumber(*arg.column, res_chars, res_offsets))) { return {std::move(res_col), std::make_shared(), arg.name}; } From 21ca5f2d65c936a2c5b5fbc8f3f0c40d0ce60a6a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Jul 2024 15:32:05 +0800 Subject: [PATCH 0131/1170] fix failed ut --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index a152066a460..873b6bbb660 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -562,6 +562,7 @@ positionCaseInsensitive positionCaseInsensitiveUTF8 positionUTF8 pow +printf proportionsZTest protocol queryID From 243edcc8aa622d47a90cb4fba33f5079269e39b8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 12 Jul 2024 01:01:52 +0200 Subject: [PATCH 0132/1170] 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 34e54fd089b5c4b4892e80bd426a9409b31f29c9 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 12 Jul 2024 13:58:41 +0200 Subject: [PATCH 0133/1170] fix test test_upload_s3_fail_upload_part_when_multi_part_upload --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 1ed70e20b79..22a5cd525f3 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -197,7 +197,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( ) assert create_multipart == 1 assert upload_parts >= 2 - assert s3_errors >= 2 + assert s3_errors == 1 @pytest.mark.parametrize( From c6b558c7915b070167649d4e88eafb2613570bd3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 4 Jul 2024 22:30:18 +0200 Subject: [PATCH 0134/1170] Done --- .../02814_currentDatabase_for_table_functions.sql | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql index 74b5cf5f432..8b1e3ba1e10 100644 --- a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql @@ -13,7 +13,13 @@ CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT J CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); INSERT INTO null_table_buffer VALUES (1); -SELECT sleep(3) FORMAT Null; + +-- OPTIMIZE query should flush Buffer table, but still it is not guaranteed +-- (see the comment StorageBuffer::optimize) +-- But the combination of OPTIMIZE + sleep + OPTIMIZE should be enough. +OPTIMIZE TABLE null_table_buffer; +SELECT sleep(1) FORMAT Null; +OPTIMIZE TABLE null_table_buffer; -- Insert about should've landed into `null_mv` SELECT count() FROM null_mv; From ca7e003c6d7af6bf0676bba7cb61ab560c202bf3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Jul 2024 15:27:03 +0000 Subject: [PATCH 0135/1170] Fixed test --- .../02814_currentDatabase_for_table_functions.reference | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference index 7ff95106d3d..20b14d9a67b 100644 --- a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference @@ -11,7 +11,12 @@ CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); INSERT INTO null_table_buffer VALUES (1); -SELECT sleep(3) FORMAT Null; +-- OPTIMIZE query should flush Buffer table, but still it is not guaranteed +-- (see the comment StorageBuffer::optimize) +-- But the combination of OPTIMIZE + sleep + OPTIMIZE should be enough. +OPTIMIZE TABLE null_table_buffer; +SELECT sleep(1) FORMAT Null; +OPTIMIZE TABLE null_table_buffer; -- Insert about should've landed into `null_mv` SELECT count() FROM null_mv; 1 From a6e737ef2afc7fb18d661295e6f84cc3e0478ae1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 17:19:30 +0000 Subject: [PATCH 0136/1170] Cleaner FilterDAGInfo. --- src/Interpreters/ActionsDAG.cpp | 25 +++++++------- src/Interpreters/ActionsDAG.h | 2 +- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 18 +++++------ src/Planner/Planner.cpp | 4 +-- src/Planner/PlannerJoinTree.cpp | 36 +++++++++------------ src/Storages/IStorage.cpp | 6 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/StorageMerge.cpp | 6 ++-- 11 files changed, 49 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 04be9d23c32..4401c83549f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1249,31 +1249,30 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) { std::unordered_map old_to_new_nodes; - return ActionsDAG::clone(from, old_to_new_nodes); + if (from == nullptr) + return nullptr; + return std::make_unique(ActionsDAG::clone(*from, old_to_new_nodes)); } -ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes) +ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes) { - if (!from) - return nullptr; + ActionsDAG actions; - auto actions = std::make_unique(); - - for (const auto & node : from->nodes) + for (const auto & node : from.nodes) { - auto & copy_node = actions->nodes.emplace_back(node); + auto & copy_node = actions.nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; } - for (auto & node : actions->nodes) + for (auto & node : actions.nodes) for (auto & child : node.children) child = old_to_new_nodes[child]; - for (const auto & output_node : from->outputs) - actions->outputs.push_back(old_to_new_nodes[output_node]); + for (const auto & output_node : from.outputs) + actions.outputs.push_back(old_to_new_nodes[output_node]); - for (const auto & input_node : from->inputs) - actions->inputs.push_back(old_to_new_nodes[input_node]); + for (const auto & input_node : from.inputs) + actions.inputs.push_back(old_to_new_nodes[input_node]); return actions; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index cf6a91b9fe7..f428ca2f01c 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -263,7 +263,7 @@ public: static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } static ActionsDAGPtr clone(const ActionsDAG * from); - static ActionsDAGPtr clone(const ActionsDAG * from, std::unordered_map & old_to_new_nodes); + static ActionsDAG clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes); static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 1c6c3f2556b..dd1d2eb703e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -75,7 +75,7 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = std::move(*ActionsDAG::clone(&actions_dag, copy_map)); + copy->actions_dag = ActionsDAG::clone(actions_dag, copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 068b6f290fa..286eda14b3f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1922,7 +1922,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && additional_filter) { - Names columns_for_additional_filter = additional_filter->actions->getRequiredColumnsNames(); + Names columns_for_additional_filter = additional_filter->actions.getRequiredColumnsNames(); additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), columns_for_additional_filter.begin(), columns_for_additional_filter.end()); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cde6e305005..e723e5f7982 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -178,12 +178,12 @@ FilterDAGInfoPtr generateFilterActions( filter_info->actions = std::move(analyzer.simpleSelectActions()->dag); filter_info->column_name = expr_list->children.at(0)->getColumnName(); - filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); + filter_info->actions.removeUnusedActions(NameSet{filter_info->column_name}); - for (const auto * node : filter_info->actions->getInputs()) - filter_info->actions->getOutputs().push_back(node); + for (const auto * node : filter_info->actions.getInputs()) + filter_info->actions.getOutputs().push_back(node); - auto required_columns_from_filter = filter_info->actions->getRequiredColumns(); + auto required_columns_from_filter = filter_info->actions.getRequiredColumns(); for (const auto & column : required_columns_from_filter) { @@ -1486,7 +1486,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), + std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1612,7 +1612,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)), + std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1620,11 +1620,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*new_filter_info->actions)), + std::move(new_filter_info->actions), new_filter_info->column_name, new_filter_info->do_remove_column); @@ -2107,7 +2107,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis else { /// Add row level security actions to prewhere. - analysis.prewhere_info->row_level_filter = std::move(*analysis.filter_info->actions); + analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.filter_info = nullptr; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 0b10cef82ce..ffed19185d3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1128,11 +1128,11 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto fake_table_expression = std::make_shared(std::move(storage), query_context); auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); - if (!filter_info.actions || !query_plan.isInitialized()) + if (!query_plan.isInitialized()) return; auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - std::move(*filter_info.actions), + std::move(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription("additional result filter"); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index fa3a3483a8e..3217d3461d3 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -458,7 +458,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end()); } -FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, +std::optional buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context, std::set & used_row_policies) @@ -479,7 +479,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); } -FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, +std::optional buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) { @@ -513,7 +513,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, } /// Apply filters from additional_table_filters setting -FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, +std::optional buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) @@ -789,9 +789,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres std::vector> where_filters; const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description) { - if (!filter_info.actions) - return; - bool is_final = table_expression_query_info.table_expression_modifiers && table_expression_query_info.table_expression_modifiers->hasFinal(); bool optimize_move_to_prewhere @@ -805,14 +802,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!prewhere_info->prewhere_actions) { - prewhere_info->prewhere_actions = std::move(*filter_info.actions); + prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->need_filter = true; } else if (!prewhere_info->row_level_filter) { - prewhere_info->row_level_filter = std::move(*filter_info.actions); + prewhere_info->row_level_filter = std::move(filter_info.actions); prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->need_filter = true; } @@ -830,17 +827,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); - if (row_policy_filter_info.actions) - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&*row_policy_filter_info.actions)); - add_filter(row_policy_filter_info, "Row-level security filter"); + if (row_policy_filter_info) + { + table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&row_policy_filter_info->actions)); + add_filter(*row_policy_filter_info, "Row-level security filter"); + } if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info - = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); - add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); + if (auto parallel_replicas_custom_key_filter_info= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context)) + add_filter(*parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else if (auto * distributed = typeid_cast(storage.get()); distributed && query_context->canUseParallelReplicasCustomKey(*distributed->getCluster())) @@ -850,9 +848,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getOriginalAlias(); - auto additional_filters_info - = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); - add_filter(additional_filters_info, "additional filter"); + if (auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context)) + add_filter(*additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage( query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); @@ -967,11 +964,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres for (auto && [filter_info, description] : where_filters) { if (query_plan.isInitialized() && - from_stage == QueryProcessingStage::FetchColumns && - filter_info.actions) + from_stage == QueryProcessingStage::FetchColumns) { auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), - std::move(*filter_info.actions), + std::move(filter_info.actions), filter_info.column_name, filter_info.do_remove_column); filter_step->setStepDescription(description); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 57f79a2cd7f..4164608b4b5 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -340,10 +340,8 @@ std::string FilterDAGInfo::dump() const WriteBufferFromOwnString ss; ss << "FilterDAGInfo for column '" << column_name <<"', do_remove_column " << do_remove_column << "\n"; - if (actions) - { - ss << "actions " << actions->dumpDAG() << "\n"; - } + + ss << "actions " << actions.dumpDAG() << "\n"; return ss.str(); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 334c8c9c5ac..88fb52a94f2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7058,7 +7058,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS ActionDAGNodes filter_nodes; if (auto additional_filter_info = select.getAdditionalQueryInfo()) - filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); + filter_nodes.nodes.push_back(&additional_filter_info->actions.findInOutputs(additional_filter_info->column_name)); if (before_where) filter_nodes.nodes.push_back(&before_where->dag.findInOutputs(where_column_name)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 5276870c037..97b36115dfd 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -93,7 +93,7 @@ struct FilterInfo /// Same as FilterInfo, but with ActionsDAG. struct FilterDAGInfo { - std::optional actions; + ActionsDAG actions; String column_name; bool do_remove_column = false; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 374abd0b0a5..18e194491b8 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -407,7 +407,7 @@ void ReadFromMerge::addFilter(FilterDAGInfo filter) { output_stream->header = FilterTransform::transformHeader( output_stream->header, - filter.actions ? &*filter.actions : nullptr, + &filter.actions, filter.column_name, filter.do_remove_column); pushed_down_filters.push_back(std::move(filter)); @@ -662,7 +662,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*filter_info.actions)), + std::move(*ActionsDAG::clone(&filter_info.actions)), filter_info.column_name, filter_info.do_remove_column); @@ -1565,7 +1565,7 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { for (const auto & filter_info : pushed_down_filters) - added_filter_nodes.nodes.push_back(&filter_info.actions->findInOutputs(filter_info.column_name)); + added_filter_nodes.nodes.push_back(&filter_info.actions.findInOutputs(filter_info.column_name)); SourceStepWithFilter::applyFilters(added_filter_nodes); From fb7cf4ab93c991b3e2cd8a3e3e1c6cecf574b936 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 17:46:03 +0000 Subject: [PATCH 0137/1170] Better. --- src/Interpreters/ActionsDAG.cpp | 16 +++++++++++----- src/Interpreters/ActionsDAG.h | 4 +++- src/Interpreters/ExpressionActions.cpp | 2 +- src/Planner/CollectTableExpressionData.cpp | 10 +++++----- src/Planner/Planner.cpp | 4 ++-- src/Planner/PlannerJoinTree.cpp | 7 ++++--- src/Planner/TableExpressionData.h | 18 +++++++++--------- 7 files changed, 35 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4401c83549f..4f03a9e1602 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1251,14 +1251,20 @@ ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) std::unordered_map old_to_new_nodes; if (from == nullptr) return nullptr; - return std::make_unique(ActionsDAG::clone(*from, old_to_new_nodes)); + return std::make_unique(from->clone(old_to_new_nodes)); } -ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes) +ActionsDAG ActionsDAG::clone() const +{ + std::unordered_map old_to_new_nodes; + return clone(old_to_new_nodes); +} + +ActionsDAG ActionsDAG::clone(std::unordered_map & old_to_new_nodes) const { ActionsDAG actions; - for (const auto & node : from.nodes) + for (const auto & node : nodes) { auto & copy_node = actions.nodes.emplace_back(node); old_to_new_nodes[&node] = ©_node; @@ -1268,10 +1274,10 @@ ActionsDAG ActionsDAG::clone(const ActionsDAG & from, std::unordered_map & old_to_new_nodes); + + ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; + ActionsDAG clone() const; static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index dd1d2eb703e..113410b1480 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -75,7 +75,7 @@ ExpressionActionsPtr ExpressionActions::clone() const auto copy = std::make_shared(ExpressionActions()); std::unordered_map copy_map; - copy->actions_dag = ActionsDAG::clone(actions_dag, copy_map); + copy->actions_dag = actions_dag.clone(copy_map); copy->actions = actions; for (auto & action : copy->actions) action.node = copy_map[action.node]; diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 162d3fe8d11..1d85476636c 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -335,22 +335,22 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr collect_source_columns_visitor.setKeepAliasColumns(false); collect_source_columns_visitor.visit(query_node_typed.getPrewhere()); - auto prewhere_actions_dag = std::make_unique(); + ActionsDAG prewhere_actions_dag; QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node); + auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "Invalid PREWHERE. Expected single boolean expression. In query {}", query_node->formatASTForErrorMessage()); - prewhere_actions_dag->getOutputs().push_back(expression_nodes.back()); + prewhere_actions_dag.getOutputs().push_back(expression_nodes.back()); - for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs()) + for (const auto & prewhere_input_node : prewhere_actions_dag.getInputs()) if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name)) - prewhere_actions_dag->getOutputs().push_back(prewhere_input_node); + prewhere_actions_dag.getOutputs().push_back(prewhere_input_node); table_expression_data.setPrewhereFilterActions(std::move(prewhere_actions_dag)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ffed19185d3..9042303d0e4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1412,7 +1412,7 @@ void Planner::buildPlanForQueryNode() checkStoragesSupportTransactions(planner_context); const auto & table_filters = planner_context->getGlobalPlannerContext()->filters_for_table_expressions; - if (!select_query_options.only_analyze && !table_filters.empty()) // && top_level) + if (!select_query_options.only_analyze && !table_filters.empty()) { for (auto & [table_node, table_expression_data] : planner_context->getTableExpressionNodeToData()) { @@ -1420,7 +1420,7 @@ void Planner::buildPlanForQueryNode() if (it != table_filters.end()) { const auto & filters = it->second; - table_expression_data.setFilterActions(ActionsDAG::clone(&*filters.filter_actions)); + table_expression_data.setFilterActions(filters.filter_actions->clone()); table_expression_data.setPrewhereInfo(filters.prewhere_info); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 3217d3461d3..d55e5e99f71 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -646,7 +646,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; - table_expression_query_info.filter_actions_dag = ActionsDAG::clone(table_expression_data.getFilterActions()); + if (const auto & filter_actions = table_expression_data.getFilterActions()) + table_expression_query_info.filter_actions_dag = std::make_shared(filter_actions->clone()); table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; @@ -776,7 +777,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (prewhere_actions) { prewhere_info = std::make_shared(); - prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(prewhere_actions)); + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->remove_prewhere_column = true; prewhere_info->need_filter = true; @@ -829,7 +830,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); if (row_policy_filter_info) { - table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&row_policy_filter_info->actions)); + table_expression_data.setRowLevelFilterActions(row_policy_filter_info->actions.clone()); add_filter(*row_policy_filter_info, "Row-level security filter"); } diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 9723a00a356..1d04fac3dc3 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -211,32 +211,32 @@ public: is_merge_tree = is_merge_tree_value; } - const ActionsDAGPtr & getPrewhereFilterActions() const + const std::optional & getPrewhereFilterActions() const { return prewhere_filter_actions; } - void setRowLevelFilterActions(ActionsDAGPtr row_level_filter_actions_value) + void setRowLevelFilterActions(ActionsDAG row_level_filter_actions_value) { row_level_filter_actions = std::move(row_level_filter_actions_value); } - const ActionsDAGPtr & getRowLevelFilterActions() const + const std::optional & getRowLevelFilterActions() const { return row_level_filter_actions; } - void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value) + void setPrewhereFilterActions(ActionsDAG prewhere_filter_actions_value) { prewhere_filter_actions = std::move(prewhere_filter_actions_value); } - const ActionsDAGPtr & getFilterActions() const + const std::optional & getFilterActions() const { return filter_actions; } - void setFilterActions(ActionsDAGPtr filter_actions_value) + void setFilterActions(ActionsDAG filter_actions_value) { filter_actions = std::move(filter_actions_value); } @@ -289,16 +289,16 @@ private: ColumnIdentifierToColumnName column_identifier_to_column_name; /// Valid for table, table function - ActionsDAGPtr filter_actions; + std::optional filter_actions; /// Valid for table, table function PrewhereInfoPtr prewhere_info; /// Valid for table, table function - ActionsDAGPtr prewhere_filter_actions; + std::optional prewhere_filter_actions; /// Valid for table, table function - ActionsDAGPtr row_level_filter_actions; + std::optional row_level_filter_actions; /// Is storage remote bool is_remote = false; From 264d7d760fedd3fc3c900d13ee1f7976887efaa7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 15:52:23 +0000 Subject: [PATCH 0138/1170] Bump rocksdb to 7.0.4 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 9 +++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index be366233921..4fc59e24001 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit be366233921293bd07a84dc4ea6991858665f202 +Subproject commit 4fc59e240016a62180b09703e2938c3d7e928de0 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 3a14407166c..f6479346063 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -59,10 +59,8 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") endif() -set (HAVE_THREAD_LOCAL 1) -if(HAVE_THREAD_LOCAL) - add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) -endif() +# thread_local is part of C++11 and later (TODO: clean up this define) +add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) @@ -182,7 +180,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/env.cc ${ROCKSDB_SOURCE_DIR}/env/env_chroot.cc ${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc - ${ROCKSDB_SOURCE_DIR}/env/env_hdfs.cc ${ROCKSDB_SOURCE_DIR}/env/file_system.cc ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc @@ -311,7 +308,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc ${ROCKSDB_SOURCE_DIR}/util/random.cc ${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc - ${ROCKSDB_SOURCE_DIR}/util/regex.cc ${ROCKSDB_SOURCE_DIR}/util/ribbon_config.cc ${ROCKSDB_SOURCE_DIR}/util/slice.cc ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc @@ -335,6 +331,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters.cc ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc + ${ROCKSDB_SOURCE_DIR}/utilities/counted_fs.cc ${ROCKSDB_SOURCE_DIR}/utilities/debug.cc ${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc ${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc From b5cb264b017e965037dbb0bd4623df5f5a65ec0b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 15:41:30 +0000 Subject: [PATCH 0139/1170] Bump ICU to 71 --- contrib/icu | 2 +- contrib/icu-cmake/CMakeLists.txt | 10 +++++----- contrib/icudata | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/contrib/icu b/contrib/icu index a56dde820dc..c205e7ee49a 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668 +Subproject commit c205e7ee49a7086a28b9c275fcfdac9ca3dc815d diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index 0a650f2bcc0..f23b0002b8d 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -4,7 +4,9 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -if (NOT ENABLE_ICU) +# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they re-generated +# the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255 +if (NOT ENABLE_ICU OR ARCH_S390X) message(STATUS "Not using ICU") return() endif() @@ -12,8 +14,6 @@ endif() set(ICU_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/icu/icu4c/source") set(ICUDATA_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/icudata/") -set (CMAKE_CXX_STANDARD 17) - # These lists of sources were generated from build log of the original ICU build system (configure + make). set(ICUUC_SOURCES @@ -462,9 +462,9 @@ file(GENERATE OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/empty.cpp" CONTENT " ") enable_language(ASM) if (ARCH_S390X) - set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt70b_dat.S" ) + set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt71b_dat.S" ) else() - set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt70l_dat.S" ) + set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt71l_dat.S" ) endif() set(ICUDATA_SOURCES diff --git a/contrib/icudata b/contrib/icudata index c8e717892a5..e7488edd1f1 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit c8e717892a557b4d2852317c7d628aacc0a0e5ab +Subproject commit e7488edd1f141b0664553a985a6fcd0125279527 From 5b6956ea234962cca5414a3fb0a6191407b4305a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 16:48:09 +0000 Subject: [PATCH 0140/1170] Bump rocksdb to v7.10.2 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 53 +++++++++++++------------- contrib/rocksdb-cmake/build_version.cc | 31 +++++++++++---- 3 files changed, 51 insertions(+), 35 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 4fc59e24001..01e43568fa9 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 4fc59e240016a62180b09703e2938c3d7e928de0 +Subproject commit 01e43568fa9f3f7bf107b2b66c00b286b456f33e diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index f6479346063..98790158baa 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,6 +5,9 @@ if (NOT ENABLE_ROCKSDB) 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) @@ -16,14 +19,6 @@ option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) -# third-party/folly is only validated to work on Linux and Windows for now. -# So only turn it on there by default. -if(CMAKE_SYSTEM_NAME MATCHES "Linux|Windows") - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON) -else() - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) -endif() - if(WITH_SNAPPY) add_definitions(-DSNAPPY) list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) @@ -44,7 +39,7 @@ if(WITH_ZSTD) list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) endif() -option(PORTABLE "build a portable binary" ON) +add_definitions(-DROCKSDB_PORTABLE) if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) add_definitions(-DHAVE_SSE42) @@ -59,9 +54,6 @@ if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") endif() -# thread_local is part of C++11 and later (TODO: clean up this define) -add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) - if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") @@ -87,19 +79,21 @@ set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") include_directories(${ROCKSDB_SOURCE_DIR}) include_directories("${ROCKSDB_SOURCE_DIR}/include") -if(WITH_FOLLY_DISTRIBUTED_MUTEX) - include_directories("${ROCKSDB_SOURCE_DIR}/third-party/folly") -endif() set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/cache.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_key.cc + ${ROCKSDB_SOURCE_DIR}/cache/cache_helpers.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_reservation_manager.cc + ${ROCKSDB_SOURCE_DIR}/cache/charged_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/compressed_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc ${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.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 ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc @@ -111,6 +105,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_source.cc ${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc ${ROCKSDB_SOURCE_DIR}/db/builder.cc ${ROCKSDB_SOURCE_DIR}/db/c.cc @@ -122,7 +117,11 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_fifo.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_level.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_universal.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_service_job.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_state.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_outputs.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/sst_partitioner.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/subcompaction_state.cc ${ROCKSDB_SOURCE_DIR}/db/convenience.cc ${ROCKSDB_SOURCE_DIR}/db/db_filesnapshot.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/compacted_db_impl.cc @@ -157,10 +156,11 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc ${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc ${ROCKSDB_SOURCE_DIR}/db/output_validator.cc - ${ROCKSDB_SOURCE_DIR}/db/periodic_work_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/db/periodic_task_scheduler.cc ${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc ${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc ${ROCKSDB_SOURCE_DIR}/db/repair.cc + ${ROCKSDB_SOURCE_DIR}/db/seqno_to_time_mapping.cc ${ROCKSDB_SOURCE_DIR}/db/snapshot_impl.cc ${ROCKSDB_SOURCE_DIR}/db/table_cache.cc ${ROCKSDB_SOURCE_DIR}/db/table_properties_collector.cc @@ -172,6 +172,8 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/version_set.cc ${ROCKSDB_SOURCE_DIR}/db/wal_edit.cc ${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc + ${ROCKSDB_SOURCE_DIR}/db/wide/wide_column_serialization.cc + ${ROCKSDB_SOURCE_DIR}/db/wide/wide_columns.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc ${ROCKSDB_SOURCE_DIR}/db/write_controller.cc @@ -230,16 +232,17 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/options/options.cc ${ROCKSDB_SOURCE_DIR}/options/options_helper.cc ${ROCKSDB_SOURCE_DIR}/options/options_parser.cc + ${ROCKSDB_SOURCE_DIR}/port/mmap.cc ${ROCKSDB_SOURCE_DIR}/port/stack_trace.cc ${ROCKSDB_SOURCE_DIR}/table/adaptive/adaptive_table_factory.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/binary_search_index_reader.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block.cc - ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_filter_block.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_builder.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_factory.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_iterator.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_reader.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_cache.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefetcher.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefix_index.cc ${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_hash_index.cc @@ -297,9 +300,12 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_result.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc + ${ROCKSDB_SOURCE_DIR}/util/async_file_reader.cc + ${ROCKSDB_SOURCE_DIR}/util/cleanable.cc ${ROCKSDB_SOURCE_DIR}/util/coding.cc ${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc ${ROCKSDB_SOURCE_DIR}/util/comparator.cc + ${ROCKSDB_SOURCE_DIR}/util/compression.cc ${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc ${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc ${ROCKSDB_SOURCE_DIR}/util/crc32c.cc @@ -312,11 +318,13 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/slice.cc ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc ${ROCKSDB_SOURCE_DIR}/util/status.cc + ${ROCKSDB_SOURCE_DIR}/util/stderr_logger.cc ${ROCKSDB_SOURCE_DIR}/util/string_util.cc ${ROCKSDB_SOURCE_DIR}/util/thread_local.cc ${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc ${ROCKSDB_SOURCE_DIR}/util/xxhash.cc - ${ROCKSDB_SOURCE_DIR}/utilities/backupable/backupable_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/agg_merge/agg_merge.cc + ${ROCKSDB_SOURCE_DIR}/utilities/backup/backup_engine.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_compaction_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl.cc @@ -419,15 +427,6 @@ list(APPEND SOURCES "${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc" "${ROCKSDB_SOURCE_DIR}/env/io_posix.cc") -if(WITH_FOLLY_DISTRIBUTED_MUTEX) - list(APPEND SOURCES - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/detail/Futex.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/AtomicNotification.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/DistributedMutex.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/ParkingLot.cpp" - "${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/WaitOptions.cpp") -endif() - add_library(_rocksdb ${SOURCES}) add_library(ch_contrib::rocksdb ALIAS _rocksdb) target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) diff --git a/contrib/rocksdb-cmake/build_version.cc b/contrib/rocksdb-cmake/build_version.cc index f9639da516f..d5ea56673e0 100644 --- a/contrib/rocksdb-cmake/build_version.cc +++ b/contrib/rocksdb-cmake/build_version.cc @@ -1,16 +1,33 @@ // Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved. -/// This file was edited for ClickHouse. #include #include "rocksdb/version.h" +#include "rocksdb/utilities/object_registry.h" #include "util/string_util.h" // The build script may replace these values with real values based // on whether or not GIT is available and the platform settings -static const std::string rocksdb_build_git_sha = "rocksdb_build_git_sha:0"; -static const std::string rocksdb_build_git_tag = "rocksdb_build_git_tag:master"; -static const std::string rocksdb_build_date = "rocksdb_build_date:2000-01-01"; +static const std::string rocksdb_build_git_sha = "rocksdb_build_git_sha:72438a678872544809393b831c7273794c074215"; +static const std::string rocksdb_build_git_tag = "rocksdb_build_git_tag:main"; +#define HAS_GIT_CHANGES 0 +#if HAS_GIT_CHANGES == 0 +// If HAS_GIT_CHANGES is 0, the GIT date is used. +// Use the time the branch/tag was last modified +static const std::string rocksdb_build_date = "rocksdb_build_date:2024-07-12 16:01:57"; +#else +// If HAS_GIT_CHANGES is > 0, the branch/tag has modifications. +// Use the time the build was created. +static const std::string rocksdb_build_date = "rocksdb_build_date:2024-07-13 17:15:50"; +#endif + +extern "C" { + +} // extern "C" + +std::unordered_map ROCKSDB_NAMESPACE::ObjectRegistry::builtins_ = { + +}; namespace ROCKSDB_NAMESPACE { static void AddProperty(std::unordered_map *props, const std::string& name) { @@ -39,12 +56,12 @@ const std::unordered_map& GetRocksBuildProperties() { } std::string GetRocksVersionAsString(bool with_patch) { - std::string version = ToString(ROCKSDB_MAJOR) + "." + ToString(ROCKSDB_MINOR); + std::string version = std::to_string(ROCKSDB_MAJOR) + "." + std::to_string(ROCKSDB_MINOR); if (with_patch) { - return version + "." + ToString(ROCKSDB_PATCH); + return version + "." + std::to_string(ROCKSDB_PATCH); } else { return version; - } + } } std::string GetRocksBuildInfoAsString(const std::string& program, bool verbose) { From 68aebce89f30eb7766c420d25e852e21e27dfe7d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 15 Jul 2024 14:41:37 +0800 Subject: [PATCH 0141/1170] fix failed uts --- src/Functions/printf.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index a890b886338..3efe854a53b 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -93,6 +93,8 @@ private: { T a = data[i]; s = fmt::sprintf(format, static_cast>(a)); + + res_chars.resize(curr_offset + s.size() + 1); memcpy(&res_chars[curr_offset], s.data(), s.size()); res_chars[curr_offset + s.size()] = 0; @@ -115,6 +117,8 @@ private: { auto a = concrete_column->getDataAt(i).toView(); s = fmt::sprintf(format, a); + + res_chars.resize(curr_offset + s.size() + 1); memcpy(&res_chars[curr_offset], s.data(), s.size()); res_chars[curr_offset + s.size()] = 0; From 2132ce52e0f72afe90e72e756d5ef494ad081ea9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 15 Jul 2024 06:48:39 +0000 Subject: [PATCH 0142/1170] Bump ICU to 75 --- contrib/icu | 2 +- contrib/icu-cmake/CMakeLists.txt | 6 +++--- contrib/icudata | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/contrib/icu b/contrib/icu index c205e7ee49a..7750081bda4 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit c205e7ee49a7086a28b9c275fcfdac9ca3dc815d +Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index f23b0002b8d..f9d05f7fe97 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -4,7 +4,7 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they re-generated +# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they generated # the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255 if (NOT ENABLE_ICU OR ARCH_S390X) message(STATUS "Not using ICU") @@ -462,9 +462,9 @@ file(GENERATE OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/empty.cpp" CONTENT " ") enable_language(ASM) if (ARCH_S390X) - set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt71b_dat.S" ) + set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75b_dat.S" ) else() - set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt71l_dat.S" ) + set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" ) endif() set(ICUDATA_SOURCES diff --git a/contrib/icudata b/contrib/icudata index e7488edd1f1..d345d6ac22f 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit e7488edd1f141b0664553a985a6fcd0125279527 +Subproject commit d345d6ac22f381c882420de9053d30ae1ff38d75 From 77272c925dc15acc5fdd0260a1c0aab35b1df3c3 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 15:10:39 +0800 Subject: [PATCH 0143/1170] disable insertion and mutation --- docs/en/operations/settings/settings.md | 6 ++ src/Core/ServerSettings.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 7 ++ src/Interpreters/InterpreterDeleteQuery.cpp | 4 + src/Interpreters/InterpreterInsertQuery.cpp | 5 ++ .../__init__.py | 0 .../config/cluster.xml | 16 ++++ .../config/reading_node.xml | 3 + .../config/storage_policy.xml | 21 +++++ .../config/writing_node.xml | 3 + .../test.py | 84 +++++++++++++++++++ 11 files changed, 150 insertions(+) create mode 100644 tests/integration/test_disable_insertion_and_mutation/__init__.py create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/cluster.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml create mode 100644 tests/integration/test_disable_insertion_and_mutation/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..143ce836beb 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,9 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## disable_insertion_and_mutation + +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`. diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..cf09874125d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,6 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ + M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c70a3397f4e..b9dd59909e6 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -46,6 +46,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNKNOWN_TABLE; extern const int UNKNOWN_DATABASE; + extern const int QUERY_IS_PROHIBITED; } @@ -191,6 +192,12 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) "to execute ALTERs of different types (replicated and non replicated) in single query"); } + if (mutation_commands.hasNonEmptyMutationCommands() || !partition_commands.empty()) + { + if (getContext()->getServerSettings().disable_insertion_and_mutation) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Mutations are prohibited"); + } + if (!alter_commands.empty()) { auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 39d5d9e9cef..5f3e3385148 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; + extern const int QUERY_IS_PROHIBITED; } @@ -50,6 +51,9 @@ BlockIO InterpreterDeleteQuery::execute() if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); + if (getContext()->getGlobalContext()->getServerSettings().disable_insertion_and_mutation) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Delete queries are prohibited"); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (database->shouldReplicateQuery(getContext(), query_ptr)) { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f396db70d21..b62a71de884 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,6 +44,7 @@ namespace ProfileEvents { extern const Event InsertQueriesWithSubqueries; extern const Event QueriesWithSubqueries; + extern const int QUERY_IS_PROHIBITED; } namespace DB @@ -406,6 +407,10 @@ BlockIO InterpreterInsertQuery::execute() StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); + if (getContext()->getServerSettings().disable_insertion_and_mutation + && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); + StoragePtr inner_table; if (const auto * mv = dynamic_cast(table.get())) inner_table = mv->getTargetTable(); diff --git a/tests/integration/test_disable_insertion_and_mutation/__init__.py b/tests/integration/test_disable_insertion_and_mutation/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml b/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml new file mode 100644 index 00000000000..17782a77679 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/cluster.xml @@ -0,0 +1,16 @@ + + + + + + writing_node + 9000 + + + reading_node + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml b/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml new file mode 100644 index 00000000000..becabce8a44 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/reading_node.xml @@ -0,0 +1,3 @@ + + true + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml new file mode 100644 index 00000000000..cec96cfcc1a --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml @@ -0,0 +1,21 @@ + + + + + s3_with_keeper + http://minio1:9001/root/data/ + minio + minio123 + + + + + +

+ s3_with_keeper +
+ + + + + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml b/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml new file mode 100644 index 00000000000..0737af7afc7 --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/config/writing_node.xml @@ -0,0 +1,3 @@ + + false + \ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py new file mode 100644 index 00000000000..5234ae9c57c --- /dev/null +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -0,0 +1,84 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +import time + +cluster = ClickHouseCluster(__file__) + +writing_node = cluster.add_instance( + "writing_node", + main_configs=["config/writing_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) +reading_node = cluster.add_instance( + "reading_node", + main_configs=["config/reading_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 2}, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_disable_insertion_and_mutation(started_cluster): + writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) SETTINGS storage_policy='s3_with_keeper' """) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("INSERT INTO my_table VALUES (1, 'hello')") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER TABLE my_table delete where 1") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER table my_table update key = 1 where 1") + ) + + assert ( + "QUERY_IS_PROHIBITED" + in reading_node.query_and_get_error("ALTER TABLE my_table drop partition 0") + ) + + reading_node.query("SELECT * from my_table"); + writing_node.query("INSERT INTO my_table VALUES (1, 'hello')") + writing_node.query("ALTER TABLE my_table delete where 1") + writing_node.query("ALTER table my_table update value = 'no hello' where 1") + + reading_node.query("ALTER TABLE my_table ADD COLUMN new_column UInt64") + writing_node.query("SELECT new_column from my_table") + reading_node.query("SELECT new_column from my_table") + + reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") + + assert( + "new_column\tString" + in reading_node.query("DESC my_table") + ) + + assert( + "new_column\tString" + in writing_node.query("DESC my_table") + ) 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 0144/1170] 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 0145/1170] 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 fc29ac7891eddd3a714f5af574c71040f91f451d Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 17:06:37 +0800 Subject: [PATCH 0146/1170] add error extern to fix compile error --- src/Interpreters/InterpreterInsertQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b62a71de884..c01b2196ac9 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -56,6 +56,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; + extern const int QUERY_IS_PROHIBITED; } InterpreterInsertQuery::InterpreterInsertQuery( From ccba078da10bed8d42e821f8bcdd47f448d198a0 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 15 Jul 2024 09:46:31 +0000 Subject: [PATCH 0147/1170] change storage policy to default --- .../config/storage_policy.xml | 21 ------------------- .../test.py | 6 +++--- 2 files changed, 3 insertions(+), 24 deletions(-) delete mode 100644 tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml diff --git a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml b/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml deleted file mode 100644 index cec96cfcc1a..00000000000 --- a/tests/integration/test_disable_insertion_and_mutation/config/storage_policy.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - s3_with_keeper - http://minio1:9001/root/data/ - minio - minio123 - - - - - -
- s3_with_keeper -
-
-
-
-
-
\ No newline at end of file diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index 5234ae9c57c..f098f130d2b 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) writing_node = cluster.add_instance( "writing_node", - main_configs=["config/writing_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + main_configs=["config/writing_node.xml", "config/cluster.xml"], with_zookeeper=True, with_minio=True, stay_alive=True, @@ -15,7 +15,7 @@ writing_node = cluster.add_instance( ) reading_node = cluster.add_instance( "reading_node", - main_configs=["config/reading_node.xml", "config/storage_policy.xml", "config/cluster.xml"], + main_configs=["config/reading_node.xml", "config/cluster.xml"], with_zookeeper=True, with_minio=True, stay_alive=True, @@ -35,7 +35,7 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): - writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) SETTINGS storage_policy='s3_with_keeper' """) + writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) assert ( "QUERY_IS_PROHIBITED" 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 0148/1170] 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 0149/1170] 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 b2466466d46ab1828b67693edf520d52ff45bc65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 15 Jul 2024 14:28:12 +0200 Subject: [PATCH 0150/1170] Rename Context::getSettings() --- programs/local/LocalServer.cpp | 2 +- src/Analyzer/QueryTreeBuilder.cpp | 2 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- src/Analyzer/Utils.cpp | 2 +- src/Bridge/IBridge.cpp | 2 +- src/Client/ClientBase.cpp | 6 +++--- src/Databases/DatabaseDictionary.cpp | 4 ++-- src/Databases/DatabaseOnDisk.cpp | 4 ++-- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Functions/formatQuery.cpp | 2 +- src/Functions/hasColumnInTable.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 5 ++--- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 2 +- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/interpretSubquery.cpp | 2 +- src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- .../QueryPlan/Optimizations/optimizeReadInOrder.cpp | 2 +- src/Server/MySQLHandler.cpp | 2 +- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 2 +- src/Storages/StorageURL.cpp | 4 ++-- src/Storages/StorageView.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 2 +- src/TableFunctions/Hive/TableFunctionHive.cpp | 2 +- 30 files changed, 37 insertions(+), 38 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 46b543e49e9..a7265ef0de4 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -80,7 +80,7 @@ namespace ErrorCodes void applySettingsOverridesForLocal(ContextMutablePtr context) { - Settings settings = context->getSettings(); + Settings settings = context->getSettingsCopy(); settings.allow_introspection_functions = true; settings.storage_file_read_method = LocalFSReadMethod::mmap; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index a62b6e56ac5..0a732a3b3b3 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -237,7 +237,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q /// Remove global settings limit and offset if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset) { - Settings settings = updated_context->getSettings(); + Settings settings = updated_context->getSettingsCopy(); limit = settings.limit; offset = settings.offset; settings.limit = 0; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 5f7b06231d9..92618dfe346 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -503,7 +503,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden ProfileEvents::increment(ProfileEvents::ScalarSubqueriesCacheMiss); auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 1; subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index d10bbd9bd23..e5f372b7368 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -867,7 +867,7 @@ void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context) * max_rows_in_join, max_bytes_in_join, join_overflow_mode, * which are checked separately (in the Set, Join objects). */ - Settings subquery_settings = mutable_context->getSettings(); + Settings subquery_settings = mutable_context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index de48a4f2b84..5682a28f899 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -232,7 +232,7 @@ int IBridge::main(const std::vector & /*args*/) auto context = Context::createGlobal(shared_context.get()); context->makeGlobalContext(); - auto settings = context->getSettings(); + auto settings = context->getSettingsCopy(); settings.set("http_max_field_value_size", http_max_field_value_size); context->setSettings(settings); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d472ba99b9..2dc603a307f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -723,7 +723,7 @@ void ClientBase::initLogsOutputStream() void ClientBase::adjustSettings() { - Settings settings = global_context->getSettings(); + Settings settings = global_context->getSettingsCopy(); /// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles) @@ -931,7 +931,7 @@ bool ClientBase::isSyncInsertWithData(const ASTInsertQuery & insert_query, const if (!insert_query.data) return false; - auto settings = context->getSettings(); + auto settings = context->getSettingsCopy(); if (insert_query.settings_ast) settings.applyChanges(insert_query.settings_ast->as()->changes); @@ -2696,7 +2696,7 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name) if (!getClientConfiguration().has("log_comment")) { - Settings settings = global_context->getSettings(); + Settings settings = global_context->getSettingsCopy(); /// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]" settings.log_comment = fs::absolute(fs::path(file_name)); global_context->setSettings(settings); diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 52196e75c4a..a9569408814 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -111,7 +111,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; @@ -133,7 +133,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) buffer << " COMMENT " << backQuote(comment_value); } - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ParserCreateQuery parser; return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 07a250e72c7..261a917c595 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -523,7 +523,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const { ASTPtr ast; - auto settings = getContext()->getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); { std::lock_guard lock(mutex); auto database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"; @@ -722,7 +722,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( return nullptr; } - auto settings = local_context->getSettingsRef(); + const auto & settings = local_context->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 2c342755337..04b4070d5af 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -89,7 +89,7 @@ static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; static ContextMutablePtr createQueryContext(ContextPtr context) { - Settings new_query_settings = context->getSettings(); + Settings new_query_settings = context->getSettingsCopy(); new_query_settings.insert_allow_materialized_columns = true; /// To avoid call AST::format diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index d10b3f9a5b7..f7f7e4b5bcb 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -39,7 +39,7 @@ public: FunctionFormatQuery(ContextPtr context, String name_, OutputFormatting output_formatting_, ErrorHandling error_handling_) : name(name_), output_formatting(output_formatting_), error_handling(error_handling_) { - const Settings & settings = context->getSettings(); + const Settings & settings = context->getSettingsRef(); max_query_size = settings.max_query_size; max_parser_depth = settings.max_parser_depth; max_parser_backtracks = settings.max_parser_backtracks; diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 00714997b4a..cc496270b01 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -143,7 +143,7 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a /* cluster_name= */ "", /* password= */ "" }; - auto cluster = std::make_shared(getContext()->getSettings(), host_names, params); + auto cluster = std::make_shared(getContext()->getSettingsRef(), host_names, params); // FIXME this (probably) needs a non-constant access to query context, // because it might initialized a storage. Ideally, the tables required diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..3a88e0ccfe1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2267,7 +2267,7 @@ bool Context::displaySecretsInShowAndSelect() const return shared->server_settings.display_secrets_in_show_and_select; } -Settings Context::getSettings() const +Settings Context::getSettingsCopy() const { SharedLockGuard lock(mutex); return *settings; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 284cac50769..61095e53a17 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -830,7 +830,8 @@ public: void setMacros(std::unique_ptr && macros); bool displaySecretsInShowAndSelect() const; - Settings getSettings() const; + Settings getSettingsCopy() const; + const Settings & getSettingsRef() const { return *settings; } void setSettings(const Settings & settings_); /// Set settings by name. @@ -955,8 +956,6 @@ public: void makeSessionContext(); void makeGlobalContext(); - const Settings & getSettingsRef() const { return *settings; } - void setProgressCallback(ProgressCallback callback); /// Used in executeQuery() to pass it to the QueryPipeline. ProgressCallback getProgressCallback() const; diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 4bd1c47d5a0..1ca8c40460c 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -74,7 +74,7 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data) static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqueriesMatcher::Data & data) { auto subquery_context = Context::createCopy(data.getContext()); - Settings subquery_settings = data.getContext()->getSettings(); + Settings subquery_settings = data.getContext()->getSettingsCopy(); subquery_settings.max_result_rows = 1; subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 16d0eb71278..1a4c02bdebb 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -171,7 +171,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( PreparedSetsPtr prepared_sets_, bool is_create_parameterized_view_) : WithContext(context_) - , query(query_), settings(getContext()->getSettings()) + , query(query_), settings(getContext()->getSettingsRef()) , subquery_depth(subquery_depth_) , syntax(syntax_analyzer_result_) , is_create_parameterized_view(is_create_parameterized_view_) @@ -983,7 +983,7 @@ static std::shared_ptr tryCreateJoin( algorithm == JoinAlgorithm::PARALLEL_HASH || algorithm == JoinAlgorithm::DEFAULT) { - const auto & settings = context->getSettings(); + const auto & settings = context->getSettingsRef(); if (analyzed_join->allowParallelHashJoin()) return std::make_shared( diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index ef298d4d45a..dffa0cbaa5b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -469,7 +469,7 @@ BlockIO InterpreterInsertQuery::execute() * to avoid unnecessary squashing. */ - Settings new_settings = getContext()->getSettings(); + Settings new_settings = getContext()->getSettingsCopy(); new_settings.max_threads = std::max(1, settings.max_insert_threads); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cd91f9532b9..cb3c478dbb1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -253,7 +253,7 @@ namespace ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 0de2bf9cb1f..c5226107f8d 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -308,7 +308,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se if (tables_with_columns.size() < 2) return {}; - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); MultiEnum join_algorithm = settings.join_algorithm; bool try_use_direct_join = join_algorithm.isSet(JoinAlgorithm::DIRECT) || join_algorithm.isSet(JoinAlgorithm::DEFAULT); auto table_join = std::make_shared(settings, context->getGlobalTemporaryVolume(), context->getTempDataOnDisk()); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 5b07852d9e3..271e23a7288 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -657,7 +657,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even { if (auto ctx = context.lock()) { - res.query_settings = std::make_shared(ctx->getSettings()); + res.query_settings = std::make_shared(ctx->getSettingsRef()); res.current_database = ctx->getCurrentDatabase(); } } diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 340f6d1d805..909875b99a0 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -62,7 +62,7 @@ std::shared_ptr interpretSubquery( * which are checked separately (in the Set, Join objects). */ auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = context->getSettingsCopy(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c23d717d52f..e467c358d1d 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -406,7 +406,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx { const Block & header = getPort().getHeader(); const IDataType & type = *header.getByPosition(column_idx).type; - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); /// Advance the token iterator until the start of the column expression readUntilTheEndOfRowAndReTokenize(column_idx); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 28eb4da2e17..415a6a11999 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1057,7 +1057,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, } auto context = read_from_merge_tree->getContext(); - const auto & settings = context->getSettings(); + const auto & settings = context->getSettingsRef(); if (!settings.optimize_read_in_window_order || (settings.optimize_read_in_order && settings.query_plan_read_in_order) || context->getSettingsRef().allow_experimental_analyzer) { return 0; diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index b6d795b1e69..3deb09bae88 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -473,7 +473,7 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool binary_protocol) query_context->setCurrentQueryId(fmt::format("mysql:{}:{}", connection_id, toString(UUIDHelpers::generateV4()))); /// --- Workaround for Bug 56173. Can be removed when the analyzer is on by default. - auto settings = query_context->getSettings(); + auto settings = query_context->getSettingsCopy(); settings.prefer_column_name_to_alias = true; query_context->setSettings(settings); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..c287fc817eb 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -283,7 +283,7 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c auto pools = createPoolsForAddresses(addresses, pool_factory, storage.log); - const auto settings = storage.getContext()->getSettings(); + const auto & settings = storage.getContext()->getSettingsRef(); return std::make_shared(std::move(pools), settings.load_balancing, settings.distributed_replica_error_half_life.totalSeconds(), diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 38869aebaa5..4ee68580d3f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -34,7 +34,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( , write_settings(write_settings_) { MergeTreeWriterSettings writer_settings( - data_part->storage.getContext()->getSettings(), + data_part->storage.getContext()->getSettingsRef(), write_settings, storage_settings, data_part->index_granularity_info.mark_type.adaptive, diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index c167ac87317..05cd77dcd40 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -23,7 +23,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeIndexGranularityInfo * index_granularity_info) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { - const auto & global_settings = data_part->storage.getContext()->getSettings(); + const auto & global_settings = data_part->storage.getContext()->getSettingsRef(); MergeTreeWriterSettings writer_settings( global_settings, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9cec8c75ebe..731bd7ec3d3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -462,7 +462,7 @@ std::pair> StorageURLSource: setCredentials(credentials, request_uri); - const auto settings = context_->getSettings(); + const auto & settings = context_->getSettingsRef(); auto proxy_config = getProxyConfiguration(request_uri.getScheme()); @@ -1324,7 +1324,7 @@ std::optional IStorageURLBase::tryGetLastModificationTime( const Poco::Net::HTTPBasicCredentials & credentials, const ContextPtr & context) { - auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); auto uri = Poco::URI(url); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 5f768bce978..929896e3246 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -97,7 +97,7 @@ bool hasJoin(const ASTSelectWithUnionQuery & ast) ContextPtr getViewContext(ContextPtr context, const StorageSnapshotPtr & storage_snapshot) { auto view_context = storage_snapshot->metadata->getSQLSecurityOverriddenContext(context); - Settings view_settings = view_context->getSettings(); + Settings view_settings = view_context->getSettingsCopy(); view_settings.max_result_rows = 0; view_settings.max_result_bytes = 0; view_settings.extremes = false; diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 56071abaa95..9d23f132759 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -65,7 +65,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), /// since this is a service query and should not lead to query failure. { - Settings new_settings = new_context->getSettings(); + Settings new_settings = new_context->getSettingsCopy(); new_settings.max_result_rows = 0; new_settings.max_result_bytes = 0; new_context->setSettings(new_settings); diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index 80494dbe5a8..759807d7a4f 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -93,7 +93,7 @@ StoragePtr TableFunctionHive::executeImpl( ColumnsDescription /*cached_columns_*/, bool /*is_insert_query*/) const { - const Settings & settings = context_->getSettings(); + const Settings & settings = context_->getSettingsRef(); ParserExpression partition_by_parser; ASTPtr partition_by_ast = parseQuery( partition_by_parser, 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 0151/1170] 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 0152/1170] 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 908f5899ddfdc701df5e9e6189760431e88b6695 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 17:28:33 +0200 Subject: [PATCH 0153/1170] Add settings to replace external engines to Null --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 3 +- src/Interpreters/InterpreterCreateQuery.cpp | 40 +++++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 52fa28a4481..7bf97896357 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,6 +891,7 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ + M(Bool, restore_replace_external_engine_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9b72209103..a23d9d17da2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,8 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } + + void setNullTableEngine(ASTStorage &storage) + { + auto engine_ast = std::make_shared(); + engine_ast->name = "Null"; + engine_ast->no_empty_args = true; + storage.set(storage.engine, engine_ast); + } } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const @@ -1000,6 +1008,38 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. if (!create.storage->engine) setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); + /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to + /// Null table engine. + else (create.storage->engine == "AzureBlobStorage" || + create.storage->engine == "AzureQueue" || + create.storage->engine == "COSN" || + create.storage->engine == "DeltaLake" || + create.storage->engine == "Dictionary" || + create.storage->engine == "Executable" || + create.storage->engine == "ExecutablePool" || + create.storage->engine == "ExternalDistributed" || + create.storage->engine == "File" || + create.storage->engine == "Hudi" || + create.storage->engine == "Iceberg" || + create.storage->engine == "JDBC" || + create.storage->engine == "Kafka" || + create.storage->engine == "MaterializedPostgreSQL" || + create.storage->engine == "MongoDB" || + create.storage->engine == "MySQL" || + create.storage->engine == "NATS" || + create.storage->engine == "ODBC" || + create.storage->engine == "OSS" || + create.storage->engine == "PostgreSQL" || + create.storage->engine == "RabbitMQ" || + create.storage->engine == "Redis" || + create.storage->engine == "S3" || + create.storage->engine == "S3Queue" || + create.storage->engine == "TinyLog" || + create.storage->engine == "URL") + { + if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) + setNullTableEngine(*create.storage) + } return; } From 344e5b716d49eda59783b6fff85757e6a5b6e98f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 15 Jul 2024 17:41:32 +0200 Subject: [PATCH 0154/1170] cancel method is noexcept --- src/Common/ConcurrentBoundedQueue.h | 2 +- .../Executors/CompletedPipelineExecutor.cpp | 2 -- .../Formats/Impl/ArrowBlockInputFormat.h | 2 +- .../Formats/Impl/DWARFBlockInputFormat.h | 2 +- .../Formats/Impl/NativeORCBlockInputFormat.h | 2 +- .../Formats/Impl/ORCBlockInputFormat.h | 2 +- .../Impl/ParallelFormattingOutputFormat.cpp | 2 +- .../Impl/ParallelFormattingOutputFormat.h | 4 ++-- .../Impl/ParallelParsingInputFormat.cpp | 2 +- .../Formats/Impl/ParallelParsingInputFormat.h | 4 ++-- .../Formats/Impl/ParquetBlockInputFormat.h | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 2 +- .../Formats/Impl/ParquetMetadataInputFormat.h | 2 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- src/Processors/IProcessor.cpp | 4 ++-- src/Processors/IProcessor.h | 4 ++-- src/Processors/Sources/RemoteSource.cpp | 13 ++++++++++-- src/Processors/Sources/RemoteSource.h | 2 +- .../Transforms/AggregatingTransform.cpp | 2 +- src/Server/TCPHandler.cpp | 1 - src/Storages/Distributed/DistributedSink.cpp | 20 +++++++++++++++---- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/LiveView/LiveViewEventsSource.h | 2 +- src/Storages/LiveView/LiveViewSource.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.h | 2 +- src/Storages/MergeTree/MergeTreeSink.cpp | 5 ----- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/MergeTree/MergeTreeSource.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 16 --------------- src/Storages/MessageQueueSink.cpp | 13 ++++++++++++ src/Storages/MessageQueueSink.h | 6 +----- .../StorageObjectStorageSink.cpp | 2 +- 33 files changed, 70 insertions(+), 64 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 16b9488c98d..a830ae157a5 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -243,7 +243,7 @@ public: } /// Clear and finish queue - void clearAndFinish() + void clearAndFinish() noexcept { { std::lock_guard lock(queue_mutex); diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 909e742ffbf..1eeee896ede 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -99,7 +99,6 @@ void CompletedPipelineExecutor::execute() if (is_cancelled_callback()) { - LOG_INFO(getLogger("CompletedPipelineExecutor"), "execute CancelCallback FULLY_CANCELLED"); data->executor->cancel(); } } @@ -121,7 +120,6 @@ CompletedPipelineExecutor::~CompletedPipelineExecutor() { if (data && data->executor) { - LOG_INFO(getLogger("CompletedPipelineExecutor"), "~CompletedPipelineExecutor"); data->executor->cancel(); } } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 4fe01d0be12..cb74a9dd93e 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -32,7 +32,7 @@ public: private: Chunk read() override; - void onCancelX() override + void onCancel() noexcept override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h index 6cab5d34994..2d94d166708 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h @@ -32,7 +32,7 @@ public: protected: Chunk read() override; - void onCancelX() override + void onCancel() noexcept override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index de9925e3737..e4f2ef9ebe3 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -64,7 +64,7 @@ public: protected: Chunk read() override; - void onCancelX() override { is_stopped = 1; } + void onCancel() noexcept override { is_stopped = 1; } private: void prepareFileReader(); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 167436ad4b9..85f1636d3dc 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -34,7 +34,7 @@ public: protected: Chunk read() override; - void onCancelX() override + void onCancel() noexcept override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index b2871310be5..5d404d493a6 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -96,7 +96,7 @@ namespace DB } - void ParallelFormattingOutputFormat::finishAndWait() + void ParallelFormattingOutputFormat::finishAndWait() noexcept { emergency_stop = true; diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 40774fcfbfa..54617c77f86 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -122,7 +122,7 @@ public: started_prefix = true; } - void onCancelX() override + void onCancel() noexcept override { finishAndWait(); } @@ -268,7 +268,7 @@ private: bool collected_suffix = false; bool collected_finalize = false; - void finishAndWait(); + void finishAndWait() noexcept; void onBackgroundException() { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index d38a299cb6e..447adb1ed48 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -176,7 +176,7 @@ Chunk ParallelParsingInputFormat::read() if (background_exception) { lock.unlock(); - onCancelX(); + onCancel(); std::rethrow_exception(background_exception); } diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index eed40dc43e5..b97bf5213e6 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -137,7 +137,7 @@ private: Chunk read() final; - void onCancelX() final + void onCancel() noexcept final { /* * The format parsers themselves are not being cancelled here, so we'll @@ -292,7 +292,7 @@ private: first_parser_finished.wait(); } - void finishAndWait() + void finishAndWait() noexcept { /// Defending concurrent segmentator thread join std::lock_guard finish_and_wait_lock(finish_and_wait_mutex); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 0123329f026..ed528cc077c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -68,7 +68,7 @@ public: private: Chunk read() override; - void onCancelX() override + void onCancel() noexcept override { is_stopped = 1; } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index d08c91d286b..01fb97223f1 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -268,7 +268,7 @@ void ParquetBlockOutputFormat::resetFormatterImpl() staging_bytes = 0; } -void ParquetBlockOutputFormat::onCancelX() +void ParquetBlockOutputFormat::onCancel() noexcept { is_stopped = true; } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 0704ba0ed90..f8f5d2556a5 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -112,7 +112,7 @@ private: void consume(Chunk) override; void finalizeImpl() override; void resetFormatterImpl() override; - void onCancelX() override; + void onCancel() noexcept override; void writeRowGroup(std::vector chunks); void writeUsingArrow(std::vector chunks); diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h index 35180d202d8..5d2d8989859 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h @@ -65,7 +65,7 @@ public: private: Chunk read() override; - void onCancelX() override + void onCancel() noexcept override { is_stopped = 1; } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index a245df8234d..c803ed5dc61 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -29,7 +29,7 @@ public: void setRowsBeforeLimit(size_t rows_before_limit) override; - void onCancelX() override + void onCancel() noexcept override { queue.clearAndFinish(); } diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index 4d95bb5f3e0..edb4d662d8b 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -9,14 +9,14 @@ namespace DB { -void IProcessor::cancel() +void IProcessor::cancel() noexcept { bool already_cancelled = is_cancelled.exchange(true, std::memory_order_acq_rel); if (already_cancelled) return; - onCancelX(); + onCancel(); } String IProcessor::debug() const diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 5f4d71fbf54..680a446173e 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -238,7 +238,7 @@ public: /// In case if query was cancelled executor will wait till all processors finish their jobs. /// Generally, there is no reason to check this flag. However, it may be reasonable for long operations (e.g. i/o). bool isCancelled() const { return is_cancelled.load(std::memory_order_acquire); } - void cancel(); + void cancel() noexcept; /// Additional method which is called in case if ports were updated while work() method. /// May be used to stop execution in rare cases. @@ -363,7 +363,7 @@ public: virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} protected: - virtual void onCancelX() {} + virtual void onCancel() noexcept {} std::atomic is_cancelled{false}; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 44cf26e0b01..42696f9c3ce 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB { @@ -182,9 +184,16 @@ std::optional RemoteSource::tryGenerate() return chunk; } -void RemoteSource::onCancelX() +void RemoteSource::onCancel() noexcept { - query_executor->cancel(); + try + { + query_executor->cancel(); + } + catch (...) + { + tryLogCurrentException(getLogger("RemoteSource"), "Error occurs on cancelation."); + } } void RemoteSource::onUpdatePorts() diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 880eb234bfb..adf1b8e9fac 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -36,7 +36,7 @@ public: protected: std::optional tryGenerate() override; - void onCancelX() override; + void onCancel() noexcept override; private: bool was_query_sent = false; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index e42c1fd3a8d..870ba84722d 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -375,7 +375,7 @@ public: return prepareTwoLevel(); } - void onCancelX() override + void onCancel() noexcept override { shared_data->is_cancelled.store(true, std::memory_order_seq_cst); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ba12cad7771..e7342ff9b6f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -557,7 +557,6 @@ void TCPHandler::runImpl() if (getQueryCancellationStatus() == CancellationStatus::FULLY_CANCELLED) { - LOG_INFO(log, "CancelCallback FULLY_CANCELLED"); return true; } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 6283594e0d2..d67910a141f 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -596,7 +596,7 @@ void DistributedSink::onFinish() } } -void DistributedSink::onCancelX() +void DistributedSink::onCancel() noexcept { std::lock_guard lock(execution_mutex); if (pool && !pool->finished()) @@ -607,14 +607,26 @@ void DistributedSink::onCancelX() } catch (...) { - tryLogCurrentException(storage.log); + tryLogCurrentException(storage.log, "Error occurs on cancelation."); } } for (auto & shard_jobs : per_shard_jobs) + { for (JobReplica & job : shard_jobs.replicas_jobs) - if (job.executor) - job.executor->cancel(); + { + try + { + if (job.executor) + job.executor->cancel(); + } + catch (...) + { + tryLogCurrentException(storage.log, "Error occurs on cancelation."); + } + } + } + } diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 1dac3eeba6d..65a5eb52787 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -53,7 +53,7 @@ public: void onFinish() override; private: - void onCancelX() override; + void onCancel() noexcept override; IColumn::Selector createSelector(const Block & source_block) const; diff --git a/src/Storages/LiveView/LiveViewEventsSource.h b/src/Storages/LiveView/LiveViewEventsSource.h index d1ed222c185..4210acbc5bc 100644 --- a/src/Storages/LiveView/LiveViewEventsSource.h +++ b/src/Storages/LiveView/LiveViewEventsSource.h @@ -54,7 +54,7 @@ public: String getName() const override { return "LiveViewEventsSource"; } - void onCancelX() override + void onCancel() noexcept override { if (storage->shutdown_called) return; diff --git a/src/Storages/LiveView/LiveViewSource.h b/src/Storages/LiveView/LiveViewSource.h index 83589067cf5..81dd5620e57 100644 --- a/src/Storages/LiveView/LiveViewSource.h +++ b/src/Storages/LiveView/LiveViewSource.h @@ -36,7 +36,7 @@ public: String getName() const override { return "LiveViewSource"; } - void onCancelX() override + void onCancel() noexcept override { if (storage->shutdown_called) return; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 03ca30dd5b3..7a9cebbcb2e 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -48,7 +48,7 @@ public: ChunkAndProgress read(); - void cancel() { is_cancelled = true; } + void cancel() noexcept { is_cancelled = true; } const MergeTreeReaderSettings & getSettings() const { return reader_settings; } diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index d2e34665962..210a7057f94 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -28,9 +28,6 @@ struct MergeTreeSink::DelayedChunk MergeTreeSink::~MergeTreeSink() { - size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; - LOG_INFO(storage.log, "~ReplicatedMergeTreeSinkImpl, delayed_chunk {}, called from {}", addr, StackTrace().toString()); - if (!delayed_chunk) return; @@ -40,8 +37,6 @@ MergeTreeSink::~MergeTreeSink() } delayed_chunk.reset(); - - LOG_INFO(storage.log, "~ReplicatedMergeTreeSinkImpl end"); } MergeTreeSink::MergeTreeSink( diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 4070ccf4433..380c47723bc 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -149,7 +149,7 @@ std::string MergeTreeSource::getName() const return processor->getName(); } -void MergeTreeSource::onCancelX() +void MergeTreeSource::onCancel() noexcept { processor->cancel(); } diff --git a/src/Storages/MergeTree/MergeTreeSource.h b/src/Storages/MergeTree/MergeTreeSource.h index c7092aa26b1..7506af4f9b8 100644 --- a/src/Storages/MergeTree/MergeTreeSource.h +++ b/src/Storages/MergeTree/MergeTreeSource.h @@ -26,7 +26,7 @@ public: protected: std::optional tryGenerate() override; - void onCancelX() override; + void onCancel() noexcept override; private: MergeTreeSelectProcessorPtr processor; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 6c7ed9bdae0..7bfe647fa7f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -155,9 +155,6 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( template ReplicatedMergeTreeSinkImpl::~ReplicatedMergeTreeSinkImpl() { - size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; - LOG_INFO(log, "~ReplicatedMergeTreeSinkImpl, delayed_chunk {}, called from {}", addr, StackTrace().toString()); - if (!delayed_chunk) return; @@ -167,8 +164,6 @@ ReplicatedMergeTreeSinkImpl::~ReplicatedMergeTreeSinkImpl() } delayed_chunk.reset(); - - LOG_INFO(log, "~ReplicatedMergeTreeSinkImpl end"); } template @@ -273,8 +268,6 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const template void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { - LOG_INFO(log, "consume"); - if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); @@ -448,9 +441,6 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { - size_t addr = delayed_chunk ? size_t(delayed_chunk.get()) : 0; - LOG_INFO(log, "finishDelayedChunk {}", addr); - if (!delayed_chunk) return; @@ -480,22 +470,16 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF { auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus::fromCurrentException("", true)); - - size_t addr1 = delayed_chunk ? size_t(delayed_chunk.get()) : 0; - LOG_INFO(log, "finishDelayedChunk exception, delayed_chunk {}", addr1); throw; } } delayed_chunk.reset(); - - LOG_INFO(log, "finishDelayedChunk end, delayed_chunk {}", bool(delayed_chunk)); } template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { - if (!delayed_chunk) return; diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 4fb81d69070..10617422f40 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace DB @@ -79,4 +80,16 @@ void MessageQueueSink::consume(Chunk chunk) } +void MessageQueueSink::onCancel() noexcept +{ + try + { + onFinish(); + } + catch (...) + { + tryLogCurrentException(getLogger("MessageQueueSink"), "Error occurs on cancelation."); + } +} + } diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index 38754e9475e..6964af8cb4d 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -33,17 +33,13 @@ public: const String & storage_name_, const ContextPtr & context_); - ~MessageQueueSink() override - { - onFinish(); - } - String getName() const override { return storage_name + "Sink"; } void consume(Chunk chunk) override; void onStart() override; void onFinish() override; + void onCancel() noexcept override; void onException(std::exception_ptr /* exception */) override { onFinish(); } protected: diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 3bd0e88ecdb..d5813015e13 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -2,8 +2,8 @@ #include #include #include -#include "base/defines.h" #include +#include namespace DB { From 12101f455b06a3df1bd00b8b070b4c7862d087bf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 15 Jul 2024 17:55:02 +0200 Subject: [PATCH 0155/1170] fix typo --- src/Processors/Sources/RemoteSource.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 4 ++-- src/Storages/MessageQueueSink.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 42696f9c3ce..48a6804de9a 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -192,7 +192,7 @@ void RemoteSource::onCancel() noexcept } catch (...) { - tryLogCurrentException(getLogger("RemoteSource"), "Error occurs on cancelation."); + tryLogCurrentException(getLogger("RemoteSource"), "Error occurs on cancellation."); } } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index fa4ba01a37c..197905c0849 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -608,7 +608,7 @@ void DistributedSink::onCancel() noexcept } catch (...) { - tryLogCurrentException(storage.log, "Error occurs on cancelation."); + tryLogCurrentException(storage.log, "Error occurs on cancellation."); } } @@ -623,7 +623,7 @@ void DistributedSink::onCancel() noexcept } catch (...) { - tryLogCurrentException(storage.log, "Error occurs on cancelation."); + tryLogCurrentException(storage.log, "Error occurs on cancellation."); } } } diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 10617422f40..9cddb2e7ce8 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -88,7 +88,7 @@ void MessageQueueSink::onCancel() noexcept } catch (...) { - tryLogCurrentException(getLogger("MessageQueueSink"), "Error occurs on cancelation."); + tryLogCurrentException(getLogger("MessageQueueSink"), "Error occurs on cancellation."); } } From 7d70968db3527d894bc6c02d51dc70f932f7eacd Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 18:47:04 +0200 Subject: [PATCH 0156/1170] try fix --- src/Interpreters/InterpreterCreateQuery.cpp | 52 ++++++++++----------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f8696caebe7..9eb13a29af7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1010,32 +1010,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else (create.storage->engine == "AzureBlobStorage" || - create.storage->engine == "AzureQueue" || - create.storage->engine == "COSN" || - create.storage->engine == "DeltaLake" || - create.storage->engine == "Dictionary" || - create.storage->engine == "Executable" || - create.storage->engine == "ExecutablePool" || - create.storage->engine == "ExternalDistributed" || - create.storage->engine == "File" || - create.storage->engine == "Hudi" || - create.storage->engine == "Iceberg" || - create.storage->engine == "JDBC" || - create.storage->engine == "Kafka" || - create.storage->engine == "MaterializedPostgreSQL" || - create.storage->engine == "MongoDB" || - create.storage->engine == "MySQL" || - create.storage->engine == "NATS" || - create.storage->engine == "ODBC" || - create.storage->engine == "OSS" || - create.storage->engine == "PostgreSQL" || - create.storage->engine == "RabbitMQ" || - create.storage->engine == "Redis" || - create.storage->engine == "S3" || - create.storage->engine == "S3Queue" || - create.storage->engine == "TinyLog" || - create.storage->engine == "URL") + else (create.storage->engine->name == "AzureBlobStorage" || + create.storage->engine->name == "AzureQueue" || + create.storage->engine->name == "COSN" || + create.storage->engine->name == "DeltaLake" || + create.storage->engine->name == "Dictionary" || + create.storage->engine->name == "Executable" || + create.storage->engine->name == "ExecutablePool" || + create.storage->engine->name == "ExternalDistributed" || + create.storage->engine->name == "File" || + create.storage->engine->name == "Hudi" || + create.storage->engine->name == "Iceberg" || + create.storage->engine->name == "JDBC" || + create.storage->engine->name == "Kafka" || + create.storage->engine->name == "MaterializedPostgreSQL" || + create.storage->engine->name == "MongoDB" || + create.storage->engine->name == "MySQL" || + create.storage->engine->name == "NATS" || + create.storage->engine->name == "ODBC" || + create.storage->engine->name == "OSS" || + create.storage->engine->name == "PostgreSQL" || + create.storage->engine->name == "RabbitMQ" || + create.storage->engine->name == "Redis" || + create.storage->engine->name == "S3" || + create.storage->engine->name == "S3Queue" || + create.storage->engine->name == "TinyLog" || + create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) setNullTableEngine(*create.storage) From fc49b1b75f9b075f28cdc4b7eeb768339bb1ebd5 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Jul 2024 19:02:21 +0200 Subject: [PATCH 0157/1170] semicolon --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9eb13a29af7..94230f0e7d1 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1038,7 +1038,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) - setNullTableEngine(*create.storage) + setNullTableEngine(*create.storage); } return; } 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 0158/1170] 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 0159/1170] 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 0160/1170] 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 0161/1170] 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 083e4b17db62121d6905c35480c3a462dc26477b Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 09:34:52 +0800 Subject: [PATCH 0162/1170] trigger CI From 8aaf9c1d9824c136bbfc0532b040b2dca7564253 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:00:55 +0800 Subject: [PATCH 0163/1170] 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 0164/1170] 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 0165/1170] 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 0166/1170] 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 0167/1170] 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 478616de3d03495cf8c324da9464a9807b51ba41 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Jul 2024 10:54:39 +0000 Subject: [PATCH 0168/1170] forgot --- src/Interpreters/InterpreterCreateQuery.cpp | 52 ++++++++++----------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 94230f0e7d1..3b23c6899e9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1010,32 +1010,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else (create.storage->engine->name == "AzureBlobStorage" || - create.storage->engine->name == "AzureQueue" || - create.storage->engine->name == "COSN" || - create.storage->engine->name == "DeltaLake" || - create.storage->engine->name == "Dictionary" || - create.storage->engine->name == "Executable" || - create.storage->engine->name == "ExecutablePool" || - create.storage->engine->name == "ExternalDistributed" || - create.storage->engine->name == "File" || - create.storage->engine->name == "Hudi" || - create.storage->engine->name == "Iceberg" || - create.storage->engine->name == "JDBC" || - create.storage->engine->name == "Kafka" || - create.storage->engine->name == "MaterializedPostgreSQL" || - create.storage->engine->name == "MongoDB" || - create.storage->engine->name == "MySQL" || - create.storage->engine->name == "NATS" || - create.storage->engine->name == "ODBC" || - create.storage->engine->name == "OSS" || - create.storage->engine->name == "PostgreSQL" || - create.storage->engine->name == "RabbitMQ" || - create.storage->engine->name == "Redis" || - create.storage->engine->name == "S3" || - create.storage->engine->name == "S3Queue" || - create.storage->engine->name == "TinyLog" || - create.storage->engine->name == "URL") + else if (create.storage->engine->name == "AzureBlobStorage" || + create.storage->engine->name == "AzureQueue" || + create.storage->engine->name == "COSN" || + create.storage->engine->name == "DeltaLake" || + create.storage->engine->name == "Dictionary" || + create.storage->engine->name == "Executable" || + create.storage->engine->name == "ExecutablePool" || + create.storage->engine->name == "ExternalDistributed" || + create.storage->engine->name == "File" || + create.storage->engine->name == "Hudi" || + create.storage->engine->name == "Iceberg" || + create.storage->engine->name == "JDBC" || + create.storage->engine->name == "Kafka" || + create.storage->engine->name == "MaterializedPostgreSQL" || + create.storage->engine->name == "MongoDB" || + create.storage->engine->name == "MySQL" || + create.storage->engine->name == "NATS" || + create.storage->engine->name == "ODBC" || + create.storage->engine->name == "OSS" || + create.storage->engine->name == "PostgreSQL" || + create.storage->engine->name == "RabbitMQ" || + create.storage->engine->name == "Redis" || + create.storage->engine->name == "S3" || + create.storage->engine->name == "S3Queue" || + create.storage->engine->name == "TinyLog" || + create.storage->engine->name == "URL") { if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) setNullTableEngine(*create.storage); From 808acea1b6677c1b6bdbc28ddeb01c28e95e43d8 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 19:45:39 +0800 Subject: [PATCH 0169/1170] 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 0170/1170] 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 7ea3324776bd4cb8cc886822a9b30d3dfcaff5a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 16:31:34 +0000 Subject: [PATCH 0171/1170] Refactor in VirtualColumnUtils --- src/Interpreters/ActionsDAG.cpp | 12 ++++----- src/Interpreters/ActionsDAG.h | 3 +-- .../useDataParallelAggregation.cpp | 8 +++--- src/Processors/QueryPlan/SortingStep.cpp | 10 +++++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +-- .../StorageObjectStorageSource.cpp | 15 ++++++++--- .../StorageObjectStorageSource.h | 2 +- src/Storages/StorageFile.cpp | 8 ++++-- src/Storages/StorageURL.cpp | 6 +++-- .../System/StorageSystemDetachedParts.cpp | 4 +-- .../StorageSystemDroppedTablesParts.cpp | 4 +-- .../System/StorageSystemDroppedTablesParts.h | 6 ++--- .../System/StorageSystemPartsBase.cpp | 12 ++++----- src/Storages/System/StorageSystemPartsBase.h | 6 ++--- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 27 +++++++++++-------- src/Storages/VirtualColumnUtils.h | 13 ++++----- 18 files changed, 87 insertions(+), 59 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4f03a9e1602..e001406408f 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -624,9 +624,9 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) } } -ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) +ActionsDAG ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { - auto actions = std::make_unique(); + ActionsDAG actions; std::unordered_map copy_map; struct Frame @@ -661,21 +661,21 @@ ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool rem if (remove_aliases && frame.node->type == ActionType::ALIAS) copy_node = copy_map[frame.node->children.front()]; else - copy_node = &actions->nodes.emplace_back(*frame.node); + copy_node = &actions.nodes.emplace_back(*frame.node); if (frame.node->type == ActionType::INPUT) - actions->inputs.push_back(copy_node); + actions.inputs.push_back(copy_node); stack.pop(); } } - for (auto & node : actions->nodes) + for (auto & node : actions.nodes) for (auto & child : node.children) child = copy_map[child]; for (const auto * output : outputs) - actions->outputs.push_back(copy_map[output]); + actions.outputs.push_back(copy_map[output]); return actions; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 05948ccf928..6f5c3d3b0df 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,13 +261,12 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr clone(const ActionsDAGPtr & from) { return clone(from.get()); } static ActionsDAGPtr clone(const ActionsDAG * from); ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; ActionsDAG clone() const; - static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); + static ActionsDAG cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases); /// Execute actions for header. Input block must have empty columns. /// Result should be equal to the execution of ExpressionActions built from this DAG. diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 7e0260c0040..0eeaec9bde7 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -74,11 +74,11 @@ void removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG::Node * nod /// Our objective is to replace injective function nodes in `actions` results with its children /// until only the irreducible subset of nodes remains. Against these set of nodes we will match partition key expression /// to determine if it maps all rows with the same value of group by key to the same partition. -NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAGPtr & actions) +NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG & actions) { NodeSet irreducible; NodeSet visited; - for (const auto & node : actions->getOutputs()) + for (const auto & node : actions.getOutputs()) removeInjectiveFunctionsFromResultsRecursively(node, irreducible, visited); return irreducible; } @@ -158,7 +158,7 @@ bool isPartitionKeySuitsGroupByKey( auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); - const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); + const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames(); const auto & partition_actions = reading.getStorageMetadata()->getPartitionKey().expression->getActionsDAG(); @@ -169,7 +169,7 @@ bool isPartitionKeySuitsGroupByKey( const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_key_actions); - const auto matches = matchTrees(group_by_key_actions->getOutputs(), partition_actions); + const auto matches = matchTrees(group_by_key_actions.getOutputs(), partition_actions); return allOutputsDependsOnlyOnAllowedNodes(partition_actions, irreducibe_nodes, matches); } diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 48fad9f5fdb..e8e761e7ab0 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -447,6 +447,13 @@ void SortingStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } + if (!partition_by_description.empty()) + { + settings.out << prefix << "Partition by description: "; + dumpSortDescription(partition_by_description, settings.out); + settings.out << '\n'; + } + if (limit) settings.out << prefix << "Limit " << limit << '\n'; } @@ -461,6 +468,9 @@ void SortingStep::describeActions(JSONBuilder::JSONMap & map) const else map.add("Sort Description", explainSortDescription(result_description)); + if (!partition_by_description.empty()) + map.add("Partition By Description", explainSortDescription(partition_by_description)); + if (limit) map.add("Limit", limit); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 878e0420665..9aa9490198a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1154,7 +1154,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (!virtual_columns_block.has(input->result_name)) valid = false; - PartitionPruner partition_pruner(metadata_snapshot, filter_dag.get(), local_context, true /* strict */); + PartitionPruner partition_pruner(metadata_snapshot, &*filter_dag, local_context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; @@ -1162,7 +1162,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (valid) { virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_dag), virtual_columns_block, local_context); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a6a40a808e5..a37dbfa554c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -465,7 +465,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset( return; part_offset_condition.emplace(KeyCondition{ - dag.get(), + &*dag, context, sample.getNames(), std::make_shared(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), @@ -488,7 +488,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar return {}; auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(dag, virtual_columns_block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), virtual_columns_block, context); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index c86b56d3f1b..e760098f10f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -139,7 +139,10 @@ std::shared_ptr StorageObjectStorageSourc paths.reserve(keys.size()); for (const auto & key : keys) paths.push_back(fs::path(configuration->getNamespace()) / key); - VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); + + VirtualColumnUtils::buildSetsForDAG(*filter_dag, local_context); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(keys, paths, actions, virtual_columns); copy_configuration->setPaths(keys); } @@ -506,7 +509,11 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } recursive = key_with_globs == "/**"; - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns)) + { + VirtualColumnUtils::buildSetsForDAG(*filter_dag, getContext()); + filter_expr = std::make_shared(std::move(*filter_dag)); + } } else { @@ -570,14 +577,14 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne ++it; } - if (filter_dag) + if (filter_expr) { std::vector paths; paths.reserve(new_batch.size()); for (const auto & object_info : new_batch) paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_expr, virtual_columns); LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index b8418ddd07c..e466621e1e1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -208,7 +208,7 @@ private: ObjectInfos object_infos; ObjectInfos * read_keys; - ActionsDAGPtr filter_dag; + ExpressionActionsPtr filter_expr; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; std::vector expanded_keys; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c6acb358d89..fe6f494db00 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1130,12 +1130,16 @@ StorageFileSource::FilesIterator::FilesIterator( bool distributed_processing_) : WithContext(context_), files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_) { - ActionsDAGPtr filter_dag; + std::optional filter_dag; if (!distributed_processing && !archive_info && !files.empty()) filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); if (filter_dag) - VirtualColumnUtils::filterByPathOrFile(files, files, filter_dag, virtual_columns, context_); + { + VirtualColumnUtils::buildSetsForDAG(*filter_dag, context_); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(files, files, actions, virtual_columns); + } } String StorageFileSource::FilesIterator::next() diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9cec8c75ebe..c61bb8ac980 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -198,7 +198,7 @@ public: { uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses); - ActionsDAGPtr filter_dag; + std::optional filter_dag; if (!uris.empty()) filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); @@ -209,7 +209,9 @@ public: for (const auto & uri : uris) paths.push_back(Poco::URI(uri).getPath()); - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); + VirtualColumnUtils::buildSetsForDAG(*filter_dag, context); + auto actions = std::make_shared(std::move(*filter_dag)); + VirtualColumnUtils::filterByPathOrFile(uris, paths, actions, virtual_columns); } } diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 7e4c1de1c65..0d0ae666c10 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -307,7 +307,7 @@ protected: std::shared_ptr storage; std::vector columns_mask; - ActionsDAGPtr filter; + std::optional filter; const size_t max_block_size; const size_t num_streams; }; @@ -359,7 +359,7 @@ void StorageSystemDetachedParts::read( void ReadFromSystemDetachedParts::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto state = std::make_shared(StoragesInfoStream(nullptr, filter, context)); + auto state = std::make_shared(StoragesInfoStream({}, std::move(filter), context)); Pipe pipe; diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index c17d6402d88..defc4ec2d2a 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -11,7 +11,7 @@ namespace DB { -StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context) +StoragesDroppedInfoStream::StoragesDroppedInfoStream(std::optional filter, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -75,7 +75,7 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAGPtr & filte { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter) - VirtualColumnUtils::filterBlockWithDAG(filter, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter), block_to_filter, context); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.h b/src/Storages/System/StorageSystemDroppedTablesParts.h index dff9e41cce3..32468fc31b2 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.h +++ b/src/Storages/System/StorageSystemDroppedTablesParts.h @@ -9,7 +9,7 @@ namespace DB class StoragesDroppedInfoStream : public StoragesInfoStreamBase { public: - StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context); + StoragesDroppedInfoStream(std::optional filter, ContextPtr context); protected: bool tryLockTable(StoragesInfo &) override { @@ -30,9 +30,9 @@ public: std::string getName() const override { return "SystemDroppedTablesParts"; } protected: - std::unique_ptr getStoragesInfoStream(const ActionsDAGPtr &, const ActionsDAGPtr & filter, ContextPtr context) override + std::unique_ptr getStoragesInfoStream(std::optional, std::optional filter, ContextPtr context) override { - return std::make_unique(filter, context); + return std::make_unique(std::move(filter), context); } }; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index f7d1c1b3eb8..a0c9a5c61bd 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -91,7 +91,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state); } -StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context) +StoragesInfoStream::StoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context) : StoragesInfoStreamBase(context) { /// Will apply WHERE to subset of columns and then add more columns. @@ -124,7 +124,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, /// Filter block_to_filter with column 'database'. if (filter_by_database) - VirtualColumnUtils::filterBlockWithDAG(filter_by_database, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_database), block_to_filter, context); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -204,7 +204,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter_by_other_columns) - VirtualColumnUtils::filterBlockWithDAG(filter_by_other_columns, block_to_filter, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_other_columns), block_to_filter, context); rows = block_to_filter.rows(); } @@ -236,8 +236,8 @@ protected: std::shared_ptr storage; std::vector columns_mask; const bool has_state_column; - ActionsDAGPtr filter_by_database; - ActionsDAGPtr filter_by_other_columns; + std::optional filter_by_database; + std::optional filter_by_other_columns; }; ReadFromSystemPartsBase::ReadFromSystemPartsBase( @@ -318,7 +318,7 @@ void StorageSystemPartsBase::read( void ReadFromSystemPartsBase::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto stream = storage->getStoragesInfoStream(filter_by_database, filter_by_other_columns, context); + auto stream = storage->getStoragesInfoStream(std::move(filter_by_database), std::move(filter_by_other_columns), context); auto header = getOutputStream().header; MutableColumns res_columns = header.cloneEmptyColumns(); diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 8671fd850f8..806af4a7bf8 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -116,7 +116,7 @@ protected: class StoragesInfoStream : public StoragesInfoStreamBase { public: - StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context); + StoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context); }; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. @@ -146,9 +146,9 @@ protected: StorageSystemPartsBase(const StorageID & table_id_, ColumnsDescription && columns); - virtual std::unique_ptr getStoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context) + virtual std::unique_ptr getStoragesInfoStream(std::optional filter_by_database, std::optional filter_by_other_columns, ContextPtr context) { - return std::make_unique(filter_by_database, filter_by_other_columns, context); + return std::make_unique(std::move(filter_by_database), std::move(filter_by_other_columns), context); } virtual void diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..85aaf4ad186 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -169,7 +169,7 @@ ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); return block.getByPosition(0).column; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1630d9fd9c4..32c6a558340 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,15 +77,20 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context) } } -void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context) +void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context) +{ + buildSetsForDAG(dag, context); + auto actions = std::make_shared(std::move(dag)); + filterBlockWithExpression(actions, block); +} + +void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block) { - buildSetsForDAG(*dag, context); - auto actions = std::make_shared(std::move(*ActionsDAG::clone(dag))); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); /// Filter the block. - String filter_column_name = dag->getOutputs().at(0)->result_name; + String filter_column_name = actions->getActionsDAG().getOutputs().at(0)->result_name; ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst(); ConstantFilterDescription constant_filter(*filter_column); @@ -155,7 +160,7 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s block.getByName("_idx").column->assumeMutableRef().insert(idx); } -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) { if (!predicate || virtual_columns.empty()) return {}; @@ -171,7 +176,7 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con return splitFilterDagForAllowedInputs(predicate, &block); } -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) { Block block; for (const auto & column : virtual_columns) @@ -184,7 +189,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const for (size_t i = 0; i != paths.size(); ++i) addPathAndFileToVirtualColumns(block, paths[i], i); - filterBlockWithDAG(dag, block, context); + filterBlockWithExpression(actions, block); return block.getByName("_idx").column; } @@ -355,15 +360,15 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) { if (!predicate) - return nullptr; + return {}; ActionsDAG::Nodes additional_nodes; const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes); if (!res) - return nullptr; + return {}; return ActionsDAG::cloneSubDAG({res}, true); } @@ -372,7 +377,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, { auto dag = splitFilterDagForAllowedInputs(predicate, &block); if (dag) - filterBlockWithDAG(dag, block, context); + filterBlockWithDAG(std::move(*dag), block, context); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 208aa7a8100..72c45964ff4 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,8 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(const ActionsDAGPtr & dag, Block & block, ContextPtr context); +void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context); +void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block); /// Builds sets used by ActionsDAG inplace. void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); @@ -32,7 +33,7 @@ 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. -ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); /// Extract from the input stream a set of `name` column values template @@ -49,14 +50,14 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns); -ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context); +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns); template -void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) { - auto indexes_column = getFilterByPathAndFileIndexes(paths, dag, virtual_columns, context); + auto indexes_column = getFilterByPathAndFileIndexes(paths, actions, virtual_columns); const auto & indexes = typeid_cast(*indexes_column).getData(); if (indexes.size() == sources.size()) return; From 0954eefb076d36ec5804b46e594005cd7f4030bf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 17:01:35 +0000 Subject: [PATCH 0172/1170] Revert SortingStep changes. --- src/Processors/QueryPlan/SortingStep.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index e8e761e7ab0..48fad9f5fdb 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -447,13 +447,6 @@ void SortingStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } - if (!partition_by_description.empty()) - { - settings.out << prefix << "Partition by description: "; - dumpSortDescription(partition_by_description, settings.out); - settings.out << '\n'; - } - if (limit) settings.out << prefix << "Limit " << limit << '\n'; } @@ -468,9 +461,6 @@ void SortingStep::describeActions(JSONBuilder::JSONMap & map) const else map.add("Sort Description", explainSortDescription(result_description)); - if (!partition_by_description.empty()) - map.add("Partition By Description", explainSortDescription(partition_by_description)); - if (limit) map.add("Limit", limit); } From b6a790124cd670749b4c504f58a4854307bf7d83 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 20:16:47 +0000 Subject: [PATCH 0173/1170] Handling parallel replicas protocol with priority for async communication --- src/Processors/IProcessor.h | 2 + src/Processors/Sources/RemoteSource.cpp | 23 +++++++++++ src/Processors/Sources/RemoteSource.h | 3 ++ src/QueryPipeline/RemoteQueryExecutor.cpp | 38 +++++++++++++++++-- src/QueryPipeline/RemoteQueryExecutor.h | 4 +- .../RemoteQueryExecutorReadContext.h | 2 + 6 files changed, 67 insertions(+), 5 deletions(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 02f7b6b3d12..358983a2179 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,6 +221,8 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } + virtual void asyncJobReady() {} + /** You must call this method if 'prepare' returned ExpandPipeline. * This method cannot access any port, but it can create new ports for current processor. * diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..f1d47f69782 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -89,6 +89,12 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { + if (async_immediate_work.exchange(false)) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work was true"); + return; + } + /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. /// See issue: https://github.com/ClickHouse/ClickHouse/issues/60844 @@ -101,6 +107,23 @@ void RemoteSource::work() ISource::work(); } +void RemoteSource::asyncJobReady() +{ + chassert(async_read); + + if (!was_query_sent) + return; + + auto res = query_executor->readAsync(/*probe=*/true); + if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is {}", async_immediate_work); + work(); + async_immediate_work = true; + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is true"); + } +} + std::optional RemoteSource::tryGenerate() { /// onCancel() will do the cancel if the query was sent. diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 052567bc261..fa04985f101 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -32,6 +32,8 @@ public: int schedule() override { return fd; } + void asyncJobReady() override; + void setStorageLimits(const std::shared_ptr & storage_limits_) override; protected: @@ -52,6 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; + std::atomic_bool async_immediate_work{false}; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b08f2002f64..3ca05b53417 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet_type_only) { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) @@ -486,7 +486,21 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "was_cancelled"); return ReadResult(Block()); + } + + if (has_postponed_packet) + { + has_postponed_packet = false; + auto read_result = processPacket(read_context->getPacket()); + if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) + return read_result; + + if (got_duplicated_part_uuids) + break; + } read_context->resume(); @@ -506,12 +520,28 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() /// Check if packet is not ready yet. if (read_context->isInProgress()) + { + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "read_context still in progress"); return ReadResult(read_context->getFileDescriptor()); + } - auto anything = processPacket(read_context->getPacket()); + const auto packet_type = read_context->getPacketType(); + LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Packet type: {}", packet_type); - if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken) - return anything; + if (check_packet_type_only) + { + has_postponed_packet = true; + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest + || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + return ReadResult(ReadResult::Type::ParallelReplicasToken); + } + return ReadResult(ReadResult::Type::Nothing); + } + + auto read_result = processPacket(read_context->getPacket()); + if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) + return read_result; if (got_duplicated_part_uuids) break; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 04a59cc3b7e..6849c3e0a07 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -183,7 +183,7 @@ public: ReadResult read(); /// Async variant of read. Returns ready block or file descriptor which may be used for polling. - ReadResult readAsync(); + ReadResult readAsync(bool check_packet_type_only = false); /// Receive all remain packets and finish query. /// It should be cancelled after read returned empty block. @@ -303,6 +303,8 @@ private: */ bool got_duplicated_part_uuids = false; + bool has_postponed_packet = false; + /// Parts uuids, collected from remote replicas std::vector duplicated_part_uuids; diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index b8aa8bb9111..c054e75f6f1 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -39,6 +39,8 @@ public: Packet getPacket() { return std::move(packet); } + UInt64 getPacketType() const { return packet.type; } + private: bool checkTimeout(bool blocking = false); From a7310e51939ad6053d6ab94b07f0171457e5d779 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 19:32:27 +0200 Subject: [PATCH 0174/1170] Ignore async_load_databases for ATTACH query It is quite odd that when ATTACH finishes the tables may not be exists, due to async_load_databases. For server startup it makes total sense, but not for queries. Plus, you can execute queries in parallel if you want to make it faster. Note, that server startup does not uses this code, see loadMetadata.cpp. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterCreateQuery.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 84d7f0a587c..1d8d885b216 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -362,18 +362,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) TablesLoader loader{getContext()->getGlobalContext(), {{database_name, database}}, mode}; auto load_tasks = loader.loadTablesAsync(); auto startup_tasks = loader.startupTablesAsync(); - if (getContext()->getGlobalContext()->getServerSettings().async_load_databases) - { - scheduleLoad(load_tasks); - scheduleLoad(startup_tasks); - } - else - { - /// First prioritize, schedule and wait all the load table tasks - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), load_tasks); - /// Only then prioritize, schedule and wait all the startup tasks - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_tasks); - } + /// First prioritize, schedule and wait all the load table tasks + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), load_tasks); + /// Only then prioritize, schedule and wait all the startup tasks + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_tasks); } } catch (...) From f30d35ae2926948f1e6a268917113e757df4e2df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 19:34:18 +0200 Subject: [PATCH 0175/1170] Revert "Merge pull request #65571 from ClickHouse/fix-flaky-test-4" Reverts: https://github.com/ClickHouse/ClickHouse/pull/65571 This reverts commit da9a34ea46b504881ffe5aa605c933106862ba25, reversing changes made to cbdb9833f207d4b0e35ad09cf4757f5d5b506b77. Signed-off-by: Azat Khuzhin --- .../0_stateless/01254_dict_load_after_detach_attach.reference | 2 +- .../queries/0_stateless/01254_dict_load_after_detach_attach.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference index 9c2c59f6379..2f2d638a294 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference @@ -1,4 +1,4 @@ -NOT_LOADED +0 NOT_LOADED 0 LOADED 10 1 LOADED diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql index 11473c6ce32..ef9e940df8b 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql @@ -12,7 +12,7 @@ LAYOUT(FLAT()); DETACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; ATTACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; -SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict')::Nullable(String), 'NOT_LOADED'); +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SYSTEM RELOAD DICTIONARY dict; SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT dictGetUInt64('dict', 'val', toUInt64(0)); From df636100d3643031ee8d2102b737e992a6e44a40 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:00:53 +0200 Subject: [PATCH 0176/1170] add entry in documentation for use_same_password_for_base_backup --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index fc861e25e9f..248fdbc156f 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -85,6 +85,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `use_same_s3_credentials_for_base_backup`: whether base backup to S3 should inherit credentials from the query. Only works with `S3`. + - `use_same_password_for_base_backup`: whether base backup archive should inherit the password from the query. - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` From d8c68a27744dca3f68e4de08036148c2ecfd2ed6 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:00:53 +0200 Subject: [PATCH 0177/1170] working incremental password protected backups --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupImpl.cpp | 13 +++++++++++-- src/Backups/BackupImpl.h | 7 +++++-- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 +++ src/Backups/BackupsWorker.cpp | 2 ++ src/Backups/RestoreSettings.cpp | 1 + src/Backups/RestoreSettings.h | 3 +++ .../registerBackupEngineAzureBlobStorage.cpp | 6 ++++-- src/Backups/registerBackupEngineS3.cpp | 6 ++++-- src/Backups/registerBackupEnginesFileAndDisk.cpp | 6 ++++-- 11 files changed, 39 insertions(+), 10 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index e13a9a12ca2..807b8516d49 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -41,6 +41,7 @@ public: bool allow_s3_native_copy = true; bool allow_azure_native_copy = true; bool use_same_s3_credentials_for_base_backup = false; + bool use_same_password_for_base_backup = false; bool azure_attempt_to_create_container = true; ReadSettings read_settings; WriteSettings write_settings; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 3f972c36e47..23f067a62f5 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -92,7 +92,8 @@ BackupImpl::BackupImpl( std::shared_ptr reader_, const ContextPtr & context_, bool is_internal_backup_, - bool use_same_s3_credentials_for_base_backup_) + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_) : backup_info(backup_info_) , backup_name_for_logging(backup_info.toStringForLogging()) , use_archive(!archive_params_.archive_name.empty()) @@ -104,6 +105,7 @@ BackupImpl::BackupImpl( , version(INITIAL_BACKUP_VERSION) , base_backup_info(base_backup_info_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) + , use_same_password_for_base_backup(use_same_password_for_base_backup_) , log(getLogger("BackupImpl")) { open(); @@ -120,7 +122,8 @@ BackupImpl::BackupImpl( const std::shared_ptr & coordination_, const std::optional & backup_uuid_, bool deduplicate_files_, - bool use_same_s3_credentials_for_base_backup_) + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_) : backup_info(backup_info_) , backup_name_for_logging(backup_info.toStringForLogging()) , use_archive(!archive_params_.archive_name.empty()) @@ -135,6 +138,7 @@ BackupImpl::BackupImpl( , base_backup_info(base_backup_info_) , deduplicate_files(deduplicate_files_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) + , use_same_password_for_base_backup(use_same_password_for_base_backup_) , log(getLogger("BackupImpl")) { open(); @@ -258,6 +262,11 @@ std::shared_ptr BackupImpl::getBaseBackupUnlocked() const params.is_internal_backup = is_internal_backup; /// use_same_s3_credentials_for_base_backup should be inherited for base backups params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup; + /// use_same_password_for_base_backup should be inherited for base backups + params.use_same_password_for_base_backup = use_same_password_for_base_backup; + + if (params.use_same_password_for_base_backup) + params.password = archive_params.password; base_backup = BackupFactory::instance().createBackup(params); diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 2b27e2ab090..d7846104c4c 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -41,7 +41,8 @@ public: std::shared_ptr reader_, const ContextPtr & context_, bool is_internal_backup_, - bool use_same_s3_credentials_for_base_backup_); + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_); BackupImpl( const BackupInfo & backup_info_, @@ -53,7 +54,8 @@ public: const std::shared_ptr & coordination_, const std::optional & backup_uuid_, bool deduplicate_files_, - bool use_same_s3_credentials_for_base_backup_); + bool use_same_s3_credentials_for_base_backup_, + bool use_same_password_for_base_backup_); ~BackupImpl() override; @@ -153,6 +155,7 @@ private: bool writing_finalized = false; bool deduplicate_files = true; bool use_same_s3_credentials_for_base_backup = false; + bool use_same_password_for_base_backup = false; const LoggerPtr log; }; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index e33880f88e3..37ddd344001 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes M(Bool, allow_s3_native_copy) \ M(Bool, allow_azure_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ + M(Bool, use_same_password_for_base_backup) \ M(Bool, azure_attempt_to_create_container) \ M(Bool, read_from_filesystem_cache) \ M(UInt64, shard_num) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index a6c4d5d7181..d8f48f6e1ac 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -50,6 +50,9 @@ struct BackupSettings /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; + /// Wheter base backup archive should be unlocked using the same password as the incremental archive + bool use_same_password_for_base_backup = false; + /// Whether a new Azure container should be created if it does not exist (requires permissions at storage account level) bool azure_attempt_to_create_container = true; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 15a7d7c1eca..0614fb2da01 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -601,6 +601,7 @@ void BackupsWorker::doBackup( backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; backup_create_params.allow_azure_native_copy = backup_settings.allow_azure_native_copy; backup_create_params.use_same_s3_credentials_for_base_backup = backup_settings.use_same_s3_credentials_for_base_backup; + backup_create_params.use_same_password_for_base_backup = backup_settings.use_same_password_for_base_backup; backup_create_params.azure_attempt_to_create_container = backup_settings.azure_attempt_to_create_container; backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings); backup_create_params.write_settings = getWriteSettingsForBackup(context); @@ -912,6 +913,7 @@ void BackupsWorker::doRestore( backup_open_params.password = restore_settings.password; backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; backup_open_params.use_same_s3_credentials_for_base_backup = restore_settings.use_same_s3_credentials_for_base_backup; + backup_open_params.use_same_password_for_base_backup = restore_settings.use_same_password_for_base_backup; backup_open_params.read_settings = getReadSettingsForRestore(context); backup_open_params.write_settings = getWriteSettingsForRestore(context); backup_open_params.is_internal_backup = restore_settings.internal; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 7bbfd9ed751..a974fc11d00 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -164,6 +164,7 @@ namespace M(RestoreUDFCreationMode, create_function) \ M(Bool, allow_s3_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ + M(Bool, use_same_password_for_base_backup) \ M(Bool, restore_broken_parts_as_detached) \ M(Bool, internal) \ M(String, host_id) \ diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 06ecbc80aef..0fe5ee1a4bf 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -113,6 +113,9 @@ struct RestoreSettings /// Whether base backup from S3 should inherit credentials from the RESTORE query. bool use_same_s3_credentials_for_base_backup = false; + /// Wheter base backup archive should be unlocked using the same password as the incremental archive + bool use_same_password_for_base_backup = false; + /// If it's true RESTORE won't stop on broken parts while restoring, instead they will be restored as detached parts /// to the `detached` folder with names starting with `broken-from-backup'. bool restore_broken_parts_as_detached = false; diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 626df99b00c..45f0386375a 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -141,7 +141,8 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) reader, params.context, params.is_internal_backup, - /* use_same_s3_credentials_for_base_backup*/ false); + /* use_same_s3_credentials_for_base_backup*/ false, + params.use_same_password_for_base_backup); } else { @@ -164,7 +165,8 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - /* use_same_s3_credentials_for_base_backup */ false); + /* use_same_s3_credentials_for_base_backup */ false, + params.use_same_password_for_base_backup); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "AzureBlobStorage support is disabled"); diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 59ed9506af0..79e3e945557 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -120,7 +120,8 @@ void registerBackupEngineS3(BackupFactory & factory) reader, params.context, params.is_internal_backup, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } else { @@ -144,7 +145,8 @@ void registerBackupEngineS3(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "S3 support is disabled"); diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 35263d39cba..c486f79a77a 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -178,7 +178,8 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) reader, params.context, params.is_internal_backup, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } else { @@ -197,7 +198,8 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) params.backup_coordination, params.backup_uuid, params.deduplicate_files, - params.use_same_s3_credentials_for_base_backup); + params.use_same_s3_credentials_for_base_backup, + params.use_same_password_for_base_backup); } }; From 74de7833b8d429edcac9c4735143fc92349abdaf Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 12:34:53 +0200 Subject: [PATCH 0178/1170] fix typo --- src/Backups/BackupSettings.h | 2 +- src/Backups/RestoreSettings.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index d8f48f6e1ac..0abeb897db4 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -50,7 +50,7 @@ struct BackupSettings /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; - /// Wheter base backup archive should be unlocked using the same password as the incremental archive + /// Whether base backup archive should be unlocked using the same password as the incremental archive bool use_same_password_for_base_backup = false; /// Whether a new Azure container should be created if it does not exist (requires permissions at storage account level) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 0fe5ee1a4bf..fe07a0a7208 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -113,7 +113,7 @@ struct RestoreSettings /// Whether base backup from S3 should inherit credentials from the RESTORE query. bool use_same_s3_credentials_for_base_backup = false; - /// Wheter base backup archive should be unlocked using the same password as the incremental archive + /// Whether base backup archive should be unlocked using the same password as the incremental archive bool use_same_password_for_base_backup = false; /// If it's true RESTORE won't stop on broken parts while restoring, instead they will be restored as detached parts From 12fb08648670dcf39659596ccb552f3462004fd8 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Mon, 8 Jul 2024 16:25:07 +0200 Subject: [PATCH 0179/1170] added test for use_same_password_for_base_backup --- ...se_same_password_for_base_backup.reference | 21 ++++++++ ...ackup_use_same_password_for_base_backup.sh | 50 +++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference create mode 100755 tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference new file mode 100644 index 00000000000..7354d50a7c0 --- /dev/null +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -0,0 +1,21 @@ +use_same_password_for_base_backup +base +BACKUP_CREATED +add_more_data_1 +inc_1 +BACKUP_CREATED +add_more_data_2 +inc_2 +BACKUP_CREATED +inc_2_bad +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +restore_inc_1 +RESTORED +restore_inc_2 +RESTORED +restore_inc_2_bad +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +count_inc_1 +20 +count_inc_2 +30 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh new file mode 100755 index 00000000000..a2b1a953e24 --- /dev/null +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS data; + DROP TABLE IF EXISTS data_1; + DROP TABLE IF EXISTS data_2; + CREATE TABLE data (key Int) ENGINE=MergeTree() ORDER BY tuple(); + INSERT INTO data SELECT * from numbers(10); +" + +echo 'use_same_password_for_base_backup' +echo "base" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 + +echo 'add_more_data_1' +$CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" + +echo "inc_1" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo 'add_more_data_2' +$CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" + +echo "inc_2" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "inc_2_bad" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" + +echo "restore_inc_1" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "restore_inc_2" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 + +echo "restore_inc_2_bad" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" + +echo "count_inc_1" +$CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 + +echo "count_inc_2" +$CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_2" | cut -f2 + +exit 0 From 6f25aacd71948b682630889e75774eb4f2469aaa Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Tue, 9 Jul 2024 16:36:59 +0200 Subject: [PATCH 0180/1170] use CLICKHOUSE_TEST_NAME instead of CLICKHOUSE_TEST_UNIQUE_NAME for backup filename --- ...kup_use_same_password_for_base_backup.reference | 4 ++-- ...843_backup_use_same_password_for_base_backup.sh | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..1a331cca46b 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index a2b1a953e24..4c5bec3775c 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -15,31 +15,31 @@ $CLICKHOUSE_CLIENT -nm -q " echo 'use_same_password_for_base_backup' echo "base" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip') SETTINGS password='password';" | cut -f2 echo 'add_more_data_1' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" echo "inc_1" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo 'add_more_data_2' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" echo "inc_2" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From cdd955f421e442cdb776e95e8e7ae8608bc8636e Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 09:20:39 +0200 Subject: [PATCH 0181/1170] Revert "use CLICKHOUSE_TEST_NAME instead of CLICKHOUSE_TEST_UNIQUE_NAME for backup filename" This reverts commit 72f6368a2ede6a01390db770f1b9ddfa00d3f1fe. --- ...kup_use_same_password_for_base_backup.reference | 4 ++-- ...843_backup_use_same_password_for_base_backup.sh | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 1a331cca46b..7354d50a7c0 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index 4c5bec3775c..a2b1a953e24 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -15,31 +15,31 @@ $CLICKHOUSE_CLIENT -nm -q " echo 'use_same_password_for_base_backup' echo "base" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip') SETTINGS password='password';" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip') SETTINGS password='password';" | cut -f2 echo 'add_more_data_1' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(10,10);" echo "inc_1" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_base.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo 'add_more_data_2' $CLICKHOUSE_CLIENT -q "INSERT INTO data SELECT * FROM numbers(20,10);" echo "inc_2" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From ba1c6fe3ef3a7f263939b2ae9e6c249d9c6b38c0 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 09:23:19 +0200 Subject: [PATCH 0182/1170] shorter grep output --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- .../02843_backup_use_same_password_for_base_backup.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..cbcb6b4cb7c 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index a2b1a953e24..f2f1265c1a0 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -30,7 +30,7 @@ echo "inc_2" $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 @@ -39,7 +39,7 @@ echo "restore_inc_2" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From 01c8faf190462d508d9cb1b7d342d6ccfbeeff88 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 11:15:33 +0200 Subject: [PATCH 0183/1170] Revert "shorter grep output" This reverts commit b0cbf1495dd0bcaba828706e895347e4ec550e29. --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- .../02843_backup_use_same_password_for_base_backup.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index cbcb6b4cb7c..7354d50a7c0 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh index f2f1265c1a0..a2b1a953e24 100755 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.sh @@ -30,7 +30,7 @@ echo "inc_2" $CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password',use_same_password_for_base_backup=1" | cut -f2 echo "inc_2_bad" -$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "BACKUP TABLE data TO Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2_bad.zip') SETTINGS base_backup=Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip'),password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "restore_inc_1" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_1 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 @@ -39,7 +39,7 @@ echo "restore_inc_2" $CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password',use_same_password_for_base_backup=1" | cut -f2 echo "restore_inc_2_bad" -$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" +$CLICKHOUSE_CLIENT -q "RESTORE TABLE data AS data_2 FROM Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_2.zip') SETTINGS password='password'" |& grep -m1 -o "Couldn't unpack zip archive '${CLICKHOUSE_TEST_UNIQUE_NAME}_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE)" echo "count_inc_1" $CLICKHOUSE_CLIENT -q "SELECT COUNT(*) FROM data_1" | cut -f2 From 9dae370569054b4933b374c12dee6461c938fea8 Mon Sep 17 00:00:00 2001 From: Samuele Guerrini Date: Wed, 10 Jul 2024 11:23:36 +0200 Subject: [PATCH 0184/1170] fix db name in backup_use_same_password_for_base_backup test reference file --- .../02843_backup_use_same_password_for_base_backup.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference index 7354d50a7c0..cdcf0532cd9 100644 --- a/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference +++ b/tests/queries/0_stateless/02843_backup_use_same_password_for_base_backup.reference @@ -8,13 +8,13 @@ add_more_data_2 inc_2 BACKUP_CREATED inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_default_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) restore_inc_1 RESTORED restore_inc_2 RESTORED restore_inc_2_bad -Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_test_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) +Couldn't unpack zip archive '02843_backup_use_same_password_for_base_backup_default_inc_1.zip': Password is required. (CANNOT_UNPACK_ARCHIVE) count_inc_1 20 count_inc_2 From f29700e04d3ca4d455908ed354472945597da4f5 Mon Sep 17 00:00:00 2001 From: morning-color Date: Wed, 17 Jul 2024 20:01:01 +0800 Subject: [PATCH 0185/1170] 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 523e0abb4ec329c0535602c43c17991f4ef043a3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Jul 2024 13:15:14 +0000 Subject: [PATCH 0186/1170] Remove debug logs --- src/Processors/Sources/RemoteSource.cpp | 7 +------ src/QueryPipeline/RemoteQueryExecutor.cpp | 10 +--------- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index f1d47f69782..e33613564a2 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -90,10 +90,7 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { if (async_immediate_work.exchange(false)) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work was true"); return; - } /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. @@ -114,13 +111,11 @@ void RemoteSource::asyncJobReady() if (!was_query_sent) return; - auto res = query_executor->readAsync(/*probe=*/true); + auto res = query_executor->readAsync(/*check_packet_type_only=*/true); if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is {}", async_immediate_work); work(); async_immediate_work = true; - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "async_immediate_work is true"); } } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 3ca05b53417..87f634b8334 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -486,10 +486,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "was_cancelled"); return ReadResult(Block()); - } if (has_postponed_packet) { @@ -520,17 +517,12 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet /// Check if packet is not ready yet. if (read_context->isInProgress()) - { - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "read_context still in progress"); return ReadResult(read_context->getFileDescriptor()); - } - - const auto packet_type = read_context->getPacketType(); - LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "Packet type: {}", packet_type); if (check_packet_type_only) { has_postponed_packet = true; + const auto packet_type = read_context->getPacketType(); if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) { From 2dbd04c8a77bf262f0965ddef9a2c166c22fcf55 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 17 Jul 2024 17:01:22 +0200 Subject: [PATCH 0187/1170] add toIntXYZ documentation --- .../functions/type-conversion-functions.md | 1275 ++++++++++++++++- 1 file changed, 1239 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 61e84ca72d1..057083d317f 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -49,105 +49,1308 @@ SETTINGS cast_keep_nullable = 1 └──────────────────┴─────────────────────┴──────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256) +## toInt8 -Converts an input value to a value the [Int](../data-types/int-uint.md) data type. This function family includes: +Converts an input value to a value of type `Int8`. -- `toInt8(expr)` — Converts to a value of data type `Int8`. -- `toInt16(expr)` — Converts to a value of data type `Int16`. -- `toInt32(expr)` — Converts to a value of data type `Int32`. -- `toInt64(expr)` — Converts to a value of data type `Int64`. -- `toInt128(expr)` — Converts to a value of data type `Int128`. -- `toInt256(expr)` — Converts to a value of data type `Int256`. +**Syntax** + +```sql +toInt8(expr) +``` **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions) returning a number or a string with the decimal representation of a number. Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: **Returned value** -Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` data type. +- 8-bit integer value. [Int8](../data-types/int-uint.md). -Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: -The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt8(-8), + toInt8(-8.8), + toInt8('-8'); +``` + +Result: + +```response + ┌─toInt8(-8)─┬─toInt8(-8.8)─┬─toInt8('-8')─┐ +1. │ -8 │ -8 │ -8 │ + └────────────┴──────────────┴──────────────┘ +``` + +**See also** + +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrNull`](#toint8ornull). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrZero + +Like [`toInt8`](#toint8), it takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt8OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise `0`. [Int8](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); +SELECT + toInt8OrZero('-8'), + toInt8OrZero('abc'); ``` Result: ```response -┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ -│ -9223372036854775808 │ 32 │ 16 │ 8 │ -└──────────────────────┴─────────────┴───────────────┴─────────────┘ + ┌─toInt8OrZero('-8')─┬─toInt8OrZero('abc')─┐ +1. │ -8 │ 0 │ + └────────────────────┴─────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrZero +**See also** -Takes an argument of type [String](../data-types/string.md) and tries to parse it into an Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `0`. +- [`toInt8`](#toint8). +- [`toInt8OrNull`](#toint8ornull). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrNull + +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt8OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise `NULL`. [Int8](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrZero('123123'), toInt8OrZero('123qwe123'); +SELECT toInt8OrNull('-8'), toInt8OrNull('abc'); ``` Result: ```response -┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ -│ 123123 │ 0 │ -└─────────────────────────┴───────────────────────────┘ + ┌─toInt8OrNull('-8')─┬─toInt8OrNull('abc')─┐ +1. │ -8 │ ᴺᵁᴸᴸ │ + └────────────────────┴─────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrNull +**See also** -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `NULL`. +- [`toInt8`](#toint8). +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrDefault`](#toint8ordefault). + +## toInt8OrDefault + +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt8OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 8-bit integer value if successful, otherwise returns the default value. [Int8](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); +SELECT + toInt8OrDefault('-8', CAST('-1', 'Int8')), + toInt8OrDefault('abc', CAST('-1', 'Int8')); ``` Result: ```response -┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ -│ 123123 │ ᴺᵁᴸᴸ │ -└─────────────────────────┴───────────────────────────┘ + ┌─toInt8OrDefault('-8', CAST('-1', 'Int8'))─┬─toInt8OrDefault('abc', CAST('-1', 'Int8'))─┐ +1. │ -8 │ -1 │ + └───────────────────────────────────────────┴────────────────────────────────────────────┘ ``` -## toInt(8\|16\|32\|64\|128\|256)OrDefault +**See also** -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns the default type value. +- [`toInt8`](#toint8). +- [`toInt8OrZero`](#toint8orzero). +- [`toInt8OrNull`](#toint8orNull). + +## toInt16 + +Converts an input value to a value of type `Int16`. + +**Syntax** + +```sql +toInt16(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value. [Int16](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt16(-16), + toInt16(-16.16), + toInt16('-16'); +``` + +Result: + +```response + ┌─toInt16(-16)─┬─toInt16(-16.16)─┬─toInt16('-16')─┐ +1. │ -16 │ -16 │ -16 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrNull`](#toint16ornull). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrZero + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt16OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise `0`. [Int16](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: **Example** Query: ``` sql -SELECT toInt64OrDefault('123123', cast('-1' as Int64)), toInt8OrDefault('123qwe123', cast('-1' as Int8)); +SELECT + toInt16OrZero('-16'), + toInt16OrZero('abc'); ``` Result: ```response -┌─toInt64OrDefault('123123', CAST('-1', 'Int64'))─┬─toInt8OrDefault('123qwe123', CAST('-1', 'Int8'))─┐ -│ 123123 │ -1 │ -└─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ + ┌─toInt16OrZero('-16')─┬─toInt16OrZero('abc')─┐ +1. │ -16 │ 0 │ + └──────────────────────┴──────────────────────┘ ``` +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrNull`](#toint16ornull). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrNull + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt16OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise `NULL`. [Int16](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt16OrNull('-16'), + toInt16OrNull('abc'); +``` + +Result: + +```response + ┌─toInt16OrNull('-16')─┬─toInt16OrNull('abc')─┐ +1. │ -16 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrDefault`](#toint16ordefault). + +## toInt16OrDefault + +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt16OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 16-bit integer value if successful, otherwise returns the default value. [Int16](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt16OrDefault('-16', cast('-1' as Int16)), toInt16OrDefault('abc', cast('-1' as Int16)); +``` + +Result: + +```response + ┌─toInt16OrDefault('-16', CAST('-1', 'Int16'))─┬─toInt16OrDefault('abc', CAST('-1', 'Int16'))─┐ +1. │ -16 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt16`](#toint16). +- [`toInt16OrZero`](#toint16orzero). +- [`toInt16OrNull`](#toint16ornull). + +## toInt32 + +Converts an input value to a value of type `Int32`. + +**Syntax** + +```sql +toInt32(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value. [Int32](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt32(-32), + toInt32(-32.32), + toInt32('-32') +``` + +Result: + +```response + ┌─toInt32(-32)─┬─toInt32(-32.32)─┬─toInt32('-32')─┐ +1. │ -32 │ -32 │ -32 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrNull`](#toint32ornull). +- [`toInt32OrDefault`](#toint32ordefault). + +## toInt32OrZero + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt32OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise `0`. [Int32](../data-types/int-uint.md) + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrZero('-32'), toInt32OrZero('abc'); +``` + +Result: + +```response + ┌─toInt32OrZero('-32')─┬─toInt32OrZero('abc')─┐ +1. │ -32 │ 0 │ + └──────────────────────┴──────────────────────┘ +``` +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrNull`](#toint32ornull). +- [`toInt32OrDefault`](#toint32ordefault). +- +## toInt32OrNull + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt32OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise `NULL`. [Int32](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrNull('-32'), toInt32OrNull('abc'); +``` + +Result: + +```response + ┌─toInt32OrNull('-32')─┬─toInt32OrNull('abc')─┐ +1. │ -32 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrDefault`](#toint32ordefault). + +## toInt32OrDefault + +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt32OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 32-bit integer value if successful, otherwise returns the default value. [Int32](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. + ::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT toInt32OrDefault('-32', cast('-1' as Int32)), toInt32OrDefault('abc', cast('-1' as Int32)); +``` + +Result: + +```response + ┌─toInt32OrDefault('-32', CAST('-1', 'Int32'))─┬─toInt32OrDefault('abc', CAST('-1', 'Int32'))─┐ +1. │ -32 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt32`](#toint32). +- [`toInt32OrZero`](#toint32orzero). +- [`toInt32OrNull`](#toint32ornull). + +## toInt64 + +Converts an input value to a value of type `Int64`. + +**Syntax** + +```sql +toInt64(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt64(-64), + toInt64(-64.64), + toInt64('-64'); +``` + +Result: + +```response + ┌─toInt64(-64)─┬─toInt64(-64.64)─┬─toInt64('-64')─┐ +1. │ -64 │ -64 │ -64 │ + └──────────────┴─────────────────┴────────────────┘ +``` + +**See also** + +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrNull`](#toint64ornull). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrZero + +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt64OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 64-bit integer value if successful, otherwise `0`. [Int64](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrZero('-64'), + toInt64OrZero('abc'); +``` + +Result: + +```response + ┌─toInt64OrZero('-64')─┬─toInt64OrZero('abc')─┐ +1. │ -64 │ 0 │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrNull`](#toint64ornull). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrNull + +Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt64OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrNull('-64'), + toInt64OrNull('abc'); +``` + +Result: + +```response + ┌─toInt64OrNull('-64')─┬─toInt64OrNull('abc')─┐ +1. │ -64 │ ᴺᵁᴸᴸ │ + └──────────────────────┴──────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrDefault`](#toint64ordefault). + +## toInt64OrDefault + +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt64OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. + ::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt64OrDefault('-64', CAST('-1', 'Int64')), + toInt64OrDefault('abc', CAST('-1', 'Int64')); +``` + +Result: + +```response + ┌─toInt64OrDefault('-64', CAST('-1', 'Int64'))─┬─toInt64OrDefault('abc', CAST('-1', 'Int64'))─┐ +1. │ -64 │ -1 │ + └──────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt64`](#toint64). +- [`toInt64OrZero`](#toint64orzero). +- [`toInt64OrNull`](#toint64ornull). + +## toInt128 + +Converts an input value to a value of type `Int128`. + +**Syntax** + +```sql +toInt128(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value. [Int128](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt128(-128), + toInt128(-128.8), + toInt128('-128'), +``` + +Result: + +```response + ┌─toInt128(-128)─┬─toInt128(-128.8)─┬─toInt128('-128')─┐ +1. │ -128 │ -128 │ -128 │ + └────────────────┴──────────────────┴──────────────────┘ +``` + +**See also** + +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrNull`](#toint128ornull). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrZero + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt128OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise `0`. [Int128](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrZero('-128'), + toInt128OrZero('abc'); +``` + +Result: + +```response + ┌─toInt128OrZero('-128')─┬─toInt128OrZero('abc')─┐ +1. │ -128 │ 0 │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrNull`](#toint128ornull). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrNull + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt128OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise `NULL`. [Int128](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrNull('-128'), + toInt128OrNull('abc'); +``` + +Result: + +```response + ┌─toInt128OrNull('-128')─┬─toInt128OrNull('abc')─┐ +1. │ -128 │ ᴺᵁᴸᴸ │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrDefault`](#toint128ordefault). + +## toInt128OrDefault + +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt128OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 128-bit integer value if successful, otherwise returns the default value. [Int128](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt128OrDefault('-128', CAST('-1', 'Int128')), + toInt128OrDefault('abc', CAST('-1', 'Int128')); +``` + +Result: + +```response + ┌─toInt128OrDefault('-128', CAST('-1', 'Int128'))─┬─toInt128OrDefault('abc', CAST('-1', 'Int128'))─┐ +1. │ -128 │ -1 │ + └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt128`](#toint128). +- [`toInt128OrZero`](#toint128orzero). +- [`toInt128OrNull`](#toint128ornull). + +## toInt256 + +Converts an input value to a value of type `Int256`. + +**Syntax** + +```sql +toInt256(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value. [Int256](../data-types/int-uint.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +```sql +SELECT + toInt256(-256), + toInt256(-256.256), + toInt256('-256'); +``` + +Result: + +```response + ┌─toInt256(-256)─┬─toInt256(-256.256)─┬─toInt256('-256')─┐ +1. │ -256 │ -256 │ -256 │ + └────────────────┴────────────────────┴──────────────────┘ +``` + +**See also** + +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrNull`](#toint256ornull). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrZero + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `0`. + +**Syntax** + +```sql +toInt256OrZero(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise `0`. [Int256](../data-types/int-uint.md). + +:::note +Functions uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrZero('-256'), + toInt256OrZero('abc'); +``` + +Result: + +```response + ┌─toInt256OrZero('-256')─┬─toInt256OrZero('abc')─┐ +1. │ -256 │ 0 │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrNull`](#toint256ornull). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrNull + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `NULL`. + +**Syntax** + +```sql +toInt256OrNull(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise `NULL`. [Int256](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). + +:::note +Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrNull('-256'), + toInt256OrNull('abc'); +``` + +Result: + +```response + ┌─toInt256OrNull('-256')─┬─toInt256OrNull('abc')─┐ +1. │ -256 │ ᴺᵁᴸᴸ │ + └────────────────────────┴───────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrDefault`](#toint256ordefault). + +## toInt256OrDefault + +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns the default type value. + +**Syntax** + +```sql +toInt256OrDefault(expr, def) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). + +:::note +Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +::: + +**Returned value** + +- 256-bit integer value if successful, otherwise returns the default value. [Int256](../data-types/int-uint.md). + +:::note +- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The default value type should be the same as the cast type. +::: + +:::danger +An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +::: + +**Example** + +Query: + +``` sql +SELECT + toInt256OrDefault('-256', CAST('-1', 'Int256')), + toInt256OrDefault('abc', CAST('-1', 'Int256')); +``` + +Result: + +```response + ┌─toInt256OrDefault('-256', CAST('-1', 'Int256'))─┬─toInt256OrDefault('abc', CAST('-1', 'Int256'))─┐ +1. │ -256 │ -1 │ + └─────────────────────────────────────────────────┴────────────────────────────────────────────────┘ +``` + +**See also** + +- [`toInt256`](#toint256). +- [`toInt256OrZero`](#toint256orzero). +- [`toInt256OrNull`](#toint256ornull). ## toUInt(8\|16\|32\|64\|256) @@ -167,7 +1370,7 @@ Converts an input value to the [UInt](../data-types/int-uint.md) data type. This - Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type. -Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. +Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. @@ -2289,7 +3492,7 @@ Result: └─────────────────────┴─────────────────┴─────────────────────────────────────┘ ``` -**See Also** +**See also** - [RFC 1123](https://datatracker.ietf.org/doc/html/rfc1123) - [toDate](#todate) From 80e1377e5d2223176274c319938187f0da799280 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 17:30:28 +0000 Subject: [PATCH 0188/1170] Fixing build. --- src/Storages/System/StorageSystemTables.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 43b761d84b1..d6b577bf6c8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -125,7 +125,7 @@ ColumnPtr getFilteredTables( block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); return block.getByPosition(0).column; } From 47573f1ae158faabd6ac0a62598c907c85fbd4c1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 17 Jul 2024 20:44:44 +0000 Subject: [PATCH 0189/1170] add options --- src/Core/SettingsEnums.h | 9 +++++++++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + 2 files changed, 10 insertions(+) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index f6d9593ca56..ac3264fe041 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -315,6 +315,15 @@ enum class LightweightMutationProjectionMode : uint8_t DECLARE_SETTING_ENUM(LightweightMutationProjectionMode) +enum class DeduplicateMergeProjectionMode : uint8_t +{ + THROW, + DROP, + REBUILD, +}; + +DECLARE_SETTING_ENUM(DeduplicateMergeProjectionMode) + DECLARE_SETTING_ENUM(LocalFSReadMethod) enum class ObjectStorageQueueMode : uint8_t diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f5ada81cf55..166a18a7bab 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -215,6 +215,7 @@ struct Settings; M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ + M(DeduplicateMergeProjectionMode, deduplicate_merge_projection_mode, DeduplicateMergeProjectionMode::THROW, "Whether to allow create projection for the table with non-classic MergeTree, if allowed, what is the next action.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) From b8fbfd227fb60e0f244bda716ef5a9bb89376986 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 15:41:08 +0800 Subject: [PATCH 0190/1170] format --- src/Interpreters/InterpreterInsertQuery.cpp | 1 - .../test.py | 40 ++++++++----------- 2 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 4064cd82b67..181fb064b54 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -49,7 +49,6 @@ namespace ProfileEvents { extern const Event InsertQueriesWithSubqueries; extern const Event QueriesWithSubqueries; - extern const int QUERY_IS_PROHIBITED; } namespace DB diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index f098f130d2b..f25964d27b8 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -37,29 +37,25 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("INSERT INTO my_table VALUES (1, 'hello')") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "INSERT INTO my_table VALUES (1, 'hello')" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "INSERT INTO my_table SETTINGS async_insert = 1 VALUES (1, 'hello')" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER TABLE my_table delete where 1") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER TABLE my_table delete where 1" + ) + + + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER table my_table update key = 1 where 1" ) - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER table my_table update key = 1 where 1") - ) - - assert ( - "QUERY_IS_PROHIBITED" - in reading_node.query_and_get_error("ALTER TABLE my_table drop partition 0") + assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( + "ALTER TABLE my_table drop partition 0" ) reading_node.query("SELECT * from my_table"); @@ -73,12 +69,10 @@ def test_disable_insertion_and_mutation(started_cluster): reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") - assert( - "new_column\tString" - in reading_node.query("DESC my_table") + assert "new_column\tString" in reading_node.query( + "DESC my_table" ) - assert( - "new_column\tString" - in writing_node.query("DESC my_table") + assert "new_column\tString" in writing_node.query( + "DESC my_table" ) From 2e1f679ceb05afe4d5d813eb4048555c6311c3e1 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 16 Jul 2024 18:35:33 +0200 Subject: [PATCH 0191/1170] add S3DiskNoKeyErrors metric --- src/Common/CurrentMetrics.cpp | 2 ++ src/IO/S3/Client.cpp | 21 +++++++++++++++------ 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 7c97e73f278..2fedba0175b 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -306,6 +306,8 @@ \ M(FilteringMarksWithPrimaryKey, "Number of threads currently doing filtering of mark ranges by the primary key") \ M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \ + \ + M(S3DiskNoKeyErrors, "Number of no-key S3 disk errors") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 3b958dea046..db20420db9f 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -24,6 +24,7 @@ #include #include +#include #include #include @@ -43,6 +44,11 @@ namespace ProfileEvents extern const Event TinyS3Clients; } +namespace CurrentMetrics +{ + extern const Metric S3DiskNoKeyErrors; +} + namespace DB { @@ -379,10 +385,10 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - /// The next call is NOT a recurcive call - /// This is a virtuall call Aws::S3::S3Client::HeadObject(const Model::HeadObjectRequest&) - return enrichErrorMessage( - HeadObject(static_cast(request))); + if (isClientForDisk()) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + + return enrichErrorMessage(std::move(result)); } /// For each request, we wrap the request functions from Aws::S3::Client with doRequest @@ -402,8 +408,11 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { - return enrichErrorMessage( - doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); })); + auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); + if (!resp.IsSuccess() && isClientForDisk()) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + + return enrichErrorMessage(std::move(resp)); } Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const From 884dc496a0a978074d3e0bd70f4df8d0225e69c1 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 20:58:37 +0800 Subject: [PATCH 0192/1170] format --- .../test_disable_insertion_and_mutation/test.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index f25964d27b8..b6431690245 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -35,7 +35,9 @@ def started_cluster(): def test_disable_insertion_and_mutation(started_cluster): - writing_node.query("""CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """) + writing_node.query( + """CREATE TABLE my_table on cluster default (key UInt64, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{shard}/default.my_table', '{replica}') ORDER BY key partition by (key % 5) """ + ) assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "INSERT INTO my_table VALUES (1, 'hello')" @@ -58,7 +60,7 @@ def test_disable_insertion_and_mutation(started_cluster): "ALTER TABLE my_table drop partition 0" ) - reading_node.query("SELECT * from my_table"); + reading_node.query("SELECT * from my_table") writing_node.query("INSERT INTO my_table VALUES (1, 'hello')") writing_node.query("ALTER TABLE my_table delete where 1") writing_node.query("ALTER table my_table update value = 'no hello' where 1") @@ -69,10 +71,6 @@ def test_disable_insertion_and_mutation(started_cluster): reading_node.query("ALter Table my_table MODIFY COLUMN new_column String") - assert "new_column\tString" in reading_node.query( - "DESC my_table" - ) + assert "new_column\tString" in reading_node.query("DESC my_table") - assert "new_column\tString" in writing_node.query( - "DESC my_table" - ) + assert "new_column\tString" in writing_node.query("DESC my_table") From c01e2cbeea02ebecfc4dea4692baffff3087b043 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 18 Jul 2024 22:58:30 +0800 Subject: [PATCH 0193/1170] format --- tests/integration/test_disable_insertion_and_mutation/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_disable_insertion_and_mutation/test.py b/tests/integration/test_disable_insertion_and_mutation/test.py index b6431690245..b6da7ed548f 100644 --- a/tests/integration/test_disable_insertion_and_mutation/test.py +++ b/tests/integration/test_disable_insertion_and_mutation/test.py @@ -50,7 +50,6 @@ def test_disable_insertion_and_mutation(started_cluster): assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "ALTER TABLE my_table delete where 1" ) - assert "QUERY_IS_PROHIBITED" in reading_node.query_and_get_error( "ALTER table my_table update key = 1 where 1" From cda846339be22c66cd0d35d49273a314fa3bdf69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 15:12:32 +0000 Subject: [PATCH 0194/1170] Remove ActionsDAG::clone --- src/Interpreters/ActionsDAG.cpp | 8 ---- src/Interpreters/ActionsDAG.h | 3 -- src/Interpreters/ExpressionAnalyzer.cpp | 9 +---- src/Interpreters/InterpreterSelectQuery.cpp | 24 ++++++------ src/Interpreters/MutationsInterpreter.cpp | 4 +- src/Planner/Planner.cpp | 38 +++++++++---------- src/Planner/PlannerExpressionAnalysis.h | 16 ++++---- src/Planner/PlannerJoins.cpp | 4 +- src/Processors/QueryPlan/ExpressionStep.cpp | 4 +- src/Processors/QueryPlan/FilterStep.cpp | 4 +- .../Optimizations/distinctReadInOrder.cpp | 10 ++--- .../Optimizations/filterPushDown.cpp | 4 +- .../QueryPlan/Optimizations/liftUpUnion.cpp | 2 +- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +-- .../Optimizations/optimizeReadInOrder.cpp | 28 +++++++------- .../optimizeUseAggregateProjection.cpp | 4 +- .../Optimizations/projectionsCommon.cpp | 7 ++-- .../Optimizations/removeRedundantDistinct.cpp | 20 +++++----- .../QueryPlan/ReadFromMergeTree.cpp | 16 ++++---- .../QueryPlan/SourceStepWithFilter.cpp | 8 ++-- src/Processors/QueryPlan/TotalsHavingStep.cpp | 22 +++++++---- .../Transforms/FillingTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- src/Storages/SelectQueryInfo.h | 4 +- src/Storages/StorageBuffer.cpp | 10 +++-- src/Storages/StorageMerge.cpp | 8 ++-- src/Storages/VirtualColumnUtils.cpp | 2 +- 28 files changed, 135 insertions(+), 140 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e001406408f..53e04f24829 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1246,14 +1246,6 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) return true; } -ActionsDAGPtr ActionsDAG::clone(const ActionsDAG * from) -{ - std::unordered_map old_to_new_nodes; - if (from == nullptr) - return nullptr; - return std::make_unique(from->clone(old_to_new_nodes)); -} - ActionsDAG ActionsDAG::clone() const { std::unordered_map old_to_new_nodes; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 6f5c3d3b0df..6f6c3f9bccb 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -261,8 +261,6 @@ public: void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif - static ActionsDAGPtr clone(const ActionsDAG * from); - ActionsDAG clone(std::unordered_map & old_to_new_nodes) const; ActionsDAG clone() const; @@ -491,7 +489,6 @@ public: const ActionsDAG::Node * find(const String & output_name); private: - //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 0f350602777..6b5b129085d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,10 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - ActionsDAG dag = std::move(*ActionsDAG::clone(&prewhere_dag_and_flags->dag)); - ExpressionActions( - std::move(dag), - ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); + prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1979,9 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - ExpressionActions( - std::move(*ActionsDAG::clone(&before_where->dag)), - ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); + before_where->dag.updateHeader(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c85eb8310dc..e0073a6af5d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1501,7 +1501,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), + expressions.filter_info->actions.clone(), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1515,7 +1515,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.prewhere_info->row_level_filter)), + expressions.prewhere_info->row_level_filter->clone(), expressions.prewhere_info->row_level_column_name, true); @@ -1525,7 +1525,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&*expressions.prewhere_info->prewhere_actions)), + expressions.prewhere_info->prewhere_actions->clone(), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -1627,7 +1627,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&expressions.filter_info->actions)), + expressions.filter_info->actions.clone(), expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -2056,20 +2056,22 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & prewhere_info = *query_info.prewhere_info; + auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); if (prewhere_info.row_level_filter) { pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, - std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.row_level_filter))), + row_level_actions, prewhere_info.row_level_column_name, true); }); } + auto filter_actions = std::make_shared(prewhere_info.prewhere_actions->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info.prewhere_actions))), + header, filter_actions, prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); }); } @@ -2589,7 +2591,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&*alias_actions))); + auto table_aliases = std::make_unique(query_plan.getCurrentDataStream(), alias_actions->clone()); table_aliases->setStepDescription("Add table aliases"); query_plan.addStep(std::move(table_aliases)); } @@ -2597,7 +2599,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2771,7 +2773,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2789,7 +2791,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( std::optional dag; if (expression) { - dag = std::move(*ActionsDAG::clone(&expression->dag)); + dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2838,7 +2840,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act if (!expression) return; - ActionsDAG dag = std::move(*ActionsDAG::clone(&expression->dag)); + auto dag = expression->dag.clone(); if (expression->project_input) dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 5b710149d85..57ad5caa4c7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1281,7 +1281,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v if (i < stage.filter_column_names.size()) { - auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. @@ -1289,7 +1289,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } else { - auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag)); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e087c3691b4..fb721069e6e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -333,11 +333,11 @@ public: }; void addExpressionStep(QueryPlan & query_plan, - const ActionsAndProjectInputsFlagPtr & expression_actions, + ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = std::move(*ActionsDAG::clone(&expression_actions->dag)); + auto actions = std::move(expression_actions->dag); if (expression_actions->project_input) actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -348,11 +348,11 @@ void addExpressionStep(QueryPlan & query_plan, } void addFilterStep(QueryPlan & query_plan, - const FilterAnalysisResult & filter_analysis_result, + FilterAnalysisResult & filter_analysis_result, const std::string & step_description, UsefulSets & useful_sets) { - auto actions = std::move(*ActionsDAG::clone(&filter_analysis_result.filter_actions->dag)); + auto actions = std::move(filter_analysis_result.filter_actions->dag); if (filter_analysis_result.filter_actions->project_input) actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -544,7 +544,7 @@ void addMergingAggregatedStep(QueryPlan & query_plan, } void addTotalsHavingStep(QueryPlan & query_plan, - const PlannerExpressionsAnalysisResult & expression_analysis_result, + PlannerExpressionsAnalysisResult & expression_analysis_result, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const QueryNode & query_node, @@ -553,14 +553,14 @@ void addTotalsHavingStep(QueryPlan & query_plan, const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); - const auto & having_analysis_result = expression_analysis_result.getHaving(); + auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); + auto & having_analysis_result = expression_analysis_result.getHaving(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); std::optional actions; if (having_analysis_result.filter_actions) { - actions = std::move(*ActionsDAG::clone(&having_analysis_result.filter_actions->dag)); + actions = std::move(having_analysis_result.filter_actions->dag); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -886,7 +886,7 @@ bool addPreliminaryLimitOptimizationStepIfNeeded(QueryPlan & query_plan, * WINDOW functions. */ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, - const PlannerExpressionsAnalysisResult & expressions_analysis_result, + PlannerExpressionsAnalysisResult & expressions_analysis_result, const QueryAnalysisResult & query_analysis_result, const PlannerContextPtr & planner_context, const PlannerQueryProcessingInfo & query_processing_info, @@ -922,7 +922,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, if (expressions_analysis_result.hasLimitBy()) { - const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); + auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy(); addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1549,7 +1549,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasAggregation()) { - const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); + auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); if (aggregation_analysis_result.before_aggregation_actions) addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets); @@ -1568,7 +1568,7 @@ void Planner::buildPlanForQueryNode() * window functions, we can't execute ORDER BY and DISTINCT * now, on shard (first_stage). */ - const auto & window_analysis_result = expression_analysis_result.getWindow(); + auto & window_analysis_result = expression_analysis_result.getWindow(); if (window_analysis_result.before_window_actions) addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets); } @@ -1578,7 +1578,7 @@ void Planner::buildPlanForQueryNode() * Projection expressions, preliminary DISTINCT and before ORDER BY expressions * now, on shards (first_stage). */ - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) @@ -1594,7 +1594,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { - const auto & sort_analysis_result = expression_analysis_result.getSort(); + auto & sort_analysis_result = expression_analysis_result.getSort(); addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } @@ -1648,7 +1648,7 @@ void Planner::buildPlanForQueryNode() { if (expression_analysis_result.hasWindow()) { - const auto & window_analysis_result = expression_analysis_result.getWindow(); + auto & window_analysis_result = expression_analysis_result.getWindow(); if (expression_analysis_result.hasAggregation()) addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets); @@ -1658,7 +1658,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasQualify()) addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets); - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets); if (query_node.isDistinct()) @@ -1674,7 +1674,7 @@ void Planner::buildPlanForQueryNode() if (expression_analysis_result.hasSort()) { - const auto & sort_analysis_result = expression_analysis_result.getSort(); + auto & sort_analysis_result = expression_analysis_result.getSort(); addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets); } } @@ -1727,7 +1727,7 @@ void Planner::buildPlanForQueryNode() if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy()) { - const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); + auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets); addLimitByStep(query_plan, limit_by_analysis_result, query_node); } @@ -1759,7 +1759,7 @@ void Planner::buildPlanForQueryNode() /// Project names is not done on shards, because initiator will not find columns in blocks if (!query_processing_info.isToAggregationState()) { - const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets); } diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 820df7131a7..283fcac7aba 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -64,7 +64,7 @@ public: : projection_analysis_result(std::move(projection_analysis_result_)) {} - const ProjectionAnalysisResult & getProjection() const + ProjectionAnalysisResult & getProjection() { return projection_analysis_result; } @@ -74,7 +74,7 @@ public: return where_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getWhere() const + FilterAnalysisResult & getWhere() { return where_analysis_result; } @@ -89,7 +89,7 @@ public: return !aggregation_analysis_result.aggregation_keys.empty() || !aggregation_analysis_result.aggregate_descriptions.empty(); } - const AggregationAnalysisResult & getAggregation() const + AggregationAnalysisResult & getAggregation() { return aggregation_analysis_result; } @@ -104,7 +104,7 @@ public: return having_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getHaving() const + FilterAnalysisResult & getHaving() { return having_analysis_result; } @@ -119,7 +119,7 @@ public: return !window_analysis_result.window_descriptions.empty(); } - const WindowAnalysisResult & getWindow() const + WindowAnalysisResult & getWindow() { return window_analysis_result; } @@ -134,7 +134,7 @@ public: return qualify_analysis_result.filter_actions != nullptr; } - const FilterAnalysisResult & getQualify() const + FilterAnalysisResult & getQualify() { return qualify_analysis_result; } @@ -149,7 +149,7 @@ public: return sort_analysis_result.before_order_by_actions != nullptr; } - const SortAnalysisResult & getSort() const + SortAnalysisResult & getSort() { return sort_analysis_result; } @@ -164,7 +164,7 @@ public: return limit_by_analysis_result.before_limit_by_actions != nullptr; } - const LimitByAnalysisResult & getLimitBy() const + LimitByAnalysisResult & getLimitBy() { return limit_by_analysis_result; } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 4cf1b138bed..5acff9dac82 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -591,10 +591,10 @@ JoinClausesAndActions buildJoinClausesAndActions( } } - result.left_join_expressions_actions = std::move(*ActionsDAG::clone(&left_join_actions)); + result.left_join_expressions_actions = left_join_actions.clone(); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names); - result.right_join_expressions_actions = std::move(*ActionsDAG::clone(&right_join_actions)); + result.right_join_expressions_actions = right_join_actions.clone(); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 94098f443d9..6f88c4527a4 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -61,13 +61,13 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, settings.indent_char); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); expression->describeActions(settings.out, prefix); } void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const { - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 5f15c5defac..0c6b71387b7 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << " (removed)"; settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); expression->describeActions(settings.out, prefix); } @@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Filter Column", filter_column_name); map.add("Removes Filter", remove_filter_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag))); + auto expression = std::make_shared(actions_dag.clone()); map.add("Expression", expression->toTree()); } diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 8666912514e..37e61a6c388 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -10,18 +10,18 @@ namespace DB::QueryPlanOptimizations { /// build actions DAG from stack of steps -static ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) +static std::optional buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) - return nullptr; + return {}; - ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); + ActionsDAG path_actions = dag_stack.back()->clone(); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); + ActionsDAG clone = dag_stack.back()->clone(); dag_stack.pop_back(); - path_actions->mergeInplace(std::move(*clone)); + path_actions.mergeInplace(std::move(clone)); } return path_actions; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 411b20b1a32..73314f005b6 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -599,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes filter_node.step = std::make_unique( filter_node.children.front()->step->getOutputStream(), - std::move(*ActionsDAG::clone(&filter->getExpression())), + filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()); } @@ -613,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * read_from_merge = typeid_cast(child.get())) { - FilterDAGInfo info{std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()}; + FilterDAGInfo info{filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()}; read_from_merge->addFilter(std::move(info)); std::swap(*parent_node, *child_node); return 1; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index 53f59198d0f..c48551732c9 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), - std::move(*ActionsDAG::clone(&expression->getExpression()))); + expression->getExpression().clone()); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 71a7ca327b1..63b4e019066 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->prewhere_actions->clone()), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->row_level_filter->clone()), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(ActionsDAG::clone(&filter_step->getExpression()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(std::make_unique(filter_step->getExpression().clone()), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 99aaef6d054..252420e19fe 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -171,17 +171,17 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt } } -static void appendExpression(ActionsDAGPtr & dag, const ActionsDAG & expression) +static void appendExpression(std::optional & dag, const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); + dag->mergeInplace(expression.clone()); else - dag = ActionsDAG::clone(&expression); + dag = expression.clone(); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, build a set of fixed columns. -void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns, size_t & limit) +void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, FixedColumns & fixed_columns, size_t & limit) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) @@ -330,7 +330,7 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) InputOrderInfoPtr buildInputOrderInfo( const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, const KeyDescription & sorting_key, size_t limit) @@ -507,7 +507,7 @@ struct AggregationInputOrder AggregationInputOrder buildInputOrderInfo( const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys, const ActionsDAG & sorting_key_dag, const Names & sorting_key_columns) @@ -693,7 +693,7 @@ AggregationInputOrder buildInputOrderInfo( InputOrderInfoPtr buildInputOrderInfo( const ReadFromMergeTree * reading, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, size_t limit) { @@ -709,7 +709,7 @@ InputOrderInfoPtr buildInputOrderInfo( InputOrderInfoPtr buildInputOrderInfo( ReadFromMerge * merge, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const SortDescription & description, size_t limit) { @@ -745,7 +745,7 @@ InputOrderInfoPtr buildInputOrderInfo( AggregationInputOrder buildInputOrderInfo( ReadFromMergeTree * reading, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys) { const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); @@ -760,7 +760,7 @@ AggregationInputOrder buildInputOrderInfo( AggregationInputOrder buildInputOrderInfo( ReadFromMerge * merge, const FixedColumns & fixed_columns, - const ActionsDAGPtr & dag, + const std::optional & dag, const Names & group_by_keys) { const auto & tables = merge->getSelectedTables(); @@ -801,7 +801,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n const auto & description = sorting.getSortDescription(); size_t limit = sorting.getLimit(); - ActionsDAGPtr dag; + std::optional dag; FixedColumns fixed_columns; buildSortingDAG(node, dag, fixed_columns, limit); @@ -855,7 +855,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl const auto & keys = aggregating.getParams().keys; size_t limit = 0; - ActionsDAGPtr dag; + std::optional dag; FixedColumns fixed_columns; buildSortingDAG(node, dag, fixed_columns, limit); @@ -1076,13 +1076,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, for (const auto & actions_dag : window_desc.partition_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } for (const auto & actions_dag : window_desc.order_by_actions) { order_by_elements_actions.emplace_back( - std::make_shared(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); + std::make_shared(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); } auto order_optimizer = std::make_shared( diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 4448d4b7869..ad89cec5f79 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -43,7 +43,7 @@ static DAGIndex buildDAGIndex(const ActionsDAG & dag) /// Required analysis info from aggregate projection. struct AggregateProjectionInfo { - ActionsDAGPtr before_aggregation; + std::optional before_aggregation; Names keys; AggregateDescriptions aggregates; @@ -78,7 +78,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = ActionsDAG::clone(&analysis_result.before_aggregation->dag); + info.before_aggregation = analysis_result.before_aggregation->dag.clone(); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index ad76976becc..571d1dd0cc1 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -68,9 +68,9 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea void QueryDAG::appendExpression(const ActionsDAG & expression) { if (dag) - dag->mergeInplace(std::move(*ActionsDAG::clone(&expression))); + dag->mergeInplace(expression.clone()); else - dag = std::move(*ActionsDAG::clone(&expression)); + dag = expression.clone(); } const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) @@ -239,7 +239,8 @@ bool analyzeProjectionCandidate( auto projection_query_info = query_info; projection_query_info.prewhere_info = nullptr; - projection_query_info.filter_actions_dag = ActionsDAG::clone(dag); + if (dag) + projection_query_info.filter_actions_dag = std::make_unique(dag->clone()); auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp index d0acd8221d4..7664822cc7e 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantDistinct.cpp @@ -43,10 +43,10 @@ namespace } } - void logActionsDAG(const String & prefix, const ActionsDAGPtr & actions) + void logActionsDAG(const String & prefix, const ActionsDAG & actions) { if constexpr (debug_logging_enabled) - LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG()); + LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions.dumpDAG()); } using DistinctColumns = std::set; @@ -65,19 +65,19 @@ namespace } /// build actions DAG from stack of steps - ActionsDAGPtr buildActionsForPlanPath(std::vector & dag_stack) + std::optional buildActionsForPlanPath(std::vector & dag_stack) { if (dag_stack.empty()) - return nullptr; + return {}; - ActionsDAGPtr path_actions = ActionsDAG::clone(dag_stack.back()); + ActionsDAG path_actions = dag_stack.back()->clone(); dag_stack.pop_back(); while (!dag_stack.empty()) { - ActionsDAGPtr clone = ActionsDAG::clone(dag_stack.back()); + ActionsDAG clone = dag_stack.back()->clone(); logActionsDAG("DAG to merge", clone); dag_stack.pop_back(); - path_actions->mergeInplace(std::move(*clone)); + path_actions.mergeInplace(std::move(clone)); } return path_actions; } @@ -260,15 +260,15 @@ namespace if (distinct_columns.size() != inner_distinct_columns.size()) return false; - ActionsDAGPtr path_actions; + ActionsDAG path_actions; if (!dag_stack.empty()) { /// build actions DAG to find original column names - path_actions = buildActionsForPlanPath(dag_stack); + path_actions = std::move(*buildActionsForPlanPath(dag_stack)); logActionsDAG("distinct pass: merged DAG", path_actions); /// compare columns of two DISTINCTs - FindOriginalNodeForOutputName original_node_finder(*path_actions); + FindOriginalNodeForOutputName original_node_finder(path_actions); for (const auto & column : distinct_columns) { const auto * alias_node = original_node_finder.find(String(column)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index add53f9d6b3..5dda4ddc18b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -802,7 +802,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ info.use_uncompressed_cache); }; - auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); + auto sorting_expr = metadata_for_reading->getSortingKey().expression; SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( metadata_for_reading->getPrimaryKey(), @@ -1215,7 +1215,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. RangesInDataParts non_intersecting_parts_by_primary_key; - auto sorting_expr = std::make_shared(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG()))); + auto sorting_expr = metadata_for_reading->getSortingKey().expression; if (prewhere_info) { @@ -1523,7 +1523,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*ActionsDAG::clone(&*filter_actions_dag))); + query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); buildIndexes( indexes, @@ -2004,7 +2004,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result.sampling.use_sampling) { - auto sampling_actions = std::make_shared(std::move(*ActionsDAG::clone(result.sampling.filter_expression.get()))); + auto sampling_actions = std::make_shared(result.sampling.filter_expression->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2137,7 +2137,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -2146,7 +2146,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -2172,7 +2172,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -2182,7 +2182,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index 55c9b5e442e..b91debc8239 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); expression->describeActions(format_settings.out, prefix); } @@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); expression->describeActions(format_settings.out, prefix); } } @@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions))); + auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); @@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const { std::unique_ptr row_level_filter_map = std::make_unique(); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter))); + auto expression = std::make_shared(prewhere_info->row_level_filter->clone()); row_level_filter_map->add("Row level filter expression", expression->toTree()); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 70457918de1..2554053064f 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -101,13 +101,16 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const if (actions_dag) { bool first = true; - auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); - for (const auto & action : expression->getActions()) + if (actions_dag) { - settings.out << prefix << (first ? "Actions: " - : " "); - first = false; - settings.out << action.toString() << '\n'; + auto expression = std::make_shared(actions_dag->clone()); + for (const auto & action : expression->getActions()) + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } } } @@ -118,8 +121,11 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const if (actions_dag) { map.add("Filter column", filter_column_name); - auto expression = std::make_shared(std::move(*ActionsDAG::clone(getActions()))); - map.add("Expression", expression->toTree()); + if (actions_dag) + { + auto expression = std::make_shared(actions_dag->clone()); + map.add("Expression", expression->toTree()); + } } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 36ffc515f43..9601f821cc8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -203,7 +203,7 @@ FillingTransform::FillingTransform( , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) - interpolate_actions = std::make_shared(std::move(*ActionsDAG::clone(&interpolate_description->actions))); + interpolate_actions = std::make_shared(interpolate_description->actions.clone()); std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = fill_description.size(); i < size; ++i) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index ca31ffc9de5..c0875ed184d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -265,7 +265,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( if (!set->buildOrderedSetInplace(context)) return; - auto filter_actions_dag = std::move(*ActionsDAG::clone(filter_dag)); + auto filter_actions_dag = filter_dag->clone(); const auto * filter_actions_dag_node = filter_actions_dag.getOutputs().at(0); std::unordered_map node_to_result_node; @@ -319,7 +319,7 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node return index_hint.getActions().getOutputs(); /// Import the DAG and map argument pointers. - auto actions_clone = std::move(*ActionsDAG::clone(&index_hint.getActions())); + auto actions_clone = index_hint.getActions().clone(); chassert(storage); result_dag_or_null->mergeNodes(std::move(actions_clone), storage); return *storage; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 22289187cfa..f1df9e231c4 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep row_level_filter_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)), actions_settings), + .actions = std::make_shared(prewhere_info->row_level_filter->clone(), actions_settings), .filter_column_name = prewhere_info->row_level_column_name, .remove_filter_column = true, .need_filter = true, @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)), actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 589698fcc30..60f103fdb70 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -66,10 +66,10 @@ struct PrewhereInfo PrewhereInfoPtr prewhere_info = std::make_shared(); if (row_level_filter) - prewhere_info->row_level_filter = std::move(*ActionsDAG::clone(&*row_level_filter)); + prewhere_info->row_level_filter = row_level_filter->clone(); if (prewhere_actions) - prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(&*prewhere_actions)); + prewhere_info->prewhere_actions = prewhere_actions->clone(); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c096504170e..aee4e4683ad 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -313,7 +313,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->row_level_filter) { src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( - std::move(*ActionsDAG::clone(&actions_dag)), + actions_dag.clone(), std::move(*src_table_query_info.prewhere_info->row_level_filter)); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); @@ -322,7 +322,7 @@ void StorageBuffer::read( if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( - std::move(*ActionsDAG::clone(&actions_dag)), + actions_dag.clone(), std::move(*src_table_query_info.prewhere_info->prewhere_actions)); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); @@ -429,21 +429,23 @@ void StorageBuffer::read( if (query_info.prewhere_info->row_level_filter) { + auto actions = std::make_shared(query_info.prewhere_info->row_level_filter->clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->row_level_filter)), actions_settings), + actions, query_info.prewhere_info->row_level_column_name, false); }); } + auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - std::make_shared(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->prewhere_actions)), actions_settings), + actions, query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e16e2a07685..0e1568c8e79 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -663,7 +663,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ { auto filter_step = std::make_unique( child.plan.getCurrentDataStream(), - std::move(*ActionsDAG::clone(&filter_info.actions)), + filter_info.actions.clone(), filter_info.column_name, filter_info.do_remove_column); @@ -1241,7 +1241,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); - filter_actions = std::make_shared(std::move(*ActionsDAG::clone(&actions_dag)), + filter_actions = std::make_shared(actions_dag.clone(), ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); @@ -1279,12 +1279,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(ActionsDAG::clone(&actions_dag), filter_column_name); + step->addFilter(std::make_unique(actions_dag.clone()), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - auto filter_step = std::make_unique(plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&actions_dag)), filter_column_name, true /* remove filter column */); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag.clone(), filter_column_name, true /* remove filter column */); plan.addStep(std::move(filter_step)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 32c6a558340..146a54eda78 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -323,7 +323,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) { - auto index_hint_dag = std::move(*ActionsDAG::clone(&index_hint->getActions())); + 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)) From 7fc8ee726e3ef2dfb7d778fbb1a70fb147a33067 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 18 Jul 2024 12:32:16 +0200 Subject: [PATCH 0195/1170] add replication lag and recovery time metrics --- src/Databases/DatabaseReplicated.cpp | 44 +++++++++---------- src/Databases/DatabaseReplicated.h | 10 ++++- src/Databases/DatabaseReplicatedWorker.cpp | 4 ++ src/Databases/DatabaseReplicatedWorker.h | 4 ++ src/Storages/System/StorageSystemClusters.cpp | 33 +++++++++----- src/Storages/System/StorageSystemClusters.h | 2 +- .../test_recovery_time_metric/__init__.py | 0 .../configs/config.xml | 41 +++++++++++++++++ .../test_recovery_time_metric/test.py | 26 +++++++++++ 9 files changed, 129 insertions(+), 35 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 diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7ce2859e962..b11b9382732 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -338,42 +338,40 @@ 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; + ReplicasInfo res; + + auto zookeeper = getZooKeeper(); 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) - { - for (const auto & replica : addresses_with_failover[shard_index]) - { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - } - } try { - auto current_zookeeper = getZooKeeper(); - auto res = current_zookeeper->exists(paths); + UInt32 max_log_ptr = parse(zookeeper->get(zookeeper_path + "/max_log_ptr")); - std::vector statuses; - statuses.resize(paths.size()); - - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error == Coordination::Error::ZOK) - statuses[i] = 1; - - return statuses; - } - catch (...) + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); + UInt32 log_ptr = parse(zookeeper->get(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr")); + bool is_active = zookeeper->exists(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + res.push_back(ReplicaInfo{ + .is_active = is_active, + .replication_lag = max_log_ptr - log_ptr, + .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, + }); + } + } + return res; + } 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 eab5b2ff931..db02b5ef30f 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -17,6 +17,14 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; +struct ReplicaInfo +{ + bool is_active; + UInt32 replication_lag; + UInt64 recovery_time; +}; +using ReplicasInfo = std::vector; + class DatabaseReplicated : public DatabaseAtomic { public: @@ -84,7 +92,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..cea2d123f87 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,6 +32,8 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { + initialization_duration_timer.emplace(); + while (!stop_flag) { try @@ -69,6 +71,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; + initialization_duration_timer.reset(); return true; } catch (...) @@ -78,6 +81,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } + initialization_duration_timer.reset(); return false; } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 41edf2221b8..aea3b71173d 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 { return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; } private: bool initializeMainThread() override; void initializeReplication(); @@ -56,6 +58,8 @@ 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; }; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 160c8d6270e..0da4bd70cbd 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -31,6 +31,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({ @@ -46,24 +48,23 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co writeCluster(res_columns, name_and_cluster, {}); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & name_and_database : databases) + for (const auto & [database_name, database] : databases) { - if (const auto * replicated = typeid_cast(name_and_database.second.get())) + if (const auto * replicated = typeid_cast(database.get())) { - if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, {database_name, database_cluster}, + replicated->tryGetReplicasInfo(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + database_name, database_cluster}, + replicated->tryGetReplicasInfo(database_cluster)); } } } void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const std::vector & is_active) + const ReplicasInfo & replicas_info) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; @@ -99,10 +100,22 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); res_columns[i++]->insert(address.database_shard_name); res_columns[i++]->insert(address.database_replica_name); - if (is_active.empty()) + if (replicas_info.empty()) + { res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + } else - res_columns[i++]->insert(is_active[replica_idx++]); + { + const auto & replica_info = replicas_info[replica_idx++]; + res_columns[i++]->insert(replica_info.is_active); + res_columns[i++]->insert(replica_info.replication_lag); + if (replica_info.recovery_time != 0) + res_columns[i++]->insert(replica_info.recovery_time); + else + res_columns[i++]->insertDefault(); + } } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 0f7c792261d..ead123aa79e 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -27,7 +27,7 @@ protected: using NameAndCluster = std::pair>; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); + static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const ReplicasInfo & replicas_info); }; } 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..9ceb0cce288 --- /dev/null +++ b/tests/integration/test_recovery_time_metric/test.py @@ -0,0 +1,26 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/config.xml"], with_zookeeper=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("CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');") + node.query("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() + assert ( + node.query("SELECT any(recovery_time) FROM system.clusters;") + != "0\n" + ) + From a6d4db342b2fc83e385d549ba5ce9ebf9e63064e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Jul 2024 16:45:40 +0000 Subject: [PATCH 0196/1170] Automatic style fix --- .../integration/test_recovery_time_metric/test.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 9ceb0cce288..90155f81ba2 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -2,7 +2,9 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/config.xml"], with_zookeeper=True) +node = cluster.add_instance( + "node", main_configs=["configs/config.xml"], with_zookeeper=True +) @pytest.fixture(scope="module") @@ -15,12 +17,10 @@ def start_cluster(): def test_recovery_time_metric(start_cluster): - node.query("CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');") + node.query( + "CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');" + ) node.query("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() - assert ( - node.query("SELECT any(recovery_time) FROM system.clusters;") - != "0\n" - ) - + assert node.query("SELECT any(recovery_time) FROM system.clusters;") != "0\n" From 225af356c7ca3fd3c401f0fa8273d3dd751297fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 17:06:03 +0000 Subject: [PATCH 0197/1170] Better. --- src/Interpreters/ActionsDAG.cpp | 1 - src/Interpreters/ActionsDAG.h | 1 - src/Interpreters/ActionsVisitor.h | 8 +------- src/Interpreters/ExpressionAnalyzer.h | 3 --- src/Interpreters/WindowDescription.h | 1 - src/Interpreters/addMissingDefaults.h | 6 ------ 6 files changed, 1 insertion(+), 19 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 53e04f24829..85b2b38da17 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3150,7 +3150,6 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp } FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_) - //: actions(actions_) { const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 6f6c3f9bccb..76cc9327530 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -501,7 +501,6 @@ public: const ActionsDAG::Node * find(const String & name); private: - //const ActionsDAG & actions; NameToNodeIndex index; }; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 496d9b9b587..5b638fc14c8 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -18,12 +18,6 @@ namespace DB class ASTExpressionList; class ASTFunction; -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; - -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; @@ -32,7 +26,7 @@ FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets); /** For ActionsVisitor - * A stack of ExpressionActions corresponding to nested lambda expressions. + * A stack of ActionsDAG corresponding to nested lambda expressions. * The new action should be added to the highest possible level. * For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)" * calculation of the product must be done outside the lambda expression (it does not depend on x), diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 737d36eb504..0c00247df85 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -38,9 +38,6 @@ using StorageMetadataPtr = std::shared_ptr; class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false); diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index 17bfe619c30..d51d9ca94d8 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -14,7 +14,6 @@ namespace DB class ASTFunction; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; struct WindowFunctionDescription { diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index 5299bae9745..551583a0006 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -2,11 +2,6 @@ #include -#include -#include -#include - - namespace DB { @@ -15,7 +10,6 @@ class NamesAndTypesList; class ColumnsDescription; class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; /** Adds three types of columns into block * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) From 4ebb189691c2d553887e9d49b52f9e0a45eaf004 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 17:59:59 +0000 Subject: [PATCH 0198/1170] Better. --- src/Planner/CollectTableExpressionData.cpp | 8 ++++---- src/Planner/PlannerJoinTree.cpp | 12 ++++++------ src/Planner/TableExpressionData.h | 6 +++--- src/Storages/SelectQueryInfo.cpp | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 1d85476636c..2fe62aa9be0 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -88,15 +88,15 @@ public: auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node); - ActionsDAGPtr alias_column_actions_dag = std::make_unique(); + ActionsDAG alias_column_actions_dag; PlannerActionsVisitor actions_visitor(planner_context, false); - auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression()); + auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression()); if (outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size()); const auto & column_name = column_node->getColumnName(); - const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name); - alias_column_actions_dag->addOrReplaceInOutputs(alias_node); + const auto & alias_node = alias_column_actions_dag.addAlias(*outputs[0], column_name); + alias_column_actions_dag.addOrReplaceInOutputs(alias_node); table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 10b5a761d58..048bfa4b577 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -591,16 +591,16 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) } std::unique_ptr createComputeAliasColumnsStep( - const std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) + std::unordered_map & alias_column_expressions, const DataStream & current_data_stream) { ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs(); - for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) + for (auto & [column_name, alias_column_actions_dag] : alias_column_expressions) { - const auto & current_outputs = alias_column_actions_dag->getOutputs(); + const auto & current_outputs = alias_column_actions_dag.getOutputs(); action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); - merged_alias_columns_actions_dag.mergeNodes(std::move(*alias_column_actions_dag)); + merged_alias_columns_actions_dag.mergeNodes(std::move(alias_column_actions_dag)); } for (const auto * output_node : action_dag_outputs) @@ -996,7 +996,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } } - const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) { auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream()); @@ -1085,7 +1085,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan = std::move(subquery_planner).extractQueryPlan(); } - const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); + auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns) { auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream()); diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 1d04fac3dc3..72412a869e4 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -73,7 +73,7 @@ public: } /// Add alias column - void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true) + void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAG actions_dag, bool is_selected_column = true) { alias_column_expressions.emplace(column.name, std::move(actions_dag)); addColumnImpl(column, column_identifier, is_selected_column); @@ -94,7 +94,7 @@ public: } /// Get ALIAS columns names mapped to expressions - const std::unordered_map & getAliasColumnExpressions() const + std::unordered_map & getAliasColumnExpressions() { return alias_column_expressions; } @@ -277,7 +277,7 @@ private: NameSet selected_column_names_set; /// Expression to calculate ALIAS columns - std::unordered_map alias_column_expressions; + std::unordered_map alias_column_expressions; /// Valid for table, table function, array join, query, union nodes ColumnNameToColumn column_name_to_column; diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp index d59ccf0dfaf..c9c96ed5837 100644 --- a/src/Storages/SelectQueryInfo.cpp +++ b/src/Storages/SelectQueryInfo.cpp @@ -18,7 +18,7 @@ std::unordered_map SelectQueryInfo::buildNod std::unordered_map node_name_to_input_node_column; if (planner_context) { - const auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) { From e806123856f5ded0f2e92f4f4b42c38132276c15 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 18 Jul 2024 20:30:56 +0000 Subject: [PATCH 0199/1170] Fix non x86 build --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 87f634b8334..d7edbc9ed35 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync(bool check_packet_type_only) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] bool check_packet_type_only) { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) From 76904d4ae645aaf9a4cfce938be23a0c60a03fb6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 18 Jul 2024 21:08:35 +0000 Subject: [PATCH 0200/1170] support throw option --- src/Core/SettingsEnums.cpp | 5 +++ src/Interpreters/InterpreterCreateQuery.cpp | 21 ++++++++++++ ...ojection_merge_special_mergetree.reference | 0 ...206_projection_merge_special_mergetree.sql | 32 +++++++++++++++++++ 4 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference create mode 100644 tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82e7d6db410..74b6c793849 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -177,6 +177,11 @@ IMPLEMENT_SETTING_ENUM(LightweightMutationProjectionMode, ErrorCodes::BAD_ARGUME {{"throw", LightweightMutationProjectionMode::THROW}, {"drop", LightweightMutationProjectionMode::DROP}}) +IMPLEMENT_SETTING_ENUM(DeduplicateMergeProjectionMode, ErrorCodes::BAD_ARGUMENTS, + {{"throw", DeduplicateMergeProjectionMode::THROW}, + {"drop", DeduplicateMergeProjectionMode::DROP}, + {"rebuild", DeduplicateMergeProjectionMode::THROW}}) + IMPLEMENT_SETTING_AUTO_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_ENUM(ParquetVersion, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ea10ad59db4..df5ec4525eb 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1281,6 +1281,27 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); + /// Projection is only supported in (Replictaed)MergeTree. + if (std::string_view engine_name(create.storage->engine->name); + !properties.projections.empty() && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") + { + bool projection_support = false; + if (auto * setting = create.storage->settings; setting != nullptr) + { + for (const auto & change : setting->changes) + { + if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) + { + projection_support = true; + break; + } + } + } + if (!projection_support) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Projection is only supported in (Replictaed)MergeTree. Consider drop or rebuild option of deduplicate_merge_projection_mode."); + } + /// Check type compatible for materialized dest table and select columns if (create.select && create.is_materialized_view && create.to_table_id && mode <= LoadingStrictnessLevel::CREATE) { diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql new file mode 100644 index 00000000000..6b5e516ad21 --- /dev/null +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS tp; + +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = ReplacingMergeTree order by type; -- { serverError SUPPORT_IS_DISABLED } + +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = ReplacingMergeTree order by type +SETTINGS deduplicate_merge_projection_mode = 'throw'; -- { serverError SUPPORT_IS_DISABLED } + +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = ReplacingMergeTree order by type +SETTINGS deduplicate_merge_projection_mode = 'drop'; + +DROP TABLE tp; + +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = ReplacingMergeTree order by type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; + +DROP TABLE tp; From 7d9b7cc79611751adc6d22aa47c5e179228a2840 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 02:54:29 +0200 Subject: [PATCH 0201/1170] Introduce ASTDataType --- src/DataTypes/DataTypeFactory.cpp | 14 ++--- src/Databases/DatabasesCommon.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 15 +++-- .../InterpreterShowCreateQuery.cpp | 2 - .../MySQL/InterpretersMySQLDDLQuery.cpp | 28 ++++----- .../formatWithPossiblyHidingSecrets.h | 8 ++- src/Parsers/ASTColumnDeclaration.cpp | 4 -- src/Parsers/ASTDataType.cpp | 57 +++++++++++++++++++ src/Parsers/ASTDataType.h | 36 ++++++++++++ src/Parsers/ASTFunction.cpp | 4 -- src/Parsers/ASTFunction.h | 2 +- src/Parsers/IAST.h | 15 +++-- src/Parsers/ParserCreateQuery.cpp | 7 ++- src/Parsers/ParserCreateQuery.h | 8 +-- src/Parsers/ParserDataType.cpp | 14 ++--- .../StorageMaterializedPostgreSQL.cpp | 13 ++--- src/Storages/WindowView/StorageWindowView.cpp | 3 +- 17 files changed, 156 insertions(+), 76 deletions(-) create mode 100644 src/Parsers/ASTDataType.cpp create mode 100644 src/Parsers/ASTDataType.h diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index af37cde2846..45552e506cd 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -83,15 +83,9 @@ DataTypePtr DataTypeFactory::tryGet(const ASTPtr & ast) const template DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const { - if (const auto * func = ast->as()) + if (const auto * type = ast->as()) { - if (func->parameters) - { - if constexpr (nullptr_on_error) - return nullptr; - throw Exception(ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE, "Data type cannot have multiple parenthesized parameters."); - } - return getImpl(func->name, func->arguments); + return getImpl(type->name, type->arguments); } if (const auto * ident = ast->as()) @@ -107,7 +101,7 @@ DataTypePtr DataTypeFactory::getImpl(const ASTPtr & ast) const if constexpr (nullptr_on_error) return nullptr; - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type."); + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST element for data type: {}.", ast->getID()); } DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fe0baf30e57..cacba581745 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -149,7 +149,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_ columns = metadata_ptr->columns.getAll(); for (const auto & column_name_and_type: columns) { - const auto & ast_column_declaration = std::make_shared(); + const auto ast_column_declaration = std::make_shared(); ast_column_declaration->name = column_name_and_type.name; /// parser typename { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index a846e23cd4f..032fc33ea16 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -12,9 +12,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -25,6 +25,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -432,7 +433,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co auto metadata_snapshot = storage->getInMemoryMetadataPtr(); for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) { - const auto & column_declaration = std::make_shared(); + const auto column_declaration = std::make_shared(); column_declaration->name = column_type_and_name.name; column_declaration->type = getColumnDeclaration(column_type_and_name.type); columns_expression_list->children.emplace_back(column_declaration); @@ -470,17 +471,15 @@ ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) c WhichDataType which(data_type); if (which.isNullable()) - return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isArray()) - return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isDateTime64()) - { - return makeASTFunction("DateTime64", std::make_shared(static_cast(6))); - } + return makeASTDataType("DateTime64", std::make_shared(static_cast(6))); - return std::make_shared(data_type->getName()); + return makeASTDataType(data_type->getName()); } void registerDatabasePostgreSQL(DatabaseFactory & factory) diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 0fca7b64d5a..ca5b7a3b5c1 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -1,9 +1,7 @@ #include #include -#include #include #include -#include #include #include #include diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4821d607d0e..f73965cfcc8 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ #include #include + namespace DB { @@ -95,22 +97,22 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) } ASTPtr data_type = declare_column->data_type; - auto * data_type_function = data_type->as(); + auto * data_type_node = data_type->as(); - if (data_type_function) + if (data_type_node) { - String type_name_upper = Poco::toUpper(data_type_function->name); + String type_name_upper = Poco::toUpper(data_type_node->name); if (is_unsigned) { /// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED) if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED") && !endsWith(type_name_upper, "UNSIGNED")) - data_type_function->name = type_name_upper + " UNSIGNED"; + data_type_node->name = type_name_upper + " UNSIGNED"; } if (type_name_upper == "SET") - data_type_function->arguments.reset(); + data_type_node->arguments.reset(); /// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs /// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3) @@ -119,7 +121,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) if (type_name_upper.find("ENUM") != String::npos) { UInt16 i = 0; - for (ASTPtr & child : data_type_function->arguments->children) + for (ASTPtr & child : data_type_node->arguments->children) { auto new_child = std::make_shared(); new_child->name = "equals"; @@ -133,10 +135,10 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) } if (type_name_upper == "DATE") - data_type_function->name = "Date32"; + data_type_node->name = "Date32"; } if (is_nullable) - data_type = makeASTFunction("Nullable", data_type); + data_type = makeASTDataType("Nullable", data_type); columns_name_and_type.emplace_back(declare_column->name, DataTypeFactory::instance().get(data_type)); } @@ -175,7 +177,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col return columns_description; } -static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, const NamesAndTypesList & columns) +static NamesAndTypesList getNames(const ASTDataType & expr, ContextPtr context, const NamesAndTypesList & columns) { if (expr.arguments->children.empty()) return NamesAndTypesList{}; @@ -219,9 +221,9 @@ static std::tuplechildren.empty()) { @@ -482,7 +484,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( { auto column_declaration = std::make_shared(); column_declaration->name = name; - column_declaration->type = makeASTFunction(type); + column_declaration->type = makeASTDataType(type); column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_expression = std::make_shared(default_value); column_declaration->children.emplace_back(column_declaration->type); diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 039bcbc2bca..ea8c295b169 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -1,11 +1,14 @@ #pragma once -#include "Access/ContextAccess.h" -#include "Interpreters/Context.h" + +#include +#include + #include namespace DB { + struct SecretHidingFormatSettings { // We can't store const Context& as there's a dangerous usage {.ctx = *getContext()} @@ -24,4 +27,5 @@ inline String format(const SecretHidingFormatSettings & settings) return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets); } + } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 4a8a3d2967d..c96499095d5 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include namespace DB @@ -15,8 +13,6 @@ ASTPtr ASTColumnDeclaration::clone() const if (type) { - // Type may be an ASTFunction (e.g. `create table t (a Decimal(9,0))`), - // so we have to clone it properly as well. res->type = type->clone(); res->children.push_back(res->type); } diff --git a/src/Parsers/ASTDataType.cpp b/src/Parsers/ASTDataType.cpp new file mode 100644 index 00000000000..3c17ae8c380 --- /dev/null +++ b/src/Parsers/ASTDataType.cpp @@ -0,0 +1,57 @@ +#include +#include +#include + + +namespace DB +{ + +String ASTDataType::getID(char delim) const +{ + return "DataType" + (delim + name); +} + +ASTPtr ASTDataType::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (arguments) + { + res->arguments = arguments->clone(); + res->children.push_back(res->arguments); + } + + return res; +} + +void ASTDataType::updateTreeHashImpl(SipHash & hash_state, bool) const +{ + hash_state.update(name.size()); + hash_state.update(name); + /// Children are hashed automatically. +} + +void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_function : "") << name; + + if (arguments && !arguments->children.empty()) + { + settings.ostr << '(' << (settings.hilite ? hilite_none : ""); + + for (size_t i = 0, size = arguments->children.size(); i < size; ++i) + { + if (i != 0) + settings.ostr << ", "; + + arguments->children[i]->formatImpl(settings, state, frame); + } + + settings.ostr << (settings.hilite ? hilite_function : "") << ')'; + } + + settings.ostr << (settings.hilite ? hilite_none : ""); +} + +} diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h new file mode 100644 index 00000000000..c8f3c6e2e9d --- /dev/null +++ b/src/Parsers/ASTDataType.h @@ -0,0 +1,36 @@ +#pragma once + +#include + + +namespace DB +{ + +/// AST for data types, e.g. UInt8 or Tuple(x UInt8, y Enum(a = 1)) +class ASTDataType : public IAST +{ +public: + String name; + ASTPtr arguments; + + String getID(char delim) const override; + ASTPtr clone() const override; + void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +template +std::shared_ptr makeASTDataType(const String & name, Args &&... args) +{ + auto function = std::make_shared(); + + function->name = name; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + + function->arguments->children = { std::forward(args)... }; + + return function; +} + +} diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f39229d7566..7d42b6d1e9c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -19,9 +18,6 @@ #include #include #include -#include - -#include using namespace std::literals; diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 3a94691f25d..be2b6beae54 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -46,7 +46,7 @@ public: NullsAction nulls_action = NullsAction::EMPTY; - /// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names. + /// do not print empty parentheses if there are no args - compatibility with engine names. bool no_empty_args = false; /// Specifies where this function-like expression is used. diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index ee70fed0f07..e7e2ff2ec4a 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -271,16 +271,15 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID()); } - // Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. - // You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. + /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. + /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const; - /* - * formatForLogging and formatForErrorMessage always hide secrets. This inconsistent - * behaviour is due to the fact such functions are called from Client which knows nothing about - * access rights and settings. Moreover, the only use case for displaying secrets are backups, - * and backup tools use only direct input and ignore logs and error messages. - */ + /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent + * behaviour is due to the fact such functions are called from Client which knows nothing about + * access rights and settings. Moreover, the only use case for displaying secrets are backups, + * and backup tools use only direct input and ignore logs and error messages. + */ String formatForLogging(size_t max_length = 0) const { return formatWithPossiblyHidingSensitiveData(max_length, true, false); diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 014dc7bd3bf..fa232954cd6 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -75,9 +76,9 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!close.ignore(pos, expected)) return false; - auto func = std::make_shared(); + auto func = std::make_shared(); tryGetIdentifierNameInto(name, func->name); - // FIXME(ilezhankin): func->no_empty_args = true; ? + func->arguments = columns; func->children.push_back(columns); node = func; @@ -749,7 +750,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto * table_id = table->as(); - // Shortcut for ATTACH a previously detached table + /// A shortcut for ATTACH a previously detached table. bool short_attach = attach && !from_path; if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon)) { diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index bb37491a366..7bd1d1bf588 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -101,17 +101,15 @@ class IParserColumnDeclaration : public IParserBase { public: explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false) - : require_type(require_type_) - , allow_null_modifiers(allow_null_modifiers_) - , check_keywords_after_name(check_keywords_after_name_) + : require_type(require_type_) + , allow_null_modifiers(allow_null_modifiers_) + , check_keywords_after_name(check_keywords_after_name_) { } void enableCheckTypeKeyword() { check_type_keyword = true; } protected: - using ASTDeclarePtr = std::shared_ptr; - const char * getName() const override{ return "column declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index ad33c7e4558..63800819899 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -198,13 +199,12 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - auto function_node = std::make_shared(); - function_node->name = type_name; - function_node->no_empty_args = true; + auto data_type_node = std::make_shared(); + data_type_node->name = type_name; if (pos->type != TokenType::OpeningRoundBracket) { - node = function_node; + node = data_type_node; return true; } ++pos; @@ -222,10 +222,10 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; ++pos; - function_node->arguments = expr_list_args; - function_node->children.push_back(function_node->arguments); + data_type_node->arguments = expr_list_args; + data_type_node->children.push_back(data_type_node->arguments); - node = function_node; + node = data_type_node; return true; } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a904b29e12f..f7dde509d4e 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -7,10 +7,8 @@ #include #include -#include #include -#include #include #include #include @@ -22,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -295,7 +294,7 @@ std::shared_ptr StorageMaterializedPostgreSQL::getMaterial auto column_declaration = std::make_shared(); column_declaration->name = std::move(name); - column_declaration->type = makeASTFunction(type); + column_declaration->type = makeASTDataType(type); column_declaration->default_specifier = "MATERIALIZED"; column_declaration->default_expression = std::make_shared(default_value); @@ -312,17 +311,17 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d WhichDataType which(data_type); if (which.isNullable()) - return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); if (which.isArray()) - return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + return makeASTDataType("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); /// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it if (which.isDecimal()) { auto make_decimal_expression = [&](std::string type_name) { - auto ast_expression = std::make_shared(); + auto ast_expression = std::make_shared(); ast_expression->name = type_name; ast_expression->arguments = std::make_shared(); @@ -354,7 +353,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d return ast_expression; } - return std::make_shared(data_type->getName()); + return makeASTDataType(data_type->getName()); } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e15da0074d5..65a30b18e96 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -805,7 +806,7 @@ ASTPtr StorageWindowView::getInnerTableCreateQuery(const ASTPtr & inner_query, c { auto column_window = std::make_shared(); column_window->name = window_id_name; - column_window->type = std::make_shared("UInt32"); + column_window->type = makeASTDataType("UInt32"); columns_list->children.push_back(column_window); } From 2860aa514d0d8837bd72a87390686b20e05ecae7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 03:50:38 +0200 Subject: [PATCH 0202/1170] Fix style --- src/DataTypes/DataTypeFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 45552e506cd..db6e1738d22 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNKNOWN_TYPE; - extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE; extern const int UNEXPECTED_AST_STRUCTURE; extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; } From 985f398925266c5867cd25fcdf655a5a306928fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 05:33:44 +0200 Subject: [PATCH 0203/1170] Fix error --- src/Client/ClientBase.cpp | 1 - src/DataTypes/DataTypeAggregateFunction.cpp | 4 +-- src/Parsers/ParserDataType.cpp | 36 ++++++++++++++------- src/TableFunctions/ITableFunction.cpp | 1 - 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13dce05cabc..95d2dff54e6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -70,7 +70,6 @@ #include #include -#include #include #include #include diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index ef7d86d2a81..09175617bf1 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -257,8 +257,8 @@ static DataTypePtr create(const ASTPtr & arguments) } else throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected AST element passed as aggregate function name for data type AggregateFunction. " - "Must be identifier or function."); + "Unexpected AST element {} passed as aggregate function name for data type AggregateFunction. " + "Must be identifier or function", data_type_ast->getID()); for (size_t i = argument_types_start_idx; i < arguments->children.size(); ++i) argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 63800819899..78b5aaa93a6 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -70,20 +70,32 @@ private: DynamicArgumentsParser parser; return parser.parse(pos, node, expected); } - - ParserNestedTable nested_parser; - ParserDataType data_type_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - - const char * operators[] = {"=", "equals", nullptr}; - ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); - - if (pos->type == TokenType::BareWord && std::string_view(pos->begin, pos->size()) == "Nested") + else if (type_name == "Nested") + { + ParserNestedTable nested_parser; return nested_parser.parse(pos, node, expected); + } + else if (type_name == "AggregateFunction") + { + ParserFunction function_parser; + ParserIdentifier identifier_parser; + ParserAllCollectionsOfLiterals literal_parser(false); + return literal_parser.parse(pos, node, expected) + || identifier_parser.parse(pos, node, expected) + || function_parser.parse(pos, node, expected); + } + else + { + ParserDataType data_type_parser; + ParserAllCollectionsOfLiterals literal_parser(false); - return enum_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || data_type_parser.parse(pos, node, expected); + const char * operators[] = {"=", "equals", nullptr}; + ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); + + return enum_parser.parse(pos, node, expected) + || literal_parser.parse(pos, node, expected) + || data_type_parser.parse(pos, node, expected); + } } std::string_view type_name; diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index e5676c5c25d..916ff7ec022 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include From 15be94ee14a7affe6643dd4c3ac1b104e3c69eeb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:47:27 +0200 Subject: [PATCH 0204/1170] Update src/Parsers/ASTDataType.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Parsers/ASTDataType.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h index c8f3c6e2e9d..abe5cbb8626 100644 --- a/src/Parsers/ASTDataType.h +++ b/src/Parsers/ASTDataType.h @@ -22,13 +22,13 @@ public: template std::shared_ptr makeASTDataType(const String & name, Args &&... args) { - auto function = std::make_shared(); + auto data_type = std::make_shared(); - function->name = name; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); + data_type->name = name; + data_type->arguments = std::make_shared(); + data_type->children.push_back(function->arguments); - function->arguments->children = { std::forward(args)... }; + data_type->arguments->children = { std::forward(args)... }; return function; } From 87fa2c64e96c6bac67275207bd708ac231fa9fb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:48:00 +0200 Subject: [PATCH 0205/1170] Apply review comments --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index f7dde509d4e..e795cd9c6c6 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -345,7 +345,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d if (which.isDateTime64()) { - auto ast_expression = std::make_shared(); + auto ast_expression = std::make_shared(); ast_expression->name = "DateTime64"; ast_expression->arguments = std::make_shared(); From 88dce34be907863b6f3cf6890be87b0d7278a101 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:48:31 +0200 Subject: [PATCH 0206/1170] Fix error --- src/Parsers/ASTDataType.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h index abe5cbb8626..d9427c2fd9e 100644 --- a/src/Parsers/ASTDataType.h +++ b/src/Parsers/ASTDataType.h @@ -26,11 +26,11 @@ std::shared_ptr makeASTDataType(const String & name, Args &&... arg data_type->name = name; data_type->arguments = std::make_shared(); - data_type->children.push_back(function->arguments); + data_type->children.push_back(data_type->arguments); data_type->arguments->children = { std::forward(args)... }; - return function; + return data_type; } } From 4d4e0901881221b39cce0e0527f530fe90eb7ad3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:58:06 +0200 Subject: [PATCH 0207/1170] Fix build --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index e795cd9c6c6..e7b58841c4e 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include From 2832f8c57e37a1fc7d0c91b9ad621785c6d2a5a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 04:59:04 +0200 Subject: [PATCH 0208/1170] Fix a typo --- src/Databases/DatabasesCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index cacba581745..fdbdb610275 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -164,7 +164,7 @@ ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_ if (!parser.parse(pos, ast_type, expected)) { if (throw_on_error) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parser metadata of {}.{}", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parse metadata of {}.{}", backQuote(table_id.database_name), backQuote(table_id.table_name)); else return nullptr; From 6a7a4df1eca0cda4fd2efdcc8aaf2e8741f4cbcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:02:26 +0200 Subject: [PATCH 0209/1170] Fix error --- src/Parsers/ParserDataType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 78b5aaa93a6..af1a299a887 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -75,7 +75,7 @@ private: ParserNestedTable nested_parser; return nested_parser.parse(pos, node, expected); } - else if (type_name == "AggregateFunction") + else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") { ParserFunction function_parser; ParserIdentifier identifier_parser; From cc201745620490c7d885a9e45d9f4b92f7492c10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:19:58 +0200 Subject: [PATCH 0210/1170] Fix bad code: it was catching exceptions --- src/IO/WithFileSize.cpp | 48 +++++++++---------- ...ry_and_native_with_binary_encoded_types.sh | 4 +- 2 files changed, 25 insertions(+), 27 deletions(-) diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 3660d962c08..8cea12fa200 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -14,40 +14,38 @@ namespace ErrorCodes } template -static size_t getFileSize(T & in) +static std::optional tryGetFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - { return with_file_size->getFileSize(); - } + + return std::nullopt; +} + +template +static size_t getFileSize(T & in) +{ + if (auto maybe_size = tryGetFileSize(in)) + return *maybe_size; throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } -size_t getFileSizeFromReadBuffer(ReadBuffer & in) -{ - if (auto * delegate = dynamic_cast(&in)) - { - return getFileSize(delegate->getWrappedReadBuffer()); - } - else if (auto * compressed = dynamic_cast(&in)) - { - return getFileSize(compressed->getWrappedReadBuffer()); - } - - return getFileSize(in); -} - std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in) { - try - { - return getFileSizeFromReadBuffer(in); - } - catch (...) - { - return std::nullopt; - } + if (auto * delegate = dynamic_cast(&in)) + return tryGetFileSize(delegate->getWrappedReadBuffer()); + else if (auto * compressed = dynamic_cast(&in)) + return tryGetFileSize(compressed->getWrappedReadBuffer()); + return tryGetFileSize(in); +} + +size_t getFileSizeFromReadBuffer(ReadBuffer & in) +{ + if (auto maybe_size = tryGetFileSizeFromReadBuffer(in)) + return *maybe_size; + + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } bool isBufferWithFileSize(const ReadBuffer & in) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index 723b11ad620..0c585d36348 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test { - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" } test "materialize(42)::UInt8" From e1a24c9dd6f6320ce02714265e91e83f8dbf43f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 05:45:57 +0200 Subject: [PATCH 0211/1170] Fix error --- src/Parsers/ParserDataType.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index af1a299a887..c5da4a32e92 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -80,9 +79,9 @@ private: ParserFunction function_parser; ParserIdentifier identifier_parser; ParserAllCollectionsOfLiterals literal_parser(false); - return literal_parser.parse(pos, node, expected) - || identifier_parser.parse(pos, node, expected) - || function_parser.parse(pos, node, expected); + return function_parser.parse(pos, node, expected) + || literal_parser.parse(pos, node, expected) + || identifier_parser.parse(pos, node, expected); } else { From bc1146389617f9e9198d0dd25eea89a9206421ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 03:16:23 +0200 Subject: [PATCH 0212/1170] Fix error --- src/Parsers/ParserDataType.cpp | 142 +++++++++++++++++++-------------- 1 file changed, 84 insertions(+), 58 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index c5da4a32e92..2edb0141e12 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -46,60 +46,6 @@ private: } }; -/// Wrapper to allow mixed lists of nested and normal types. -/// Parameters are either: -/// - Nested table elements; -/// - Enum element in form of 'a' = 1; -/// - literal; -/// - Dynamic type arguments; -/// - another data type (or identifier); -class ParserDataTypeArgument : public IParserBase -{ -public: - explicit ParserDataTypeArgument(std::string_view type_name_) : type_name(type_name_) - { - } - -private: - const char * getName() const override { return "data type argument"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override - { - if (type_name == "Dynamic") - { - DynamicArgumentsParser parser; - return parser.parse(pos, node, expected); - } - else if (type_name == "Nested") - { - ParserNestedTable nested_parser; - return nested_parser.parse(pos, node, expected); - } - else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") - { - ParserFunction function_parser; - ParserIdentifier identifier_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - return function_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || identifier_parser.parse(pos, node, expected); - } - else - { - ParserDataType data_type_parser; - ParserAllCollectionsOfLiterals literal_parser(false); - - const char * operators[] = {"=", "equals", nullptr}; - ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); - - return enum_parser.parse(pos, node, expected) - || literal_parser.parse(pos, node, expected) - || data_type_parser.parse(pos, node, expected); - } - } - - std::string_view type_name; -}; - } bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) @@ -221,11 +167,91 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; /// Parse optional parameters - ParserList args_parser(std::make_unique(type_name), std::make_unique(TokenType::Comma)); - ASTPtr expr_list_args; + ASTPtr expr_list_args = std::make_shared(); + + /// Allow mixed lists of nested and normal types. + /// Parameters are either: + /// - Nested table elements; + /// - Enum element in form of 'a' = 1; + /// - literal; + /// - Dynamic type arguments; + /// - another data type (or identifier); + + size_t arg_num = 0; + bool have_version_of_aggregate_function = false; + while (true) + { + if (arg_num > 0) + { + if (pos->type == TokenType::Comma) + ++pos; + else + break; + } + + ASTPtr arg; + if (type_name == "Dynamic") + { + DynamicArgumentsParser parser; + parser.parse(pos, arg, expected); + } + else if (type_name == "Nested") + { + ParserNestedTable nested_parser; + nested_parser.parse(pos, arg, expected); + } + else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") + { + /// This is less trivial. + /// The first optional argument for AggregateFunction is a numeric literal, defining the version. + /// The next argument is the function name, optionally with parameters. + /// Subsequent arguments are data types. + + if (arg_num == 0 && type_name == "AggregateFunction") + { + ParserUnsignedInteger version_parser; + if (version_parser.parse(pos, arg, expected)) + { + have_version_of_aggregate_function = true; + expr_list_args->children.emplace_back(std::move(arg)); + ++arg_num; + continue; + } + } + + if (arg_num == (have_version_of_aggregate_function ? 1 : 0)) + { + ParserFunction function_parser; + ParserIdentifier identifier_parser; + function_parser.parse(pos, arg, expected) + || identifier_parser.parse(pos, arg, expected); + } + else + { + ParserDataType data_type_parser; + data_type_parser.parse(pos, arg, expected); + } + } + else + { + ParserDataType data_type_parser; + ParserAllCollectionsOfLiterals literal_parser(false); + + const char * operators[] = {"=", "equals", nullptr}; + ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); + + enum_parser.parse(pos, arg, expected) + || literal_parser.parse(pos, arg, expected) + || data_type_parser.parse(pos, arg, expected); + } + + if (!arg) + break; + + expr_list_args->children.emplace_back(std::move(arg)); + ++arg_num; + } - if (!args_parser.parse(pos, expr_list_args, expected)) - return false; if (pos->type == TokenType::Comma) // ignore trailing comma inside Nested structures like Tuple(Int, Tuple(Int, String),) ++pos; From 06594935f0e2dd9fc61882c7e643677de474fd7c Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 19 Jul 2024 01:34:03 +0000 Subject: [PATCH 0213/1170] forbid optimize deduplicate --- src/Interpreters/InterpreterOptimizeQuery.cpp | 5 +++++ .../03206_projection_merge_special_mergetree.sql | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 907a01b0432..8d1ac3455b7 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; + extern const int NOT_IMPLEMENTED; } @@ -42,6 +43,10 @@ BlockIO InterpreterOptimizeQuery::execute() auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext()); + /// Don't allow OPTIMIZE DEDUPLICATE for all engines with projections. + if (ast.deduplicate && !metadata_snapshot->projections.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE with projections are not supported yet"); + // Empty list of names means we deduplicate by all columns, but user can explicitly state which columns to use. Names column_names; if (ast.deduplicate_by_columns) diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index 6b5e516ad21..06fb9a30aca 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -30,3 +30,12 @@ CREATE TABLE tp ( SETTINGS deduplicate_merge_projection_mode = 'rebuild'; DROP TABLE tp; + + +-- don't allow OPTIMIZE DEDUPLICATE for all engines with projections +CREATE TABLE test ( + a INT PRIMARY KEY, + PROJECTION p (SELECT * ORDER BY a) +) engine = MergeTree; + +OPTIMIZE TABLE test DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } From 48e7708d7bcf575123ea20cee9455e0a4cf26791 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 19 Jul 2024 10:29:13 +0800 Subject: [PATCH 0214/1170] fix compile error --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Interpreters/InterpreterDeleteQuery.cpp | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 8 ++++---- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 398fe31f1a9..9b5b5dfc20a 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index b37ec4de4ab..291c8e19db0 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 181fb064b54..aef6c1249d5 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -411,10 +412,6 @@ std::pair, std::vector> InterpreterInsertQuery::buildP if (!running_group) running_group = std::make_shared(getContext()); - if (getContext()->getServerSettings().disable_insertion_and_mutation - && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) - throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); - std::vector sink_chains; std::vector presink_chains; @@ -737,6 +734,9 @@ BlockIO InterpreterInsertQuery::execute() const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); + if (getContext()->getServerSettings().disable_insertion_and_mutation + && query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited"); StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); From 4cb862432c50848e3406899f5c7079b4cf1d62a8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 09:34:20 +0000 Subject: [PATCH 0215/1170] Rename method --- src/Processors/IProcessor.h | 2 +- src/Processors/Sources/RemoteSource.cpp | 2 +- src/Processors/Sources/RemoteSource.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 358983a2179..0776921a814 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,7 +221,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } - virtual void asyncJobReady() {} + virtual void onAsyncJobReady() {} /** You must call this method if 'prepare' returned ExpandPipeline. * This method cannot access any port, but it can create new ports for current processor. diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3ec2356a121..587f6e2001b 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -104,7 +104,7 @@ void RemoteSource::work() ISource::work(); } -void RemoteSource::asyncJobReady() +void RemoteSource::onAsyncJobReady() { chassert(async_read); diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index fa04985f101..2247c781584 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -32,7 +32,7 @@ public: int schedule() override { return fd; } - void asyncJobReady() override; + void onAsyncJobReady() override; void setStorageLimits(const std::shared_ptr & storage_limits_) override; From 8349d260952a6daeb84c653c37ac000cf5302cfd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 11:25:34 +0000 Subject: [PATCH 0216/1170] Simplified implementation --- src/Processors/Sources/RemoteSource.cpp | 17 +++++------ src/Processors/Sources/RemoteSource.h | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 37 +++++++++++++++++++++++ src/QueryPipeline/RemoteQueryExecutor.h | 3 ++ 4 files changed, 49 insertions(+), 10 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 587f6e2001b..46c27676e12 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -89,9 +89,6 @@ ISource::Status RemoteSource::prepare() void RemoteSource::work() { - if (async_immediate_work.exchange(false)) - return; - /// Connection drain is a heavy operation that may take a long time. /// Therefore we move connection drain from prepare() to work(), and drain multiple connections in parallel. /// See issue: https://github.com/ClickHouse/ClickHouse/issues/60844 @@ -101,6 +98,13 @@ void RemoteSource::work() executor_finished = true; return; } + + if (preprocessed_packet) + { + preprocessed_packet = false; + return; + } + ISource::work(); } @@ -111,12 +115,7 @@ void RemoteSource::onAsyncJobReady() if (!was_query_sent) return; - auto res = query_executor->readAsync(/*check_packet_type_only=*/true); - if (res.type == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) - { - work(); - async_immediate_work = true; - } + preprocessed_packet = query_executor->processParallelReplicaPacketIfAny(); } std::optional RemoteSource::tryGenerate() diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 2247c781584..22d3921708b 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -54,7 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - std::atomic_bool async_immediate_work{false}; + bool preprocessed_packet = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index d7edbc9ed35..b15e31a120f 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -936,4 +936,41 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } +bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() +{ +#if defined(OS_LINUX) + if (!read_context || (resent_query && recreate_read_context)) + { + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + + read_context = std::make_unique(*this); + recreate_read_context = false; + } + + { + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + + chassert(!has_postponed_packet); + + read_context->resume(); + if (read_context->isInProgress()) // <- nothing to process + return false; + + const auto packet_type = read_context->getPacketType(); + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + processPacket(read_context->getPacket()); + return true; + } + + has_postponed_packet = true; + return false; + } +#endif +} + } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 6849c3e0a07..6f56df71f1d 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -222,6 +222,9 @@ public: bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } + /// return true if parallel replica packet was processed + bool processParallelReplicaPacketIfAny(); + private: RemoteQueryExecutor( const String & query_, From 4e3fdfc2d6482d42b8e152911e24ee38b1bafc89 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 19 Jul 2024 13:26:35 +0200 Subject: [PATCH 0217/1170] Save writer thread id for debugging --- src/Common/SharedMutex.cpp | 10 +++++++++- src/Common/SharedMutex.h | 2 ++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Common/SharedMutex.cpp b/src/Common/SharedMutex.cpp index 1df09ca998a..7b00ef0b28b 100644 --- a/src/Common/SharedMutex.cpp +++ b/src/Common/SharedMutex.cpp @@ -1,4 +1,5 @@ #include +#include #ifdef OS_LINUX /// Because of futex @@ -12,6 +13,7 @@ namespace DB SharedMutex::SharedMutex() : state(0) , waiters(0) + , writer_thread_id(0) {} void SharedMutex::lock() @@ -32,16 +34,22 @@ void SharedMutex::lock() value |= writers; while (value & readers) futexWaitLowerFetch(state, value); + + writer_thread_id.store(getThreadId()); } bool SharedMutex::try_lock() { UInt64 value = 0; - return state.compare_exchange_strong(value, writers); + bool success = state.compare_exchange_strong(value, writers); + if (success) + writer_thread_id.store(getThreadId()); + return success; } void SharedMutex::unlock() { + writer_thread_id.store(0); state.store(0); if (waiters) futexWakeUpperAll(state); diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index 9215ff62af3..a53e2984239 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -36,6 +36,8 @@ private: alignas(64) std::atomic state; std::atomic waiters; + /// Is set while the lock is held in exclusive mode only to facilitate debugging + std::atomic writer_thread_id; }; } From 55d1656f4d0da2f23b2df719dabeed7999645349 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 19 Jul 2024 13:27:41 +0200 Subject: [PATCH 0218/1170] Moving is not safe, prohibit it --- src/Common/SharedMutex.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index a53e2984239..c77c8765885 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -19,6 +19,8 @@ public: ~SharedMutex() = default; SharedMutex(const SharedMutex &) = delete; SharedMutex & operator=(const SharedMutex &) = delete; + SharedMutex(SharedMutex &&) = delete; + SharedMutex & operator=(SharedMutex &&) = delete; // Exclusive ownership void lock() TSA_ACQUIRE(); From 53ea5510143ded0862fd51922077a7cdc1344fe2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 11:30:55 +0000 Subject: [PATCH 0219/1170] Remove unused code --- src/QueryPipeline/RemoteQueryExecutor.cpp | 14 +------------- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b15e31a120f..b78c38a4134 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -469,7 +469,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() return restartQueryWithoutDuplicatedUUIDs(); } -RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] bool check_packet_type_only) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() { #if defined(OS_LINUX) if (!read_context || (resent_query && recreate_read_context)) @@ -519,18 +519,6 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync([[maybe_unused]] if (read_context->isInProgress()) return ReadResult(read_context->getFileDescriptor()); - if (check_packet_type_only) - { - has_postponed_packet = true; - const auto packet_type = read_context->getPacketType(); - if (packet_type == Protocol::Server::MergeTreeReadTaskRequest - || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) - { - return ReadResult(ReadResult::Type::ParallelReplicasToken); - } - return ReadResult(ReadResult::Type::Nothing); - } - auto read_result = processPacket(read_context->getPacket()); if (read_result.getType() == ReadResult::Type::Data || read_result.getType() == ReadResult::Type::ParallelReplicasToken) return read_result; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 6f56df71f1d..7289e2a2243 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -183,7 +183,7 @@ public: ReadResult read(); /// Async variant of read. Returns ready block or file descriptor which may be used for polling. - ReadResult readAsync(bool check_packet_type_only = false); + ReadResult readAsync(); /// Receive all remain packets and finish query. /// It should be cancelled after read returned empty block. From 3f66b39a18a7bf271a9a9f97dfc075866e2409eb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 14:30:27 +0200 Subject: [PATCH 0220/1170] test replication lag metric --- .../0_stateless/03206_replication_lag_metric.reference | 4 ++++ .../queries/0_stateless/03206_replication_lag_metric.sql | 9 +++++++++ 2 files changed, 13 insertions(+) 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/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..6b86553fcaf --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -0,0 +1,9 @@ +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; + +DROP DATABASE rdb1; +DROP DATABASE rdb2; From 245626e5789064fda39ccc7288b83162284a3617 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 14:30:44 +0200 Subject: [PATCH 0221/1170] small fix --- src/Storages/System/StorageSystemClusters.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index ead123aa79e..f6e08734896 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 { From 212b81da533d18a6b9f02c66b34a2161ff1e5d71 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 19 Jul 2024 14:33:20 +0200 Subject: [PATCH 0222/1170] rm debug prints, fix headers --- src/Processors/Executors/CompletedPipelineExecutor.cpp | 1 - src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ---- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 1 - 3 files changed, 6 deletions(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 1eeee896ede..888835c9beb 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -3,7 +3,6 @@ #include #include #include -#include "Common/Logger.h" #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 97335b601f9..73244b714bf 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -14,8 +14,6 @@ #include #include #include -#include "Common/Logger.h" -#include "Common/logger_useful.h" #include #include #include @@ -186,8 +184,6 @@ void updateTTL( void MergeTreeDataWriter::TemporaryPart::cancel() { - LOG_INFO(getLogger("MergeTreeDataWriter"), "TemporaryPart cancel"); - try { /// An exception context is needed to proper delete write buffers without finalization diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f6d6b8cb7a3..17662f92035 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include From 57c1d7a1011f96cea21ca66a3064b7481f8ce40b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 12:36:57 +0000 Subject: [PATCH 0223/1170] fix filling of empty Nested --- src/DataTypes/IDataType.cpp | 4 +- src/DataTypes/ObjectUtils.cpp | 31 ++++++++++ src/DataTypes/ObjectUtils.h | 3 + src/DataTypes/Serializations/ISerialization.h | 3 +- src/Interpreters/inplaceBlockConversions.cpp | 57 ++++++++++++++++--- src/Storages/MergeTree/IMergeTreeReader.cpp | 7 ++- src/Storages/MergeTree/IMergeTreeReader.h | 3 + 7 files changed, 95 insertions(+), 13 deletions(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 1cb64b65d3a..824bc6e33b0 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -90,7 +90,9 @@ void IDataType::forEachSubcolumn( { auto name = ISerialization::getSubcolumnNameForStream(subpath, prefix_len); auto subdata = ISerialization::createFromPath(subpath, prefix_len); - callback(subpath, name, subdata); + auto path_copy = subpath; + path_copy.resize(prefix_len); + callback(path_copy, name, subdata); } subpath[i].visited = true; } diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 1d525e5987f..356e609e77a 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -66,6 +67,36 @@ DataTypePtr getBaseTypeOfArray(const DataTypePtr & type) return last_array ? last_array->getNestedType() : type; } +DataTypePtr getBaseTypeOfArray(DataTypePtr type, const Names & tuple_elements) +{ + auto it = tuple_elements.begin(); + while (true) + { + if (const auto * type_array = typeid_cast(type.get())) + { + type = type_array->getNestedType(); + } + else if (const auto * type_tuple = typeid_cast(type.get())) + { + if (it == tuple_elements.end()) + break; + + auto pos = type_tuple->tryGetPositionByName(*it); + if (!pos) + break; + + ++it; + type = type_tuple->getElement(*pos); + } + else + { + break; + } + } + + return type; +} + ColumnPtr getBaseColumnOfArray(const ColumnPtr & column) { /// Get raw pointers to avoid extra copying of column pointers. diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 6599d8adef1..21e5c3b2f59 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -27,6 +27,9 @@ size_t getNumberOfDimensions(const IColumn & column); /// Returns type of scalars of Array of arbitrary dimensions. DataTypePtr getBaseTypeOfArray(const DataTypePtr & type); +/// The same as above but takes into account Tuples of Nested. +DataTypePtr getBaseTypeOfArray(DataTypePtr type, const Names & tuple_elements); + /// Returns Array type with requested scalar type and number of dimensions. DataTypePtr createArrayOfType(DataTypePtr type, size_t num_dimensions); diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 255dbbfadd2..5d0bf60c59f 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -195,7 +195,7 @@ public: /// Types of substreams that can have arbitrary name. static const std::set named_types; - Type type; + Type type = Type::Regular; /// The name of a variant element type. String variant_element_name; @@ -212,6 +212,7 @@ public: /// Flag, that may help to traverse substream paths. mutable bool visited = false; + Substream() = default; Substream(Type type_) : type(type_) {} /// NOLINT String toString() const; }; diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index f7d8a2a2daf..ce3f25d16f8 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -283,6 +283,9 @@ static ColumnPtr createColumnWithDefaultValue(const IDataType & data_type, const { auto column = data_type.createColumnConstWithDefaultValue(num_rows); + /// We must turn a constant column into a full column because the interpreter could infer + /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. + if (subcolumn_name.empty()) return column->convertToFullColumnIfConst(); @@ -293,6 +296,35 @@ static ColumnPtr createColumnWithDefaultValue(const IDataType & data_type, const return ColumnConst::create(std::move(column), num_rows)->convertToFullColumnIfConst(); } +static bool hasDefault(const StorageMetadataPtr & metadata_snapshot, const NameAndTypePair & column) +{ + if (!metadata_snapshot) + return false; + + const auto & columns = metadata_snapshot->getColumns(); + if (columns.has(column.name)) + return columns.hasDefault(column.name); + + auto name_in_storage = column.getNameInStorage(); + return columns.hasDefault(name_in_storage); +} + +static String removeTupleElementsFromSubcolumn(String subcolumn_name, const Names & tuple_elements) +{ + subcolumn_name += "."; + for (const auto & elem : tuple_elements) + { + auto pos = subcolumn_name.find(elem + "."); + if (pos != std::string::npos) + subcolumn_name.erase(pos, elem.size()); + } + + if (subcolumn_name.ends_with(".")) + subcolumn_name.pop_back(); + + return subcolumn_name; +} + void fillMissingColumns( Columns & res_columns, size_t num_rows, @@ -321,10 +353,8 @@ void fillMissingColumns( if (res_columns[i] && partially_read_columns.contains(requested_column->name)) res_columns[i] = nullptr; - if (res_columns[i]) - continue; - - if (metadata_snapshot && metadata_snapshot->getColumns().hasDefault(requested_column->getNameInStorage())) + /// Nothing to fill or default should be filled in evaluateMissingDefaults + if (res_columns[i] || hasDefault(metadata_snapshot, *requested_column)) continue; std::vector current_offsets; @@ -365,19 +395,30 @@ void fillMissingColumns( if (!current_offsets.empty()) { + + Names tuple_elements; + auto serialization = IDataType::getSerialization(*requested_column); + + IDataType::forEachSubcolumn([&](const auto & path, const auto &, const auto &) + { + if (path.back().type == ISerialization::Substream::TupleElement) + tuple_elements.push_back(path.back().name_of_substream); + }, ISerialization::SubstreamData(serialization)); + size_t num_empty_dimensions = num_dimensions - current_offsets.size(); - auto scalar_type = createArrayOfType(getBaseTypeOfArray(requested_column->getTypeInStorage()), num_empty_dimensions); + auto base_type = getBaseTypeOfArray(requested_column->getTypeInStorage(), tuple_elements); + auto scalar_type = createArrayOfType(base_type, num_empty_dimensions); size_t data_size = assert_cast(*current_offsets.back()).getData().back(); - res_columns[i] = createColumnWithDefaultValue(*scalar_type, requested_column->getSubcolumnName(), data_size); + auto subcolumn_name = removeTupleElementsFromSubcolumn(requested_column->getSubcolumnName(), tuple_elements); + + res_columns[i] = createColumnWithDefaultValue(*scalar_type, subcolumn_name, data_size); for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) res_columns[i] = ColumnArray::create(res_columns[i], *it); } else { - /// We must turn a constant column into a full column because the interpreter could infer - /// that it is constant everywhere but in some blocks (from other parts) it can be a full column. res_columns[i] = createColumnWithDefaultValue(*requested_column->getTypeInStorage(), requested_column->getSubcolumnName(), num_rows); } } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index aff1001163e..5f36e4c7c13 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -44,6 +44,7 @@ IMergeTreeReader::IMergeTreeReader( , alter_conversions(data_part_info_for_read->getAlterConversions()) /// For wide parts convert plain arrays of Nested to subcolumns /// to allow to use shared offset column from cache. + , original_requested_columns(columns_) , requested_columns(data_part_info_for_read->isWidePart() ? Nested::convertToSubcolumns(columns_) : columns_) @@ -139,7 +140,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns { try { - size_t num_columns = requested_columns.size(); + size_t num_columns = original_requested_columns.size(); if (res_columns.size() != num_columns) throw Exception(ErrorCodes::LOGICAL_ERROR, "invalid number of columns passed to MergeTreeReader::fillMissingColumns. " @@ -151,7 +152,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns /// Convert columns list to block. And convert subcolumns to full columns. /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. - auto it = requested_columns.begin(); + auto it = original_requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++it) { auto name_in_storage = it->getNameInStorage(); @@ -178,7 +179,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } /// Move columns from block. - it = requested_columns.begin(); + it = original_requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++it) { auto name_in_storage = it->getNameInStorage(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index a1ec0339fd6..d799ce57b40 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -112,6 +112,9 @@ protected: private: /// Columns that are requested to read. + NamesAndTypesList original_requested_columns; + + /// The same as above but with converted Arrays to subcolumns of Nested. NamesAndTypesList requested_columns; /// Actual columns description in part. From 189b3d306fc0e488010564384b193412acd0358b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 13:17:17 +0000 Subject: [PATCH 0224/1170] fix tests --- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../0_stateless/02026_describe_include_subcolumns.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index fde9dafffb8..ff0311dc1ca 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -101,7 +101,7 @@ NameAndTypePair MergeTreeReaderCompact::getColumnConvertedToSubcolumnOfNested(co if (!storage_columns_with_collected_nested) { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); + auto options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects(); auto storage_columns_list = Nested::collect(storage_snapshot->getColumns(options)); storage_columns_with_collected_nested = ColumnsDescription(std::move(storage_columns_list)); } diff --git a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference index dec65f62748..62efafceaad 100644 --- a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference +++ b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference @@ -26,7 +26,7 @@ 10. │ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ 11. │ t.a │ Array(Tuple( a UInt32, - b UInt32)) │ │ │ │ │ │ 1 │ + b UInt32)) │ │ │ │ ZSTD(1) │ │ 1 │ 12. │ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ 13. │ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ 14. │ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ From 79ef630d85cb445a743ee2d5950197709d75325f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 19 Jul 2024 15:25:08 +0200 Subject: [PATCH 0225/1170] fix tests --- .../0_stateless/02117_show_create_table_system.reference | 2 ++ tests/queries/0_stateless/03206_replication_lag_metric.sql | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) 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 8f62eda9233..28356632a66 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.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql index 6b86553fcaf..998c332a11c 100644 --- a/tests/queries/0_stateless/03206_replication_lag_metric.sql +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -1,9 +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; +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 f3fb729f53860d55db1d72ccfc88f9c5d018aea1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 20:12:14 +0000 Subject: [PATCH 0226/1170] Call onAsyncJobReady() --- src/Processors/Executors/ExecutorTasks.cpp | 2 ++ src/Processors/Executors/ExecutorTasks.h | 2 +- src/Processors/Executors/PipelineExecutor.h | 1 - 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/ExecutorTasks.cpp b/src/Processors/Executors/ExecutorTasks.cpp index 7e3bee239ef..d045f59a2e2 100644 --- a/src/Processors/Executors/ExecutorTasks.cpp +++ b/src/Processors/Executors/ExecutorTasks.cpp @@ -204,6 +204,8 @@ void ExecutorTasks::processAsyncTasks() while (auto task = async_task_queue.wait(lock)) { auto * node = static_cast(task.data); + node->processor->onAsyncJobReady(); + executor_contexts[task.thread_num]->pushAsyncTask(node); ++num_waiting_async_tasks; diff --git a/src/Processors/Executors/ExecutorTasks.h b/src/Processors/Executors/ExecutorTasks.h index 202ca253c6c..b2201873edf 100644 --- a/src/Processors/Executors/ExecutorTasks.h +++ b/src/Processors/Executors/ExecutorTasks.h @@ -28,7 +28,7 @@ class ExecutorTasks TaskQueue task_queue; /// Queue which stores tasks where processors returned Async status after prepare. - /// If multiple threads are using, main thread will wait for async tasks. + /// If multiple threads are used, main thread will wait for async tasks. /// For single thread, will wait for async tasks only when task_queue is empty. PollingQueue async_task_queue; diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 03f0f7f1a0a..ae119355cb5 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -9,7 +9,6 @@ #include #include -#include #include From 465a34d3dfe3e313471e10d59cab8219b3e5837e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 20:27:57 +0000 Subject: [PATCH 0227/1170] Simplify, fix build --- src/QueryPipeline/RemoteQueryExecutor.cpp | 48 +++++++++++------------ 1 file changed, 22 insertions(+), 26 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b78c38a4134..61a512bcfc5 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -924,41 +924,37 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } -bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() +bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() { #if defined(OS_LINUX) + + std::lock_guard lock(was_cancelled_mutex); + if (was_cancelled) + return false; + if (!read_context || (resent_query && recreate_read_context)) { - std::lock_guard lock(was_cancelled_mutex); - if (was_cancelled) - return false; - read_context = std::make_unique(*this); recreate_read_context = false; } - { - std::lock_guard lock(was_cancelled_mutex); - if (was_cancelled) - return false; + chassert(!has_postponed_packet); - chassert(!has_postponed_packet); - - read_context->resume(); - if (read_context->isInProgress()) // <- nothing to process - return false; - - const auto packet_type = read_context->getPacketType(); - if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) - { - processPacket(read_context->getPacket()); - return true; - } - - has_postponed_packet = true; + read_context->resume(); + if (read_context->isInProgress()) // <- nothing to process return false; - } -#endif -} + const auto packet_type = read_context->getPacketType(); + if (packet_type == Protocol::Server::MergeTreeReadTaskRequest || packet_type == Protocol::Server::MergeTreeAllRangesAnnouncement) + { + processPacket(read_context->getPacket()); + return true; + } + + has_postponed_packet = true; + +#endif + + return false; +} } From 277dbfa0574b567241d169494a459e6f2b04d5e6 Mon Sep 17 00:00:00 2001 From: "Zhukova, Maria" Date: Fri, 19 Jul 2024 13:13:42 -0700 Subject: [PATCH 0228/1170] update QPL to 1.6.0 + missing header fix --- contrib/qpl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/qpl b/contrib/qpl index d4715e0e798..c2ced94c53c 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit d4715e0e79896b85612158e135ee1a85f3b3e04d +Subproject commit c2ced94c53c1ee22191201a59878e9280bc9b9b8 From 1347bc32187b694148657459c0b745c45cb92a8d Mon Sep 17 00:00:00 2001 From: "Zhukova, Maria" Date: Fri, 19 Jul 2024 13:32:18 -0700 Subject: [PATCH 0229/1170] update qpl-cmake to reflect changes in QPL 1.5.0-1.6.0 --- contrib/qpl-cmake/CMakeLists.txt | 92 ++++++++++++++++++++------------ 1 file changed, 57 insertions(+), 35 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 7a84048e16b..b2f263252c2 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -4,7 +4,6 @@ set (QPL_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/build/contrib/qpl") set (EFFICIENT_WAIT OFF) -set (BLOCK_ON_FAULT ON) set (LOG_HW_INIT OFF) set (SANITIZE_MEMORY OFF) set (SANITIZE_THREADS OFF) @@ -16,16 +15,18 @@ function(GetLibraryVersion _content _outputVar) SET(${_outputVar} ${CMAKE_MATCH_1} PARENT_SCOPE) endfunction() -set (QPL_VERSION 1.2.0) +set (QPL_VERSION 1.6.0) message(STATUS "Intel QPL version: ${QPL_VERSION}") -# There are 5 source subdirectories under $QPL_SRC_DIR: isal, c_api, core-sw, middle-layer, c_api. -# Generate 8 library targets: middle_layer_lib, isal, isal_asm, qplcore_px, qplcore_avx512, qplcore_sw_dispatcher, core_iaa, middle_layer_lib. +# There are 5 source subdirectories under $QPL_SRC_DIR: c_api, core-iaa, core-sw, middle-layer and isal. +# Generate 8 library targets: qpl_c_api, core_iaa, qplcore_px, qplcore_avx512, qplcore_sw_dispatcher, middle_layer_lib, isal and isal_asm, +# which are then combined into static or shared qpl. # Output ch_contrib::qpl by linking with 8 library targets. -# The qpl submodule comes with its own version of isal. It contains code which does not exist in upstream isal. It would be nice to link -# only upstream isal (ch_contrib::isal) but at this point we can't. +# Note, qpl submodule comes with its own version of isal that is not compatible with upstream isal (e.g., ch_contrib::isal). + +## cmake/CompileOptions.cmake and automatic wrappers generation # ========================================================================== # Copyright (C) 2022 Intel Corporation @@ -442,6 +443,7 @@ function(generate_unpack_kernel_arrays current_directory PLATFORMS_LIST) endforeach() endfunction() +# [SUBDIR]isal enable_language(ASM_NASM) @@ -479,7 +481,6 @@ set(ISAL_ASM_SRC ${QPL_SRC_DIR}/isal/igzip/igzip_body.asm ${QPL_SRC_DIR}/isal/igzip/igzip_set_long_icf_fg_04.asm ${QPL_SRC_DIR}/isal/igzip/igzip_set_long_icf_fg_06.asm ${QPL_SRC_DIR}/isal/igzip/igzip_multibinary.asm - ${QPL_SRC_DIR}/isal/igzip/stdmac.asm ${QPL_SRC_DIR}/isal/crc/crc_multibinary.asm ${QPL_SRC_DIR}/isal/crc/crc32_gzip_refl_by8.asm ${QPL_SRC_DIR}/isal/crc/crc32_gzip_refl_by8_02.asm @@ -505,7 +506,6 @@ set_property(GLOBAL APPEND PROPERTY QPL_LIB_DEPS # Setting external and internal interfaces for ISA-L library target_include_directories(isal PUBLIC $ - PRIVATE ${QPL_SRC_DIR}/isal/include PUBLIC ${QPL_SRC_DIR}/isal/igzip) set_target_properties(isal PROPERTIES @@ -617,12 +617,9 @@ target_compile_options(qplcore_sw_dispatcher # [SUBDIR]core-iaa file(GLOB HW_PATH_SRC ${QPL_SRC_DIR}/core-iaa/sources/aecs/*.c - ${QPL_SRC_DIR}/core-iaa/sources/aecs/*.cpp ${QPL_SRC_DIR}/core-iaa/sources/driver_loader/*.c - ${QPL_SRC_DIR}/core-iaa/sources/driver_loader/*.cpp ${QPL_SRC_DIR}/core-iaa/sources/descriptors/*.c - ${QPL_SRC_DIR}/core-iaa/sources/descriptors/*.cpp - ${QPL_SRC_DIR}/core-iaa/sources/bit_rev.c) + ${QPL_SRC_DIR}/core-iaa/sources/*.c) # Create library add_library(core_iaa OBJECT ${HW_PATH_SRC}) @@ -634,31 +631,27 @@ target_include_directories(core_iaa PRIVATE ${UUID_DIR} PUBLIC $ PUBLIC $ - PRIVATE $ # status.h in own_checkers.h - PRIVATE $ # own_checkers.h + PRIVATE $ # status.h in own_checkers.h + PRIVATE $ # for own_checkers.h PRIVATE $) target_compile_features(core_iaa PRIVATE c_std_11) target_compile_definitions(core_iaa PRIVATE QPL_BADARG_CHECK - PRIVATE $<$: BLOCK_ON_FAULT_ENABLED> PRIVATE $<$:LOG_HW_INIT> PRIVATE $<$:DYNAMIC_LOADING_LIBACCEL_CONFIG>) # [SUBDIR]middle-layer file(GLOB MIDDLE_LAYER_SRC - ${QPL_SRC_DIR}/middle-layer/analytics/*.cpp - ${QPL_SRC_DIR}/middle-layer/c_wrapper/*.cpp - ${QPL_SRC_DIR}/middle-layer/checksum/*.cpp + ${QPL_SRC_DIR}/middle-layer/accelerator/*.cpp + ${QPL_SRC_DIR}/middle-layer/analytics/*.cpp ${QPL_SRC_DIR}/middle-layer/common/*.cpp ${QPL_SRC_DIR}/middle-layer/compression/*.cpp ${QPL_SRC_DIR}/middle-layer/compression/*/*.cpp ${QPL_SRC_DIR}/middle-layer/compression/*/*/*.cpp ${QPL_SRC_DIR}/middle-layer/dispatcher/*.cpp ${QPL_SRC_DIR}/middle-layer/other/*.cpp - ${QPL_SRC_DIR}/middle-layer/util/*.cpp - ${QPL_SRC_DIR}/middle-layer/inflate/*.cpp - ${QPL_SRC_DIR}/core-iaa/sources/accelerator/*.cpp) # todo + ${QPL_SRC_DIR}/middle-layer/util/*.cpp) add_library(middle_layer_lib OBJECT ${MIDDLE_LAYER_SRC}) @@ -667,6 +660,7 @@ set_property(GLOBAL APPEND PROPERTY QPL_LIB_DEPS $) target_compile_options(middle_layer_lib + PRIVATE $<$:$<$:-O3;-U_FORTIFY_SOURCE;-D_FORTIFY_SOURCE=2>> PRIVATE ${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}) target_compile_definitions(middle_layer_lib @@ -682,6 +676,7 @@ target_include_directories(middle_layer_lib PRIVATE ${UUID_DIR} PUBLIC $ PUBLIC $ + PRIVATE $ PUBLIC $ PUBLIC $ PUBLIC $) @@ -689,31 +684,58 @@ target_include_directories(middle_layer_lib target_compile_definitions(middle_layer_lib PUBLIC -DQPL_LIB) # [SUBDIR]c_api -file(GLOB_RECURSE QPL_C_API_SRC - ${QPL_SRC_DIR}/c_api/*.c - ${QPL_SRC_DIR}/c_api/*.cpp) +file(GLOB QPL_C_API_SRC + ${QPL_SRC_DIR}/c_api/compression_operations/*.c + ${QPL_SRC_DIR}/c_api/compression_operations/*.cpp + ${QPL_SRC_DIR}/c_api/filter_operations/*.cpp + ${QPL_SRC_DIR}/c_api/legacy_hw_path/*.c + ${QPL_SRC_DIR}/c_api/legacy_hw_path/*.cpp + ${QPL_SRC_DIR}/c_api/other_operations/*.cpp + ${QPL_SRC_DIR}/c_api/serialization/*.cpp + ${QPL_SRC_DIR}/c_api/*.cpp) + +add_library(qpl_c_api OBJECT ${QPL_C_API_SRC}) + +target_include_directories(qpl_c_api + PUBLIC $ + PUBLIC $ $ + PRIVATE $) + +set_target_properties(qpl_c_api PROPERTIES + $<$:C_STANDARD 17 + CXX_STANDARD 17) + +target_compile_options(qpl_c_api + PRIVATE $<$:$<$:-O3;-U_FORTIFY_SOURCE;-D_FORTIFY_SOURCE=2>> + PRIVATE $<$:${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}>) + +target_compile_definitions(qpl_c_api + PUBLIC -DQPL_BADARG_CHECK # own_checkers.h + PUBLIC -DQPL_LIB # needed for middle_layer_lib + PUBLIC $<$:LOG_HW_INIT>) # needed for middle_layer_lib + +set_property(GLOBAL APPEND PROPERTY QPL_LIB_DEPS + $) + +# Final _qpl target get_property(LIB_DEPS GLOBAL PROPERTY QPL_LIB_DEPS) -add_library(_qpl STATIC ${QPL_C_API_SRC} ${LIB_DEPS}) +add_library(_qpl STATIC ${LIB_DEPS}) target_include_directories(_qpl - PUBLIC $ $ - PRIVATE $ - PRIVATE $) + PUBLIC $ $) -target_compile_options(_qpl - PRIVATE ${QPL_LINUX_TOOLCHAIN_CPP_EMBEDDED_FLAGS}) target_compile_definitions(_qpl - PRIVATE -DQPL_LIB - PRIVATE -DQPL_BADARG_CHECK - PRIVATE $<$:DYNAMIC_LOADING_LIBACCEL_CONFIG> PUBLIC -DENABLE_QPL_COMPRESSION) target_link_libraries(_qpl - PRIVATE ch_contrib::accel-config - PRIVATE ch_contrib::isal) + PRIVATE ch_contrib::accel-config) + +# C++ filesystem library requires additional linking for older GNU/Clang +target_link_libraries(_qpl PRIVATE $<$,$,9.1>>:stdc++fs>) +target_link_libraries(_qpl PRIVATE $<$,$,9.0>>:c++fs>) target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include" From 7caa7e20601b2500f513476dfe10819f328be3d3 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 20 Jul 2024 02:28:13 +0000 Subject: [PATCH 0230/1170] block deduplicate only in throw mode --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterOptimizeQuery.cpp | 4 ---- src/Storages/StorageMergeTree.cpp | 6 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++-- ...206_projection_merge_special_mergetree.sql | 19 +++++++++---------- 5 files changed, 18 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index df5ec4525eb..05df26b0d31 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1298,7 +1298,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } } if (!projection_support) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Projection is only supported in (Replictaed)MergeTree. Consider drop or rebuild option of deduplicate_merge_projection_mode."); } diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 8d1ac3455b7..3bee235185d 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -43,10 +43,6 @@ BlockIO InterpreterOptimizeQuery::execute() auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext()); - /// Don't allow OPTIMIZE DEDUPLICATE for all engines with projections. - if (ast.deduplicate && !metadata_snapshot->projections.empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEDUPLICATE with projections are not supported yet"); - // Empty list of names means we deduplicate by all columns, but user can explicitly state which columns to use. Names column_names; if (ast.deduplicate_by_columns) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 40b3a12297b..a5d434796ba 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1580,10 +1580,12 @@ bool StorageMergeTree::optimize( { assertNotReadonly(); - if (deduplicate && getInMemoryMetadataPtr()->hasProjections()) + if (deduplicate && getInMemoryMetadataPtr()->hasProjections() + && getSettings()->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", + "User should drop all the projections manually before running the query, " + "or consider drop or rebuild option of deduplicate_merge_projection_mode", getStorageID().getTableName()); if (deduplicate) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 72f725965e0..3751883df24 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5794,10 +5794,12 @@ bool StorageReplicatedMergeTree::optimize( if (!is_leader) throw Exception(ErrorCodes::NOT_A_LEADER, "OPTIMIZE cannot be done on this replica because it is not a leader"); - if (deduplicate && getInMemoryMetadataPtr()->hasProjections()) + if (deduplicate && getInMemoryMetadataPtr()->hasProjections() + && getSettings()->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", + "User should drop all the projections manually before running the query, " + "or consider drop or rebuild option of deduplicate_merge_projection_mode", getStorageID().getTableName()); if (cleanup) diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index 06fb9a30aca..c8945fd784c 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -4,14 +4,14 @@ CREATE TABLE tp ( type Int32, eventcnt UInt64, PROJECTION p (select sum(eventcnt), type group by type) -) engine = ReplacingMergeTree order by type; -- { serverError SUPPORT_IS_DISABLED } +) engine = ReplacingMergeTree order by type; -- { serverError NOT_IMPLEMENTED } CREATE TABLE tp ( type Int32, eventcnt UInt64, PROJECTION p (select sum(eventcnt), type group by type) ) engine = ReplacingMergeTree order by type -SETTINGS deduplicate_merge_projection_mode = 'throw'; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS deduplicate_merge_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } CREATE TABLE tp ( type Int32, @@ -20,6 +20,10 @@ CREATE TABLE tp ( ) engine = ReplacingMergeTree order by type SETTINGS deduplicate_merge_projection_mode = 'drop'; +ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; + +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } + DROP TABLE tp; CREATE TABLE tp ( @@ -29,13 +33,8 @@ CREATE TABLE tp ( ) engine = ReplacingMergeTree order by type SETTINGS deduplicate_merge_projection_mode = 'rebuild'; -DROP TABLE tp; +ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } --- don't allow OPTIMIZE DEDUPLICATE for all engines with projections -CREATE TABLE test ( - a INT PRIMARY KEY, - PROJECTION p (SELECT * ORDER BY a) -) engine = MergeTree; - -OPTIMIZE TABLE test DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +DROP TABLE tp; \ No newline at end of file From e3a0b6ab5ff21518a494ebede1aea47edda22b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 11:45:52 +0200 Subject: [PATCH 0231/1170] 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 09e4faf2dbadfdf1eaedc0eec127098c6b9540f1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sun, 21 Jul 2024 15:05:26 +0000 Subject: [PATCH 0232/1170] fix --- src/Interpreters/InterpreterOptimizeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 3bee235185d..907a01b0432 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; - extern const int NOT_IMPLEMENTED; } From a564f70b66367ee4363d46e45eb5a9c66f131fec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:09:39 +0200 Subject: [PATCH 0233/1170] Fix error --- src/Parsers/ParserAlterQuery.cpp | 2 -- src/Parsers/ParserCreateIndexQuery.cpp | 4 ++-- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 28dbf781011..dbefb0cb966 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -9,8 +9,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 2fa34696c58..ddefb3d37fb 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -21,7 +21,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ParserToken close_p(TokenType::ClosingRoundBracket); ParserOrderByExpressionList order_list_p; - ParserDataType data_type_p; + ParserFunction type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -68,7 +68,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { - if (!data_type_p.parse(pos, type, expected)) + if (!type_p.parse(pos, type, expected)) return false; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index fa232954cd6..3dba58546af 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -179,7 +179,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_granularity(Keyword::GRANULARITY); ParserIdentifier name_p; - ParserDataType data_type_p; + ParserFunction type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -197,7 +197,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!s_type.ignore(pos, expected)) return false; - if (!data_type_p.parse(pos, type, expected)) + if (!type_p.parse(pos, type, expected)) return false; if (s_granularity.ignore(pos, expected)) From 8d7471f8bd2e0c6dc242231c4358448787e6c56f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 20 Jul 2024 00:03:40 +0200 Subject: [PATCH 0234/1170] Fix for deadlock in getDDLWorker --- src/Interpreters/Context.cpp | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 94bcb88ed53..48878733a00 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3490,18 +3490,22 @@ DDLWorker & Context::getDDLWorker() const if (shared->ddl_worker_startup_task) waitLoad(shared->ddl_worker_startup_task); // Just wait and do not prioritize, because it depends on all load and startup tasks - SharedLockGuard lock(shared->mutex); - if (!shared->ddl_worker) { - if (!hasZooKeeper()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); - - if (!hasDistributedDDL()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no DistributedDDL configuration in server config"); - - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "DDL background thread is not initialized"); + /// Only acquire the lock for reading ddl_worker field. + /// hasZooKeeper() and hasDistributedDDL() acquire the same lock as well and double acquisition of the lock in shared mode can lead + /// to a deadlock if an exclusive lock attempt is made in the meantime by another thread. + SharedLockGuard lock(shared->mutex); + if (shared->ddl_worker) + return *shared->ddl_worker; } - return *shared->ddl_worker; + + if (!hasZooKeeper()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); + + if (!hasDistributedDDL()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no DistributedDDL configuration in server config"); + + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "DDL background thread is not initialized"); } zkutil::ZooKeeperPtr Context::getZooKeeper() const From bbbf2fec88a331281bf51d3a02bd3f476e3bf6ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:37:54 +0200 Subject: [PATCH 0235/1170] Fix error --- src/Parsers/ParserCreateIndexQuery.cpp | 9 +++++++-- src/Parsers/ParserCreateQuery.cpp | 7 ++++++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index ddefb3d37fb..8a4c1c0b17a 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -7,9 +7,9 @@ #include #include #include -#include #include + namespace DB { @@ -69,7 +69,12 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { if (!type_p.parse(pos, type, expected)) - return false; + { + if (ParserIdentifier().parse(pos, type, expected)) + type = makeASTFunction(type->as().name()); + else + return false; + } } if (s_granularity.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 3dba58546af..bff5da4a536 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -198,7 +198,12 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!type_p.parse(pos, type, expected)) - return false; + { + if (name_p.parse(pos, type, expected)) + type = makeASTFunction(type->as().name()); + else + return false; + } if (s_granularity.ignore(pos, expected)) { From 885acd3aa80d421e82f75150b4152e227ca0fba4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 09:42:20 +0200 Subject: [PATCH 0236/1170] Compatibility --- src/Parsers/ParserCreateIndexQuery.cpp | 3 +++ src/Parsers/ParserCreateQuery.cpp | 3 +++ 2 files changed, 6 insertions(+) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 8a4c1c0b17a..2761c99738b 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -71,7 +71,10 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!type_p.parse(pos, type, expected)) { if (ParserIdentifier().parse(pos, type, expected)) + { type = makeASTFunction(type->as().name()); + type->as().no_empty_args = true; + } else return false; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index bff5da4a536..9aaecd84f59 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -200,7 +200,10 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!type_p.parse(pos, type, expected)) { if (name_p.parse(pos, type, expected)) + { type = makeASTFunction(type->as().name()); + type->as().no_empty_args = true; + } else return false; } From c2ac13291f3bf201f7189bd36f2c9be7c06aa886 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 09:06:13 +0100 Subject: [PATCH 0237/1170] fix tests --- tests/integration/test_recovery_time_metric/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 90155f81ba2..e4a44103b76 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -3,7 +3,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/config.xml"], with_zookeeper=True + "node", main_configs=["configs/config.xml"], with_zookeeper=True, stay_alive=True, ) From d040e436f3a1f8594070b04ec10cbf7391f6994a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Jul 2024 08:18:58 +0000 Subject: [PATCH 0238/1170] Automatic style fix --- tests/integration/test_recovery_time_metric/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index e4a44103b76..4dad844b950 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -3,7 +3,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/config.xml"], with_zookeeper=True, stay_alive=True, + "node", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, ) From bb0b29f6e50e74098fcc8a9b83150998f1bc2601 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 22 Jul 2024 12:11:56 +0200 Subject: [PATCH 0239/1170] Set writer_thread_id earlier, when new exclusive owener is waiting for existing readers to finish --- src/Common/SharedMutex.cpp | 6 ++++-- src/Common/SharedMutex.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/SharedMutex.cpp b/src/Common/SharedMutex.cpp index 7b00ef0b28b..2d63f8542b0 100644 --- a/src/Common/SharedMutex.cpp +++ b/src/Common/SharedMutex.cpp @@ -31,11 +31,13 @@ void SharedMutex::lock() break; } + /// The first step of acquiring the exclusive ownership is finished. + /// Now we just wait until all readers release the shared ownership. + writer_thread_id.store(getThreadId()); + value |= writers; while (value & readers) futexWaitLowerFetch(state, value); - - writer_thread_id.store(getThreadId()); } bool SharedMutex::try_lock() diff --git a/src/Common/SharedMutex.h b/src/Common/SharedMutex.h index c77c8765885..d2947645eca 100644 --- a/src/Common/SharedMutex.h +++ b/src/Common/SharedMutex.h @@ -38,7 +38,7 @@ private: alignas(64) std::atomic state; std::atomic waiters; - /// Is set while the lock is held in exclusive mode only to facilitate debugging + /// Is set while the lock is held (or is in the process of being acquired) in exclusive mode only to facilitate debugging std::atomic writer_thread_id; }; 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 0240/1170] 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 2dc264928f311e2f4d10001044d070b6a6a05471 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 11:33:51 +0000 Subject: [PATCH 0241/1170] Added tests, rewritten logic which engines and table functions to allow, added replace for create table ... AS table_function() syntax. --- docs/en/operations/settings/settings.md | 12 ++ src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.cpp | 3 +- src/Interpreters/InterpreterCreateQuery.cpp | 50 +++--- .../test_restore_external_engines/__init__.py | 0 .../configs/backups_disk.xml | 14 ++ .../configs/remote_servers.xml | 21 +++ .../test_restore_external_engines/test.py | 143 ++++++++++++++++++ 8 files changed, 217 insertions(+), 29 deletions(-) create mode 100644 tests/integration/test_restore_external_engines/__init__.py create mode 100644 tests/integration/test_restore_external_engines/configs/backups_disk.xml create mode 100644 tests/integration/test_restore_external_engines/configs/remote_servers.xml create mode 100644 tests/integration/test_restore_external_engines/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..65b8df7a9e2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,15 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## restore_replace_external_engines_to_null + +For testing purposes. Replaces all external engines to Null to not initiate external connections. + +Default value: `False` + +## restore_replace_external_table_functions_to_null + +For testing purposes. Replaces all external engines to Null to not initiate external connections. + +Default value: `False` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7bf97896357..e6d2cac359b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -891,7 +891,8 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ - M(Bool, restore_replace_external_engine_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ + M(Bool, restore_replace_external_engines_to_null, false, "Replace all the External table engines to Null on restore. Useful for testing purposes", 0) \ + M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a23d9d17da2..0abcfb0cfb9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,7 +79,8 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } + } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { + if (create.as_table_function) + { + if (getContext()->getSettingsRef().restore_replace_external_table_functions_to_null) + { + const auto & factory = TableFunctionFactory::instance(); + + auto properties = factory.tryGetProperties(create.as_table_function->as()->name); + if (properties && properties->allow_readonly) + return; + if (!create.storage) + { + auto storage_ast = std::make_shared(); + create.set(create.storage, storage_ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage should not be created yet, it's a bug."); + create.as_table_function = nullptr; + setNullTableEngine(*create.storage); + } return; + } if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) return; @@ -1010,34 +1031,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); /// For exrternal tables with restore_replace_external_engine_to_null setting we replace external engines to /// Null table engine. - else if (create.storage->engine->name == "AzureBlobStorage" || - create.storage->engine->name == "AzureQueue" || - create.storage->engine->name == "COSN" || - create.storage->engine->name == "DeltaLake" || - create.storage->engine->name == "Dictionary" || - create.storage->engine->name == "Executable" || - create.storage->engine->name == "ExecutablePool" || - create.storage->engine->name == "ExternalDistributed" || - create.storage->engine->name == "File" || - create.storage->engine->name == "Hudi" || - create.storage->engine->name == "Iceberg" || - create.storage->engine->name == "JDBC" || - create.storage->engine->name == "Kafka" || - create.storage->engine->name == "MaterializedPostgreSQL" || - create.storage->engine->name == "MongoDB" || - create.storage->engine->name == "MySQL" || - create.storage->engine->name == "NATS" || - create.storage->engine->name == "ODBC" || - create.storage->engine->name == "OSS" || - create.storage->engine->name == "PostgreSQL" || - create.storage->engine->name == "RabbitMQ" || - create.storage->engine->name == "Redis" || - create.storage->engine->name == "S3" || - create.storage->engine->name == "S3Queue" || - create.storage->engine->name == "TinyLog" || - create.storage->engine->name == "URL") + else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) { - if (getContext()->getSettingsRef().restore_replace_external_engine_to_null) + if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) setNullTableEngine(*create.storage); } return; diff --git a/tests/integration/test_restore_external_engines/__init__.py b/tests/integration/test_restore_external_engines/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_restore_external_engines/configs/backups_disk.xml b/tests/integration/test_restore_external_engines/configs/backups_disk.xml new file mode 100644 index 00000000000..f7d666c6542 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/backups_disk.xml @@ -0,0 +1,14 @@ + + + + + local + /backups/ + + + + + backups + /backups/ + + diff --git a/tests/integration/test_restore_external_engines/configs/remote_servers.xml b/tests/integration/test_restore_external_engines/configs/remote_servers.xml new file mode 100644 index 00000000000..76ad3618339 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + replica3 + 9000 + + + + + diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py new file mode 100644 index 00000000000..cde4b0deb00 --- /dev/null +++ b/tests/integration/test_restore_external_engines/test.py @@ -0,0 +1,143 @@ +import pytest + +import pymysql.cursors +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] + +node1 = cluster.add_instance("replica1", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node2 = cluster.add_instance("replica2", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node3 = cluster.add_instance("replica3", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +nodes = [node1, node2, node3] + +backup_id_counter = 0 + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}/')" + +def cleanup_nodes(nodes, dbname): + for node in nodes: + node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") + +def fill_nodes(nodes, dbname): + cleanup_nodes(nodes, dbname) + for node in nodes: + node.query(f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')") + +def drop_mysql_table(conn, tableName): + with conn.cursor() as cursor: + cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") + +def get_mysql_conn(cluster): + conn = pymysql.connect( + user="root", password="clickhouse", host=cluster.mysql8_ip, port=cluster.mysql8_port + ) + return conn + +def fill_tables(cluster, dbname): + fill_nodes(nodes, dbname) + + conn = get_mysql_conn(cluster) + + with conn.cursor() as cursor: + cursor.execute( + "DROP DATABASE IF EXISTS clickhouse" + ) + cursor.execute( + "CREATE DATABASE clickhouse" + ) + cursor.execute( + "DROP TABLE IF EXISTS clickhouse.inference_table" + ) + cursor.execute( + "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" + ) + cursor.execute( + "INSERT INTO clickhouse.inference_table VALUES (100, X'9fad5e9eefdfb449')" + ) + conn.commit() + + parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" + + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" + ) + node1.query(f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})") + + node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") + node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") + + expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" + assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") == expected + assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") == expected + assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") == "100\n" + assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + +def test_restore_table(start_cluster): + fill_tables(cluster, "replicated") + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + node2.query(f"BACKUP DATABASE replicated TO {backup_name}") + + node2.query("DROP TABLE replicated.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" + + node3.query(f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true") + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated;") + + assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine") == "1\t100\n" + assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function") == "1\t100\n" + assert node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + cleanup_nodes(nodes, "replicated") + + +def test_restore_table_null(start_cluster): + fill_tables(cluster, "replicated2") + + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") + + node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated2.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" + + node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engine_to_null=1, restore_replace_external_table_functions_to_null=1") + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") + + assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" + assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function") == "0\t0\n" + assert node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" + assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'") == "Null\nNull\n" + assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'") == "MergeTree\n" + cleanup_nodes(nodes, "replicated2") From af4c2fa8a405c53d7de6d9ed41d63988caf22b04 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Jul 2024 12:27:24 +0000 Subject: [PATCH 0242/1170] fix --- tests/integration/test_restore_external_engines/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py index cde4b0deb00..be2cae334e2 100644 --- a/tests/integration/test_restore_external_engines/test.py +++ b/tests/integration/test_restore_external_engines/test.py @@ -132,7 +132,7 @@ def test_restore_table_null(start_cluster): assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engine_to_null=1, restore_replace_external_table_functions_to_null=1") + node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1") node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2;") assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" 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 0243/1170] 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 ae5eccbf20b7198d6a3cc908e0186a384aba038a Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 22 Jul 2024 13:39:48 +0000 Subject: [PATCH 0244/1170] just a commit to trigger CI --- .../test_grant_and_revoke/test_without_table_engine_grant.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py index 210bb8ec465..4a5dfb83f79 100644 --- a/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_without_table_engine_grant.py @@ -60,6 +60,7 @@ def test_table_engine_and_source_grant(): ) # expecting grant POSTGRES instead of grant PostgreSQL due to discrepancy between source access type and table engine + # similarily, other sources should also use their own defined name instead of the name of table engine assert "grant POSTGRES ON *.*" in instance.query_and_get_error( """ CREATE TABLE test.table1(a Integer) From 85241b3b8e869879718d74c01ab5071a10d66c06 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Fri, 12 Jul 2024 18:35:01 +0200 Subject: [PATCH 0245/1170] Allow run query instantly in play Automatically execute the query after the page loads if the `play_now=true` parameter is present. By default, the query does not execute automatically. Reason: While it might be acceptable to click `Run` once or twice, it becomes tedious when using the play service frequently as a simple frontend to generate and open multiple links. This change eliminates the need to click `Run` every time. --- programs/server/play.html | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/programs/server/play.html b/programs/server/play.html index 9590a65524c..a4c01237abd 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -522,6 +522,9 @@ const current_url = new URL(window.location); const opened_locally = location.protocol == 'file:'; + /// Run query instantly after page is loaded + const play_now = current_url.searchParams.get("play_now"); + const server_address = current_url.searchParams.get('url'); if (server_address) { document.getElementById('url').value = server_address; @@ -599,6 +602,9 @@ const title = "ClickHouse Query: " + query; let history_url = window.location.pathname + '?user=' + encodeURIComponent(user); + if (play_now) { + history_url += "&play_now=" + encodeURIComponent(play_now); + } if (server_address != location.origin) { /// Save server's address in URL if it's not identical to the address of the play UI. history_url += '&url=' + encodeURIComponent(server_address); @@ -1160,6 +1166,10 @@ }); } + if (play_now === 'true') { + post(); + } + document.getElementById('toggle-light').onclick = function() { setColorTheme('light', true); } From 85afb1757e92b279dfc7171c409e09d5b6873e8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:21:35 +0200 Subject: [PATCH 0246/1170] Rename variables --- programs/server/play.html | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index a4c01237abd..b1da7408b58 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -523,7 +523,7 @@ const opened_locally = location.protocol == 'file:'; /// Run query instantly after page is loaded - const play_now = current_url.searchParams.get("play_now"); + const run_immediately = current_url.searchParams.get("run"); const server_address = current_url.searchParams.get('url'); if (server_address) { @@ -602,8 +602,8 @@ const title = "ClickHouse Query: " + query; let history_url = window.location.pathname + '?user=' + encodeURIComponent(user); - if (play_now) { - history_url += "&play_now=" + encodeURIComponent(play_now); + if (run_immediately) { + history_url += "&run=" + encodeURIComponent(run_immediately); } if (server_address != location.origin) { /// Save server's address in URL if it's not identical to the address of the play UI. @@ -1166,7 +1166,7 @@ }); } - if (play_now === 'true') { + if (run_immediately === 'true') { post(); } From 11aee643adc49cf14e73f3ea0a8fbdfe284dd2ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:23:18 +0200 Subject: [PATCH 0247/1170] Convert to bool --- programs/server/play.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index b1da7408b58..d4fc1446fa0 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -523,7 +523,7 @@ const opened_locally = location.protocol == 'file:'; /// Run query instantly after page is loaded - const run_immediately = current_url.searchParams.get("run"); + const run_immediately = !!current_url.searchParams.get("run"); const server_address = current_url.searchParams.get('url'); if (server_address) { @@ -603,7 +603,7 @@ let history_url = window.location.pathname + '?user=' + encodeURIComponent(user); if (run_immediately) { - history_url += "&run=" + encodeURIComponent(run_immediately); + history_url += "&run=" + run_immediately; } if (server_address != location.origin) { /// Save server's address in URL if it's not identical to the address of the play UI. From eb4cddd139c5ccdc43a3a0b8168220395303a276 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:28:39 +0200 Subject: [PATCH 0248/1170] Allow `run=1` to save space --- programs/server/play.html | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index d4fc1446fa0..eb861172c02 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -523,7 +523,7 @@ const opened_locally = location.protocol == 'file:'; /// Run query instantly after page is loaded - const run_immediately = !!current_url.searchParams.get("run"); + const run_immediately = current_url.searchParams.get("run") == 1; const server_address = current_url.searchParams.get('url'); if (server_address) { @@ -603,7 +603,7 @@ let history_url = window.location.pathname + '?user=' + encodeURIComponent(user); if (run_immediately) { - history_url += "&run=" + run_immediately; + history_url += "&run=" + (run_immediately ? 1 : 0); } if (server_address != location.origin) { /// Save server's address in URL if it's not identical to the address of the play UI. @@ -1166,7 +1166,7 @@ }); } - if (run_immediately === 'true') { + if (run_immediately) { post(); } From 02704bc5cc8213d6bd56183f613bda960a69b93a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:32:14 +0200 Subject: [PATCH 0249/1170] Different --- programs/server/play.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index eb861172c02..e30b4ac3450 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -522,8 +522,8 @@ const current_url = new URL(window.location); const opened_locally = location.protocol == 'file:'; - /// Run query instantly after page is loaded - const run_immediately = current_url.searchParams.get("run") == 1; + /// Run query instantly after page is loaded if the run parameter is present. + const run_immediately = !!current_url.searchParams.get("run"); const server_address = current_url.searchParams.get('url'); if (server_address) { From af53fd96a7a7adbbe6a6660a37d30b4acaf9fe40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:34:38 +0200 Subject: [PATCH 0250/1170] JavaScript Programming --- programs/server/play.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index e30b4ac3450..8f2fab36df4 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -523,7 +523,7 @@ const opened_locally = location.protocol == 'file:'; /// Run query instantly after page is loaded if the run parameter is present. - const run_immediately = !!current_url.searchParams.get("run"); + const run_immediately = current_url.searchParams.has("run"); const server_address = current_url.searchParams.get('url'); if (server_address) { From 0d2459f0de566dfa13eecf3bd176e59ec11e0239 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jul 2024 21:35:20 +0200 Subject: [PATCH 0251/1170] JavaScript --- programs/server/play.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index 8f2fab36df4..0d76a01cf7e 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -603,7 +603,7 @@ let history_url = window.location.pathname + '?user=' + encodeURIComponent(user); if (run_immediately) { - history_url += "&run=" + (run_immediately ? 1 : 0); + history_url += "&run=1"; } if (server_address != location.origin) { /// Save server's address in URL if it's not identical to the address of the play UI. From 5fa2db8e4828b004ac10d625df62efcc8711dc98 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 16:59:17 +0100 Subject: [PATCH 0252/1170] fix 01293_show_clusters stateless test --- tests/queries/0_stateless/01293_show_clusters.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01293_show_clusters.reference b/tests/queries/0_stateless/01293_show_clusters.reference index e140f207022..9569fcf2e37 100644 --- a/tests/queries/0_stateless/01293_show_clusters.reference +++ b/tests/queries/0_stateless/01293_show_clusters.reference @@ -1,3 +1,3 @@ test_shard_localhost -test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default -test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default +test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default 0 NULL +test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default 0 NULL From 2a893ed8cd10e6e3b7506f43b644b5037f96c49a Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 17:03:41 +0100 Subject: [PATCH 0253/1170] fxs --- src/IO/S3/Client.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index db20420db9f..325c820f8bd 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -388,7 +388,8 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const if (isClientForDisk()) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - return enrichErrorMessage(std::move(result)); + return enrichErrorMessage( + HeadObject(static_cast(request))); } /// For each request, we wrap the request functions from Aws::S3::Client with doRequest From 71cdf82643fb17b5b68003df314c54ebbca0842f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 Jul 2024 18:41:30 +0000 Subject: [PATCH 0254/1170] Fix: reset is_async_state flag --- src/Processors/Sources/RemoteSource.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 46c27676e12..2f9a30296be 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -115,7 +115,10 @@ void RemoteSource::onAsyncJobReady() if (!was_query_sent) return; + chassert(!preprocessed_packet); preprocessed_packet = query_executor->processParallelReplicaPacketIfAny(); + if (preprocessed_packet) + is_async_state = false; } std::optional RemoteSource::tryGenerate() From 7f5c58f599d34f690c4a04e4223a2f86a433d0e9 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Mon, 22 Jul 2024 20:58:56 +0100 Subject: [PATCH 0255/1170] fxs --- tests/queries/0_stateless/01293_show_clusters.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01293_show_clusters.reference b/tests/queries/0_stateless/01293_show_clusters.reference index 9569fcf2e37..e140f207022 100644 --- a/tests/queries/0_stateless/01293_show_clusters.reference +++ b/tests/queries/0_stateless/01293_show_clusters.reference @@ -1,3 +1,3 @@ test_shard_localhost -test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default 0 NULL -test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default 0 NULL +test_cluster_one_shard_two_replicas 1 1 0 1 127.0.0.1 127.0.0.1 9000 1 default +test_cluster_one_shard_two_replicas 1 1 0 2 127.0.0.2 127.0.0.2 9000 0 default From 72f4919fdad5217f48bd83e51ce2d1f3f083087b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mi=D1=81hael=20Stetsyuk?= <59827607+mstetsyuk@users.noreply.github.com> Date: Mon, 22 Jul 2024 21:33:47 +0100 Subject: [PATCH 0256/1170] Update src/Common/CurrentMetrics.cpp Co-authored-by: Sema Checherinda <104093494+CheSema@users.noreply.github.com> --- src/Common/CurrentMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2fedba0175b..39198147794 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -307,7 +307,7 @@ M(FilteringMarksWithPrimaryKey, "Number of threads currently doing filtering of mark ranges by the primary key") \ M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \ \ - M(S3DiskNoKeyErrors, "Number of no-key S3 disk errors") \ + M(S3DiskNoKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) From 223eee3f46b07c38de3223fb56575f9ecbc5bea7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 07:57:03 +0000 Subject: [PATCH 0257/1170] Comment to new IProcessor method --- src/Processors/IProcessor.h | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 0776921a814..94e93595f4e 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -221,6 +221,21 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'schedule' is not implemented for {} processor", getName()); } + /* The method is called right after asynchronous job is done + * i.e. when file descriptor returned by schedule() is readable. + * The sequence of method calls: + * ... prepare() -> schedule() -> onAsyncJobReady() -> work() ... + * See also comment to schedule() method + * + * It allows doing some preprocessing immediately after asynchronous job is done. + * The implementation should return control quickly, to avoid blocking another asynchronous completed jobs + * created by the same pipeline. + * + * Example, scheduling tasks for remote workers (file descriptor in this case is a socket) + * When the remote worker asks for the next task, doing it in onAsyncJobReady() we can provide it immediately. + * Otherwise, the returning of the next task for the remote worker can be delayed by current work done in the pipeline + * i.e. processor->work(), which will create unnecessary latency in query processing by remote workers Not Committed Yet + */ virtual void onAsyncJobReady() {} /** You must call this method if 'prepare' returned ExpandPipeline. From 932033fca9bdacbfdb544fac5389e03fa7732eeb Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 10:55:45 +0100 Subject: [PATCH 0258/1170] use atomic to avoid data race --- src/Databases/DatabaseReplicatedWorker.cpp | 12 +++++++++--- src/Databases/DatabaseReplicatedWorker.h | 5 +++-- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index cea2d123f87..a9a74c5f56a 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,7 +32,8 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { - initialization_duration_timer.emplace(); + initialization_duration_timer.restart(); + initializing.store(true, std::memory_order_release); while (!stop_flag) { @@ -71,7 +72,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - initialization_duration_timer.reset(); + initializing.store(false, std::memory_order_relaxed); return true; } catch (...) @@ -81,7 +82,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - initialization_duration_timer.reset(); + initializing.store(false, std::memory_order_relaxed); return false; } @@ -463,4 +464,9 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const return max_id.load(); } +UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const +{ + return initializing.load(std::memory_order_acquire) ? initialization_duration_timer.elapsedMilliseconds() : 0; +} + } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index aea3b71173d..3e5887be825 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -37,7 +37,7 @@ public: UInt32 getLogPointer() const; - UInt64 getCurrentInitializationDurationMs() const { return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; } + UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -59,7 +59,8 @@ private: /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; - std::optional initialization_duration_timer; + Stopwatch initialization_duration_timer; + std::atomic initializing = false; }; } From aaf603035e31874d6d5bcd024d0f4040715baa72 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 23 Jul 2024 13:35:37 +0100 Subject: [PATCH 0259/1170] check error type --- src/IO/S3/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 325c820f8bd..7196dfa9bdc 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -385,7 +385,7 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - if (isClientForDisk()) + if (isClientForDisk() && error.GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); return enrichErrorMessage( @@ -410,7 +410,7 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); - if (!resp.IsSuccess() && isClientForDisk()) + if (!resp.IsSuccess() && isClientForDisk() && resp.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); return enrichErrorMessage(std::move(resp)); From d74dc587d7a183225b7cf0846b85e8213dcb7fc0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 13:06:58 +0000 Subject: [PATCH 0260/1170] Fix comment --- src/Processors/IProcessor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 94e93595f4e..4fd00d5e164 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -234,7 +234,7 @@ public: * Example, scheduling tasks for remote workers (file descriptor in this case is a socket) * When the remote worker asks for the next task, doing it in onAsyncJobReady() we can provide it immediately. * Otherwise, the returning of the next task for the remote worker can be delayed by current work done in the pipeline - * i.e. processor->work(), which will create unnecessary latency in query processing by remote workers Not Committed Yet + * (by other processors), which will create unnecessary latency in query processing by remote workers */ virtual void onAsyncJobReady() {} From a9d7abbb928d43664f057dde88058c449a9a521c Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 23 Jul 2024 17:27:05 +0000 Subject: [PATCH 0261/1170] add drop option --- src/Core/SettingsEnums.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 9 +++++++++ .../03206_projection_merge_special_mergetree.sql | 12 ++++++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 74b6c793849..b53a882de4e 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -180,7 +180,7 @@ IMPLEMENT_SETTING_ENUM(LightweightMutationProjectionMode, ErrorCodes::BAD_ARGUME IMPLEMENT_SETTING_ENUM(DeduplicateMergeProjectionMode, ErrorCodes::BAD_ARGUMENTS, {{"throw", DeduplicateMergeProjectionMode::THROW}, {"drop", DeduplicateMergeProjectionMode::DROP}, - {"rebuild", DeduplicateMergeProjectionMode::THROW}}) + {"rebuild", DeduplicateMergeProjectionMode::REBUILD}}) IMPLEMENT_SETTING_AUTO_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index fc64fae9a58..0b358c0fd7c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -797,6 +797,15 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c } + const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; + /// Under throw mode, we still choose to drop projections due to backward compatibility since some + /// users might have projections before this change. + if (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP) + { + ctx->projections_iterator = ctx->tasks_for_projections.begin(); + return false; + } + const auto & projections = global_ctx->metadata_snapshot->getProjections(); for (const auto & projection : projections) diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index c8945fd784c..749f906569e 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -20,6 +20,18 @@ CREATE TABLE tp ( ) engine = ReplacingMergeTree order by type SETTINGS deduplicate_merge_projection_mode = 'drop'; +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +-- expecting no projection +SYSTEM FLUSH LOGS; +SELECT + name, + part_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'tp') AND (active = 1); + ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } From 9d55553225c4c5e253e32fb0de9944a2e29b7bcf Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Jul 2024 18:52:50 +0000 Subject: [PATCH 0262/1170] Forbid create as select even when database_replicated_allow_heavy_create is set --- src/Interpreters/InterpreterCreateQuery.cpp | 16 ++++++++++++---- ...ed_database_forbid_create_as_select.reference | 2 ++ ...eplicated_database_forbid_create_as_select.sh | 8 ++++++-- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ea10ad59db4..2f837fe4d2b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1329,8 +1329,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (need_add_to_database) database = DatabaseCatalog::instance().tryGetDatabase(database_name); - bool allow_heavy_create = getContext()->getSettingsRef().database_replicated_allow_heavy_create; - if (!allow_heavy_create && database && database->getEngineName() == "Replicated" && (create.select || create.is_populate)) + bool allow_heavy_populate = getContext()->getSettingsRef().database_replicated_allow_heavy_create && create.is_populate; + if (!allow_heavy_populate && database && database->getEngineName() == "Replicated" && (create.select || create.is_populate)) { bool is_storage_replicated = false; if (create.storage && create.storage->engine) @@ -1342,10 +1342,18 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) const bool allow_create_select_for_replicated = (create.isView() && !create.is_populate) || create.is_create_empty || !is_storage_replicated; if (!allow_create_select_for_replicated) + { + /// POPULATE can be enabled with setting, provide hint in error message + if (create.is_populate) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "CREATE with POPULATE is not supported with Replicated databases. Consider using separate CREATE and INSERT queries. " + "Alternatively, you can enable 'database_replicated_allow_heavy_create' setting to allow this operation, use with caution"); + throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, - "CREATE AS SELECT and POPULATE is not supported with Replicated databases. Consider using separate CREATE and INSERT queries. " - "Alternatively, you can enable 'database_replicated_allow_heavy_create' setting to allow this operation, use with caution"); + "CREATE AS SELECT is not supported with Replicated databases. Consider using separate CREATE and INSERT queries."); + } } if (database && database->shouldReplicateQuery(getContext(), query_ptr)) diff --git a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.reference b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.reference index 6ed281c757a..98fb6a68656 100644 --- a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.reference +++ b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.reference @@ -1,2 +1,4 @@ 1 1 +1 +1 diff --git a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh index 15f169d880f..b587549cb60 100755 --- a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh +++ b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh @@ -18,8 +18,12 @@ ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIAL ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" -# But it is allowed with the special setting -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" --database_replicated_allow_heavy_create=1 +# POPULATE is allowed with the special setting ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" --database_replicated_allow_heavy_create=1 +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv3 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" --compatibility='24.6' + +# AS SELECT is forbidden even with the setting +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" --database_replicated_allow_heavy_create=1 |& grep -cm1 "SUPPORT_IS_DISABLED" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" --compatibility='24.6' |& grep -cm1 "SUPPORT_IS_DISABLED" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" 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 0263/1170] 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 c850fac65276342e0b8694fa00d44dd3269d1abc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 09:05:06 +0200 Subject: [PATCH 0264/1170] Fix error --- src/Parsers/MySQL/tests/gtest_column_parser.cpp | 11 ++++++----- src/Parsers/ParserCreateQuery.h | 7 ++++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Parsers/MySQL/tests/gtest_column_parser.cpp b/src/Parsers/MySQL/tests/gtest_column_parser.cpp index 21c37e4ee2e..3a9a0690f06 100644 --- a/src/Parsers/MySQL/tests/gtest_column_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_column_parser.cpp @@ -1,13 +1,14 @@ #include #include #include -#include +#include #include #include #include #include #include + using namespace DB; using namespace DB::MySQLParser; @@ -19,8 +20,8 @@ TEST(ParserColumn, AllNonGeneratedColumnOption) "COLUMN_FORMAT FIXED STORAGE MEMORY REFERENCES tbl_name (col_01) CHECK 1"; ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0); EXPECT_EQ(ast->as()->name, "col_01"); - EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); - EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); ASTDeclareOptions * declare_options = ast->as()->column_options->as(); EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 0); @@ -44,8 +45,8 @@ TEST(ParserColumn, AllGeneratedColumnOption) "REFERENCES tbl_name (col_01) CHECK 1 GENERATED ALWAYS AS (1) STORED"; ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0, 0); EXPECT_EQ(ast->as()->name, "col_01"); - EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); - EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); ASTDeclareOptions * declare_options = ast->as()->column_options->as(); EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 1); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 7bd1d1bf588..53a62deb22b 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include + namespace DB { @@ -268,9 +270,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E auto default_function = std::make_shared(); default_function->name = "defaultValueOfTypeName"; default_function->arguments = std::make_shared(); - // Ephemeral columns don't really have secrets but we need to format - // into a String, hence the strange call - default_function->arguments->children.emplace_back(std::make_shared(type->as()->formatForLogging())); + /// Ephemeral columns don't really have secrets but we need to format into a String, hence the strange call + default_function->arguments->children.emplace_back(std::make_shared(type->as()->formatForLogging())); default_expression = default_function; } From c3204fb89577e50ec7ef2c7ddd3c62f913e084f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 09:25:22 +0200 Subject: [PATCH 0265/1170] Fix error --- src/Parsers/ExpressionElementParsers.cpp | 3 +-- src/Parsers/ExpressionListParsers.cpp | 17 +++++++++++++++++ src/Parsers/ExpressionListParsers.h | 10 ++++++++++ src/Parsers/ParserCreateIndexQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 5 files changed, 31 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index d4fc9a4bc4d..865d07faaa7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -9,8 +9,8 @@ #include #include #include -#include "Parsers/CommonParsers.h" +#include #include #include #include @@ -725,7 +725,6 @@ bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expect function_node->name = "STATISTICS"; function_node->arguments = stat_type; function_node->children.push_back(function_node->arguments); - node = function_node; return true; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f97c042e91e..66817fafa5e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2388,6 +2388,23 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } +bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserIdentifier id_p; + ParserFunction func_p; + + if (ParserFunction(false, false).parse(pos, node, expected)) + return true; + + if (ParserIdentifier().parse(pos, node, expected)) + { + node = makeASTFunction(node->as()->name()); + return true; + } + + return false; +} + const std::vector> ParserExpressionImpl::operators_table { {"->", Operator("lambda", 1, 2, OperatorType::Lambda)}, diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 235d5782630..6ab38416f32 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -144,6 +144,16 @@ protected: }; +/** Similar to ParserFunction (and yields ASTFunction), but can also parse identifiers without braces. + */ +class ParserExpressionWithOptionalArguments : public IParserBase +{ +protected: + const char * getName() const override { return "expression with optional parameters"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** An expression with an infix binary left-associative operator. * For example, a + b - c + d. */ diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 2761c99738b..b815ba60bab 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -21,7 +21,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ParserToken close_p(TokenType::ClosingRoundBracket); ParserOrderByExpressionList order_list_p; - ParserFunction type_p; + ParserExpressionWithOptionalArguments type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 92c0e7b2558..5da6c3a2510 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -180,7 +180,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_granularity(Keyword::GRANULARITY); ParserIdentifier name_p; - ParserFunction type_p; + ParserExpressionWithOptionalArguments type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; @@ -240,7 +240,7 @@ bool ParserStatisticsDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_type(Keyword::TYPE); ParserList columns_p(std::make_unique(), std::make_unique(TokenType::Comma), false); - ParserList types_p(std::make_unique(), std::make_unique(TokenType::Comma), false); + ParserList types_p(std::make_unique(), std::make_unique(TokenType::Comma), false); ASTPtr columns; ASTPtr types; From 284c0204b06f9ecd21b6eb361097b7d986342bd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:11:50 +0200 Subject: [PATCH 0266/1170] 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 5fe78d47bc855867f6431ad06e019b3e0278d0ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:24:13 +0200 Subject: [PATCH 0267/1170] Compatibility --- src/Parsers/ExpressionListParsers.cpp | 1 + src/Parsers/ParserCreateIndexQuery.cpp | 10 +--------- src/Parsers/ParserCreateQuery.cpp | 10 +--------- 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 66817fafa5e..a9715cec81e 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2399,6 +2399,7 @@ bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, if (ParserIdentifier().parse(pos, node, expected)) { node = makeASTFunction(node->as()->name()); + node->as().no_empty_args = true; return true; } diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index b815ba60bab..9ebee4cc852 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -69,15 +69,7 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (s_type.ignore(pos, expected)) { if (!type_p.parse(pos, type, expected)) - { - if (ParserIdentifier().parse(pos, type, expected)) - { - type = makeASTFunction(type->as().name()); - type->as().no_empty_args = true; - } - else - return false; - } + return false; } if (s_granularity.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 5da6c3a2510..bf5523152ac 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -199,15 +199,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; if (!type_p.parse(pos, type, expected)) - { - if (name_p.parse(pos, type, expected)) - { - type = makeASTFunction(type->as().name()); - type->as().no_empty_args = true; - } - else - return false; - } + return false; if (s_granularity.ignore(pos, expected)) { From e5bb485a006d93a9e00736dc37ad90a0a0a47673 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:25:23 +0200 Subject: [PATCH 0268/1170] Compatibility --- src/Parsers/ExpressionListParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a9715cec81e..d38dc6d5f37 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2399,7 +2399,7 @@ bool ParserExpressionWithOptionalArguments::parseImpl(Pos & pos, ASTPtr & node, if (ParserIdentifier().parse(pos, node, expected)) { node = makeASTFunction(node->as()->name()); - node->as().no_empty_args = true; + node->as().no_empty_args = true; return true; } From 73fc5c266f3bc254db3882bfa2f9f42db6b2bc87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:37:19 +0200 Subject: [PATCH 0269/1170] Fix error --- src/DataTypes/DataTypeObject.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 5636a46373f..91b9bfcb2a5 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -4,9 +4,10 @@ #include #include -#include +#include #include + namespace DB { @@ -53,13 +54,13 @@ static DataTypePtr create(const ASTPtr & arguments) ASTPtr schema_argument = arguments->children[0]; bool is_nullable = false; - if (const auto * func = schema_argument->as()) + if (const auto * type = schema_argument->as()) { - if (func->name != "Nullable" || func->arguments->children.size() != 1) + if (type->name != "Nullable" || type->arguments->children.size() != 1) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, - "Expected 'Nullable()' as parameter for type Object (function: {})", func->name); + "Expected 'Nullable()' as parameter for type Object (function: {})", type->name); - schema_argument = func->arguments->children[0]; + schema_argument = type->arguments->children[0]; is_nullable = true; } From b25cad23ed3b90dc8c0903710dba0714bac7219c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 11:42:28 +0000 Subject: [PATCH 0270/1170] Use unique names for tables and files --- .../integration/test_storage_s3_queue/test.py | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index bf3c28c5429..cf24e91f36b 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -7,6 +7,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance import json +from uuid import uuid4 AVAILABLE_MODES = ["unordered", "ordered"] @@ -822,7 +823,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] - table_name = f"max_set_age" + table_name = f"max_set_age_{uuid4().hex}" dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" @@ -847,11 +848,11 @@ def test_max_set_age(started_cluster): ) create_mv(node, table_name, dst_table_name) - total_values = generate_random_files( + _ = generate_random_files( started_cluster, files_path, files_to_generate, row_num=1 ) - expected_rows = 10 + expected_rows = files_to_generate node.wait_for_log_line("Checking node limits") node.wait_for_log_line("Node limits check finished") @@ -865,11 +866,11 @@ def test_max_set_age(started_cluster): time.sleep(1) assert expected_rows == get_count() - assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) time.sleep(max_age + 5) - expected_rows = 20 + expected_rows *= 2 for _ in range(20): if expected_rows == get_count(): @@ -877,7 +878,7 @@ def test_max_set_age(started_cluster): time.sleep(1) assert expected_rows == get_count() - assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) paths_count = [ int(x) @@ -885,7 +886,7 @@ def test_max_set_age(started_cluster): f"SELECT count() from {dst_table_name} GROUP BY _path" ).splitlines() ] - assert 10 == len(paths_count) + assert files_to_generate == len(paths_count) for path_count in paths_count: assert 2 == path_count @@ -901,7 +902,8 @@ def test_max_set_age(started_cluster): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - put_s3_file_content(started_cluster, f"{files_path}/fff.csv", values_csv) + file_with_error = f"fff_{uuid4().hex}.csv" + put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) for _ in range(30): if failed_count + 1 == int( @@ -920,16 +922,17 @@ def test_max_set_age(started_cluster): node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( - "SELECT exception FROM system.s3queue WHERE file_name ilike '%fff.csv'" + f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}'" ) + assert 1 == int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv'" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}'" ) ) assert 1 == int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv' AND notEmpty(exception)" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" ) ) @@ -943,11 +946,11 @@ def test_max_set_age(started_cluster): node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( - "SELECT exception FROM system.s3queue WHERE file_name ilike '%fff.csv' ORDER BY processing_end_time DESC LIMIT 1" + f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}' ORDER BY processing_end_time DESC LIMIT 1" ) assert 1 < int( node.query( - "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv' AND notEmpty(exception)" + f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" ) ) From a3f7642d05a730dac2d5030e22947598da70cb72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:01:02 +0200 Subject: [PATCH 0271/1170] 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 91b7001df6e827f801bd792e7bd9d96cdd947946 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 12:08:31 +0000 Subject: [PATCH 0272/1170] Refactor test to improve it - Create wait_for_condition that checks greedily for a period of time - Remove redundant checks - Allow other tests running in parallel to have `ObjectStorageQueueFailedFiles` errors --- .../integration/test_storage_s3_queue/test.py | 56 ++++++------------- 1 file changed, 16 insertions(+), 40 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index cf24e91f36b..e178b3b6608 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -860,24 +860,21 @@ def test_max_set_age(started_cluster): def get_count(): return int(node.query(f"SELECT count() FROM {dst_table_name}")) - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) + def wait_for_condition(check_function, max_wait_time=30): + before = time.time() + while time.time() - before < max_wait_time: + if check_function(): + return + time.sleep(0.1) + assert False - assert expected_rows == get_count() + wait_for_condition(lambda: get_count() == expected_rows) assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) time.sleep(max_age + 5) expected_rows *= 2 - - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) - - assert expected_rows == get_count() + wait_for_condition(lambda: get_count() == expected_rows) assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) paths_count = [ @@ -890,11 +887,12 @@ def test_max_set_age(started_cluster): for path_count in paths_count: assert 2 == path_count - failed_count = int( - node.query( + def get_object_storage_failures(): + return int(node.query( "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ) + )) + + failed_count = get_object_storage_failures() values = [ ["failed", 1, 1], @@ -905,31 +903,13 @@ def test_max_set_age(started_cluster): file_with_error = f"fff_{uuid4().hex}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) - for _ in range(30): - if failed_count + 1 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ): - break - time.sleep(1) - - assert failed_count + 1 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - ) - ) + wait_for_condition(lambda: failed_count + 1 <= get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( f"SELECT exception FROM system.s3queue WHERE file_name ilike '%{file_with_error}'" ) - assert 1 == int( - node.query( - f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}'" - ) - ) assert 1 == int( node.query( f"SELECT count() FROM system.s3queue_log WHERE file_name ilike '%{file_with_error}' AND notEmpty(exception)" @@ -938,11 +918,7 @@ def test_max_set_age(started_cluster): time.sleep(max_age + 1) - assert failed_count + 2 == int( - node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles'" - ) - ) + assert failed_count + 2 <= get_object_storage_failures() node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( From 01ce22049a76995dc00974618c94af9ccbcc30db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:29:14 +0200 Subject: [PATCH 0273/1170] Fix tests --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 11 +++++------ .../MySQL/tests/gtest_create_rewritten.cpp | 6 ++---- .../test_postgresql_replica_database_engine_2/test.py | 2 +- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index f73965cfcc8..3917ffb8823 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -17,7 +17,6 @@ #include #include -#include #include #include #include @@ -158,7 +157,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col /// (see git blame for details). auto column_name_and_type = columns_name_and_type.begin(); const auto * declare_column_ast = columns_definition->children.begin(); - for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++) + for (; column_name_and_type != columns_name_and_type.end(); ++column_name_and_type, ++declare_column_ast) { const auto & declare_column = (*declare_column_ast)->as(); String comment; @@ -177,7 +176,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col return columns_description; } -static NamesAndTypesList getNames(const ASTDataType & expr, ContextPtr context, const NamesAndTypesList & columns) +static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context, const NamesAndTypesList & columns) { if (expr.arguments->children.empty()) return NamesAndTypesList{}; @@ -221,9 +220,9 @@ static std::tuplechildren.empty()) { diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 6d6077a0295..81e6e6a8761 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -2,12 +2,10 @@ #include -#include #include #include #include #include -#include #include #include #include @@ -26,8 +24,8 @@ static inline ASTPtr tryRewrittenCreateQuery(const String & query, ContextPtr co context, "test_database", "test_database")[0]; } -static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8() MATERIALIZED 1" - ", `_version` UInt64() MATERIALIZED 1" +static const char MATERIALIZEDMYSQL_TABLE_COLUMNS[] = ", `_sign` Int8 MATERIALIZED 1" + ", `_version` UInt64 MATERIALIZED 1" ", INDEX _version _version TYPE minmax GRANULARITY 1"; TEST(MySQLCreateRewritten, ColumnsDataType) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 5e04c9e4d12..406b50bc486 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -654,7 +654,7 @@ def test_table_override(started_cluster): instance.query(f"SELECT count() FROM {materialized_database}.{table_name}") ) - expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` String,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nPARTITION BY key\\nORDER BY tuple(key)" + expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` String,\\n `_sign` Int8 MATERIALIZED 1,\\n `_version` UInt64 MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nPARTITION BY key\\nORDER BY tuple(key)" assert ( expected == instance.query( From 6efd29144558ded7fb95b36c6c19ee50aee0071f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:37:05 +0200 Subject: [PATCH 0274/1170] Add a test --- .../03210_inconsistent_formatting_of_data_types.reference | 1 + .../03210_inconsistent_formatting_of_data_types.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference create mode 100755 tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference new file mode 100644 index 00000000000..ccb445a0573 --- /dev/null +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference @@ -0,0 +1 @@ +ALTER TABLE columns_with_multiple_streams MODIFY COLUMN `field1` Nullable(tupleElement(x, 2), UInt8) diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh new file mode 100755 index 00000000000..6cb2d083d71 --- /dev/null +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field1 Nullable(tupleElement(x, 2), UInt8)" | $CLICKHOUSE_FORMAT --oneline From 3631bf48729bf40401bb607ba9b54f8f1445860b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:45:58 +0200 Subject: [PATCH 0275/1170] Fix tidy --- 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 5879fd50872..b96de397e96 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -743,7 +743,7 @@ void LocalServer::processConfig() DatabasePtr database = createClickHouseLocalDatabaseOverlay(default_database, global_context); if (UUID uuid = database->getUUID(); uuid != UUIDHelpers::Nil) DatabaseCatalog::instance().addUUIDMapping(uuid); - DatabaseCatalog::instance().attachDatabase(default_database, std::move(database)); + DatabaseCatalog::instance().attachDatabase(default_database, database); } global_context->setCurrentDatabase(default_database); From f03d4bb7d5d40203bba68c4f8958d584f27ae881 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 24 Jul 2024 15:07:53 +0000 Subject: [PATCH 0276/1170] Format with black --- .../integration/test_storage_s3_queue/test.py | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index e178b3b6608..4348857acd3 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -823,7 +823,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] - table_name = f"max_set_age_{uuid4().hex}" + table_name = "max_set_age" dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" @@ -848,9 +848,7 @@ def test_max_set_age(started_cluster): ) create_mv(node, table_name, dst_table_name) - _ = generate_random_files( - started_cluster, files_path, files_to_generate, row_num=1 - ) + _ = generate_random_files(started_cluster, files_path, files_to_generate, row_num=1) expected_rows = files_to_generate @@ -869,13 +867,17 @@ def test_max_set_age(started_cluster): assert False wait_for_condition(lambda: get_count() == expected_rows) - assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int( + node.query(f"SELECT uniq(_path) from {dst_table_name}") + ) time.sleep(max_age + 5) expected_rows *= 2 wait_for_condition(lambda: get_count() == expected_rows) - assert files_to_generate == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + assert files_to_generate == int( + node.query(f"SELECT uniq(_path) from {dst_table_name}") + ) paths_count = [ int(x) @@ -888,9 +890,11 @@ def test_max_set_age(started_cluster): assert 2 == path_count def get_object_storage_failures(): - return int(node.query( - "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" - )) + return int( + node.query( + "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" + ) + ) failed_count = get_object_storage_failures() @@ -900,6 +904,8 @@ def test_max_set_age(started_cluster): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() + + # use a different filename for each test to allow running a bunch of them sequentially with --count file_with_error = f"fff_{uuid4().hex}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) From 689e31b47e1c85f1ae9721b3928de658eaf9a6ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 18:31:40 +0200 Subject: [PATCH 0277/1170] More tests --- ...03210_inconsistent_formatting_of_data_types.reference | 6 ++++++ .../03210_inconsistent_formatting_of_data_types.sh | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference index ccb445a0573..836b526905a 100644 --- a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.reference @@ -1 +1,7 @@ ALTER TABLE columns_with_multiple_streams MODIFY COLUMN `field1` Nullable(tupleElement(x, 2), UInt8) +ALTER TABLE t_update_empty_nested ADD COLUMN `nested.arr2` Array(tuple('- ON NULL -', toLowCardinality(11), 11, 11, toLowCardinality(11), 11), UInt64) +ALTER TABLE t ADD COLUMN `x` Array(tuple(1), UInt8) +ALTER TABLE enum_alter_issue MODIFY COLUMN `a` Enum8(equals('one', timeSlots(timeSlots(arrayEnumerateDense(tuple('0.2147483646', toLowCardinality(toUInt128)), NULL), 4, 12.34, materialize(73), 2)), 1)) +ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN `s` Tuple(Nullable(tupleElement(s, 1), UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String)) +Syntax error +Syntax error diff --git a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh index 6cb2d083d71..86c7a5469ca 100755 --- a/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh +++ b/tests/queries/0_stateless/03210_inconsistent_formatting_of_data_types.sh @@ -4,4 +4,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# Ensure that these (possibly incorrect) queries can at least be parsed back after formatting. $CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field1 Nullable(tupleElement(x, 2), UInt8)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t_update_empty_nested ADD COLUMN \`nested.arr2\` Array(tuple('- ON NULL -', toLowCardinality(11), 11, 11, toLowCardinality(11), 11), UInt64)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t ADD COLUMN x Array((1), UInt8)" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE enum_alter_issue (MODIFY COLUMN a Enum8(equals('one', timeSlots(timeSlots(arrayEnumerateDense(tuple('0.2147483646', toLowCardinality(toUInt128(12))), NULL), 4, 12.34, materialize(73), 2)), 1)))" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN s Tuple(Nullable(tupleElement(s, 1), UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String))" | $CLICKHOUSE_FORMAT --oneline + +# These invalid queries don't parse and this is normal. +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC((2 + ignore(1, toUInt128(materialize(2)), 2 + toNullable(toNullable(3))), 3), NONE)" 2>&1 | grep -o -F 'Syntax error' +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE test_table ADD COLUMN \`array\` Array(('110', 3, toLowCardinality(3), 3, toNullable(3), toLowCardinality(toNullable(3)), 3), UInt8) DEFAULT [1, 2, 3]" 2>&1 | grep -o -F 'Syntax error' 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 0278/1170] 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 0279/1170] 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 e4b50c18c2c1918905bf44a8e1183f0cddd5a811 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Jul 2024 22:26:46 +0200 Subject: [PATCH 0280/1170] getauxval: Avoid crash under sanitizer re-exec due to high ASLR entropy --- base/glibc-compatibility/musl/getauxval.c | 38 +++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index ea5cff9fc11..86f9a546ee4 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -75,6 +75,44 @@ unsigned long NO_SANITIZE_THREAD __getauxval_procfs(unsigned long type) } static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) { +#if defined(__x86_64__) && defined(__has_feature) +# if __has_feature(memory_sanitizer) || __has_feature(thread_sanitizer) + /// Sanitizers are not compatible with high ASLR entropy, which is the default on modern Linux distributions, and + /// to workaround this limitation, TSAN and MSAN (couldn't see other sanitizers doing the same), re-exec the binary + /// without ASLR (see https://github.com/llvm/llvm-project/commit/0784b1eefa36d4acbb0dacd2d18796e26313b6c5) + + /// The problem we face is that, in order to re-exec, the sanitizer wants to use the original pathname in the call + /// and to get its value it uses getauxval (https://github.com/llvm/llvm-project/blob/20eff684203287828d6722fc860b9d3621429542/compiler-rt/lib/sanitizer_common/sanitizer_linux_libcdep.cpp#L985-L988). + /// Since we provide getauxval ourselves (to minimize the version dependency on runtime glibc), we are the ones + // being called and we fail horribly: + /// + /// ==301455==ERROR: MemorySanitizer: SEGV on unknown address 0x2ffc6d721550 (pc 0x5622c1cc0073 bp 0x000000000003 sp 0x7ffc6d721530 T301455) + /// ==301455==The signal is caused by a WRITE memory access. + /// #0 0x5622c1cc0073 in __auxv_init_procfs ./ClickHouse/base/glibc-compatibility/musl/getauxval.c:129:5 + /// #1 0x5622c1cbffe9 in getauxval ./ClickHouse/base/glibc-compatibility/musl/getauxval.c:240:12 + /// #2 0x5622c0d7bfb4 in __sanitizer::ReExec() crtstuff.c + /// #3 0x5622c0df7bfc in __msan::InitShadowWithReExec(bool) crtstuff.c + /// #4 0x5622c0d95356 in __msan_init (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x256356) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + /// #5 0x5622c0dfe878 in msan.module_ctor main.cc + /// #6 0x5622c1cc156c in __libc_csu_init (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x118256c) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + /// #7 0x73dc05dd7ea3 in __libc_start_main /usr/src/debug/glibc/glibc/csu/../csu/libc-start.c:343:6 + /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) + + /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as + /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliar vector. + /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very + /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise + // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). + + /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and it we used + /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and + /// unconditionally return "/proc/self/exe" without any preparation. Theoretically this should be fine in + /// our case, as we don't load any libraries. That's the theory at least. + if (type == AT_EXECFN) + return (unsigned long)"/proc/self/exe"; +# endif +#endif + // For debugging: // - od -t dL /proc/self/auxv // - LD_SHOW_AUX= ls From fda11dc62d81b717b9ab06c8adc8554c827764bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Jul 2024 22:51:26 +0200 Subject: [PATCH 0281/1170] Typo --- base/glibc-compatibility/musl/getauxval.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 86f9a546ee4..b5bd2f114c2 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -99,7 +99,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as - /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliar vector. + /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). From c847d2f63fdacf1ce5d636a9af8812d543547cfe Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 24 Jul 2024 20:52:35 +0000 Subject: [PATCH 0282/1170] fix --- src/Interpreters/InterpreterCreateQuery.cpp | 25 ++++++++++++--------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 05df26b0d31..a5f374ba71c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1282,24 +1282,27 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); /// Projection is only supported in (Replictaed)MergeTree. - if (std::string_view engine_name(create.storage->engine->name); - !properties.projections.empty() && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") + if (create.storage && create.storage->engine) { - bool projection_support = false; - if (auto * setting = create.storage->settings; setting != nullptr) + if (std::string_view engine_name(create.storage->engine->name); + !properties.projections.empty() && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") { - for (const auto & change : setting->changes) + bool projection_support = false; + if (auto * setting = create.storage->settings; setting != nullptr) { - if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) + for (const auto & change : setting->changes) { - projection_support = true; - break; + if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) + { + projection_support = true; + break; + } } } + if (!projection_support) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Projection is only supported in (Replictaed)MergeTree. Consider drop or rebuild option of deduplicate_merge_projection_mode."); } - if (!projection_support) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Projection is only supported in (Replictaed)MergeTree. Consider drop or rebuild option of deduplicate_merge_projection_mode."); } /// Check type compatible for materialized dest table and select columns From 23c3fa73266cb9f32c7f30a2c83437815de89291 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 25 Jul 2024 01:03:59 +0000 Subject: [PATCH 0283/1170] fix --- .../01710_aggregate_projection_with_normalized_states.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 e023c0991b3..5375823aa8e 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 @@ -9,7 +9,8 @@ CREATE TABLE r ( s Int64, PROJECTION p (SELECT a, quantilesTimingMerge(0.5, 0.95, 0.99)(q), sum(s) GROUP BY a) -) Engine=SummingMergeTree order by (x, a); +) Engine=SummingMergeTree order by (x, a) +SETTINGS deduplicate_merge_projection_mode = 'drop'; -- should set it to rebuild once projection is supported with SummingMergeTree insert into r select number%100 x, From 7612060d232a24dbd721597c8e33cd1f556cddd6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 25 Jul 2024 06:40:51 +0000 Subject: [PATCH 0284/1170] allow only equal types in lagInFrame and leadInFrame --- src/Processors/Transforms/WindowTransform.cpp | 15 ++------------ .../03210_lag_lead_inframe_types.reference | 20 +++++++++++++++++++ .../03210_lag_lead_inframe_types.sql | 4 ++++ 3 files changed, 26 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03210_lag_lead_inframe_types.reference create mode 100644 tests/queries/0_stateless/03210_lag_lead_inframe_types.sql diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 86421adf4fb..06ae2bfb25e 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2385,22 +2385,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction return; } - const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); - if (!supertype) - { + if (!argument_types[0]->equals(*argument_types[2])) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "There is no supertype for the argument type '{}' and the default value type '{}'", + "Argument type '{}' and the default value type '{}' are different", argument_types[0]->getName(), argument_types[2]->getName()); - } - if (!argument_types[0]->equals(*supertype)) - { - 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(), - argument_types[2]->getName()); - } if (argument_types.size() > 3) { diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference new file mode 100644 index 00000000000..cc3b9a096b9 --- /dev/null +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference @@ -0,0 +1,20 @@ +0 +1 +2 +2 +2 +2 +2 +2 +2 +2 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql new file mode 100644 index 00000000000..5466cfe0fad --- /dev/null +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql @@ -0,0 +1,4 @@ +SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); +SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); From c2f85c6fd062dde095ee34178450dc94c245e691 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:43:02 +0800 Subject: [PATCH 0285/1170] support map type as first argument type --- .../functions/tuple-map-functions.md | 2 +- src/Functions/map.cpp | 131 +++++++++++------- .../0_stateless/01651_map_functions.reference | 3 + .../0_stateless/01651_map_functions.sql | 9 +- 4 files changed, 89 insertions(+), 56 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 24b356eca87..ae23387f6e5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 66cd10a3f0b..5319390fb70 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -1,14 +1,17 @@ -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include @@ -178,22 +181,28 @@ public: getName(), arguments.size()); - /// The first argument should always be Array. - /// Because key type can not be nested type of Map, which is Tuple - DataTypePtr key_type; - if (const auto * keys_type = checkAndGetDataType(arguments[0].get())) - key_type = keys_type->getNestedType(); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); + auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + { + DataTypePtr nested; + if (const auto * array_type = checkAndGetDataType(type.get())) + nested = array_type->getNestedType(); + else if (const auto * map_type = checkAndGetDataType(type.get())) + nested = std::make_shared(map_type->getKeyValueTypes()); + else + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument types of function {} must be Array or Map, but {} is given", + getName(), + type->getName()); - DataTypePtr value_type; - if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) - value_type = value_array_type->getNestedType(); - else if (const auto * value_map_type = checkAndGetDataType(arguments[1].get())) - value_type = std::make_shared(value_map_type->getKeyValueTypes()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName()); + return nested; + }; + auto key_type = get_nested_type(arguments[0]); + auto value_type = get_nested_type(arguments[1]); + + /// Remove Nullable from key_type if needed for map key must not be Nullable + key_type = removeNullableOrLowCardinalityNullable(key_type); DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -201,44 +210,62 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - bool is_keys_const = isColumnConst(*arguments[0].column); - ColumnPtr holder_keys; - const ColumnArray * col_keys; - if (is_keys_const) + auto get_array_column = [this](const ColumnPtr & column) -> std::pair { - holder_keys = arguments[0].column->convertToFullColumnIfConst(); - col_keys = checkAndGetColumn(holder_keys.get()); - } - else + bool is_const = isColumnConst(*column); + ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; + + const ColumnArray * col_res = nullptr; + if (const auto * col_array = checkAndGetColumn(holder.get())) + col_res = col_array; + else if (const auto * col_map = checkAndGetColumn(holder.get())) + col_res = &col_map->getNestedColumn(); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument columns of function {} must be Array or Map, but {} is given", + getName(), + holder->getName()); + + return {col_res, holder}; + }; + + auto [col_keys, key_holder] = get_array_column(arguments[0].column); + + /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + ColumnPtr data_keys = col_keys->getDataPtr(); + if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - col_keys = checkAndGetColumn(arguments[0].column.get()); + std::cout << "data keys is nullable" << std::endl; + const NullMap * null_map = nullptr; + if (const auto * nullable = checkAndGetColumn(data_keys.get())) + { + null_map = &nullable->getNullMapData(); + data_keys = nullable->getNestedColumnPtr(); + } + else if (const auto * low_cardinality = checkAndGetColumn(data_keys.get())) + { + if (const auto * nullable_dict = checkAndGetColumn(low_cardinality->getDictionaryPtr().get())) + { + null_map = &nullable_dict->getNullMapData(); + data_keys = ColumnLowCardinality::create(nullable_dict->getNestedColumnPtr(), low_cardinality->getIndexesPtr()); + } + } + + if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); } - if (!col_keys) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName()); - - bool is_values_const = isColumnConst(*arguments[1].column); - ColumnPtr holder_values; - if (is_values_const) - holder_values = arguments[1].column->convertToFullColumnIfConst(); - else - holder_values = arguments[1].column; - - const ColumnArray * col_values; - if (const auto * col_values_array = checkAndGetColumn(holder_values.get())) - col_values = col_values_array; - else if (const auto * col_values_map = checkAndGetColumn(holder_values.get())) - col_values = &col_values_map->getNestedColumn(); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName()); - + auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); - const auto & data_keys = col_keys->getDataPtr(); const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets); + std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; + auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); + std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 471da5586b7..9114aa419b1 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -52,3 +52,6 @@ {1:4,2:5} {1:4,2:5} {1:4,2:5} +{1:3,2:4} +{1:3,2:4} +{1:3,2:4} {(1,3):'a',(2,4):'b'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index cf2460fce2c..4604ddd6db1 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -67,12 +67,15 @@ select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} +select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); -select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], [4, 5]); select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, 5])) from numbers(2); + +select mapFromArrays([1,2], [3,4]); +select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); +select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); From 65573871485c2e8ca45d791551856fd2f8622cf9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 11:05:36 +0200 Subject: [PATCH 0286/1170] Try calcualting memory with numactl if membind used --- .gitmodules | 3 + base/base/CMakeLists.txt | 4 ++ base/base/getMemoryAmount.cpp | 26 +++++++- contrib/CMakeLists.txt | 2 + contrib/numactl | 1 + contrib/numactl-cmake/CMakeLists.txt | 20 +++++++ contrib/numactl-cmake/include/config.h | 82 ++++++++++++++++++++++++++ programs/server/Server.cpp | 27 +++++++++ src/Common/config.h.in | 1 + src/configure_config.cmake | 3 + 10 files changed, 168 insertions(+), 1 deletion(-) create mode 160000 contrib/numactl create mode 100644 contrib/numactl-cmake/CMakeLists.txt create mode 100644 contrib/numactl-cmake/include/config.h diff --git a/.gitmodules b/.gitmodules index 12d865307d8..b5d7e1e56b3 100644 --- a/.gitmodules +++ b/.gitmodules @@ -372,3 +372,6 @@ [submodule "contrib/double-conversion"] path = contrib/double-conversion url = https://github.com/ClickHouse/double-conversion.git +[submodule "contrib/numactl"] + path = contrib/numactl + url = https://github.com/numactl/numactl.git diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 159502c9735..451a6eb5e8b 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -46,6 +46,10 @@ if (TARGET ch_contrib::crc32_s390x) target_link_libraries(common PUBLIC ch_contrib::crc32_s390x) endif() +if (TARGET ch_contrib::numactl) + target_link_libraries(common PUBLIC ch_contrib::numactl) +endif() + target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..") target_link_libraries (common diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index afdb6ba068a..b8162146496 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -4,12 +4,17 @@ #include #include -#include #include #include #include +#include "config.h" + +#if USE_NUMACTL +#include +#endif + namespace { @@ -63,6 +68,25 @@ uint64_t getMemoryAmountOrZero() uint64_t memory_amount = num_pages * page_size; +#if USE_NUMACTL + if (numa_available() != -1) + { + auto * membind = numa_get_membind(); + if (!numa_bitmask_equal(membind, numa_all_nodes_ptr)) + { + uint64_t total_numa_memory = 0; + auto max_node = numa_max_node(); + for (int i = 0; i <= max_node; ++i) + { + if (numa_bitmask_isbitset(membind, i)) + total_numa_memory += numa_node_size(i, nullptr); + } + + memory_amount = total_numa_memory; + } + } +#endif + /// Respect the memory limit set by cgroups v2. auto limit_v2 = getCgroupsV2MemoryLimit(); if (limit_v2.has_value() && *limit_v2 < memory_amount) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 90ae5981a21..977efda15ff 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -230,6 +230,8 @@ add_contrib (libssh-cmake libssh) add_contrib (prometheus-protobufs-cmake prometheus-protobufs prometheus-protobufs-gogo) +add_contrib(numactl-cmake numactl) + # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/numactl b/contrib/numactl new file mode 160000 index 00000000000..3871b1c42fc --- /dev/null +++ b/contrib/numactl @@ -0,0 +1 @@ +Subproject commit 3871b1c42fc71bceadafd745d2eff5dddfc2d67e diff --git a/contrib/numactl-cmake/CMakeLists.txt b/contrib/numactl-cmake/CMakeLists.txt new file mode 100644 index 00000000000..5d086366c7f --- /dev/null +++ b/contrib/numactl-cmake/CMakeLists.txt @@ -0,0 +1,20 @@ +option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_NUMACTL) + message (STATUS "Not using numactl") + return() +endif () + +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/numactl") + +set (SRCS + "${LIBRARY_DIR}/libnuma.c" + "${LIBRARY_DIR}/syscall.c" +) + +add_library(_numactl ${SRCS}) + +target_include_directories(_numactl SYSTEM PRIVATE include) +target_include_directories(_numactl SYSTEM PUBLIC "${LIBRARY_DIR}") + +add_library(ch_contrib::numactl ALIAS _numactl) diff --git a/contrib/numactl-cmake/include/config.h b/contrib/numactl-cmake/include/config.h new file mode 100644 index 00000000000..a304db38e53 --- /dev/null +++ b/contrib/numactl-cmake/include/config.h @@ -0,0 +1,82 @@ +/* config.h. Generated from config.h.in by configure. */ +/* config.h.in. Generated from configure.ac by autoheader. */ + +/* Checking for symver attribute */ +#define HAVE_ATTRIBUTE_SYMVER 0 + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDIO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to the sub-directory where libtool stores uninstalled libraries. */ +#define LT_OBJDIR ".libs/" + +/* Name of package */ +#define PACKAGE "numactl" + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "numactl" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "numactl 2.1" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "numactl" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "2.1" + +/* Define to 1 if all of the C89 standard headers exist (not just the ones + required in a freestanding environment). This macro is provided for + backward compatibility; new code need not use it. */ +#define STDC_HEADERS 1 + +/* If the compiler supports a TLS storage class define it to that here */ +#define TLS __thread + +/* Version number of package */ +#define VERSION "2.1" + +/* Number of bits in a file offset, on hosts where this is settable. */ +/* #undef _FILE_OFFSET_BITS */ + +/* Define to 1 on platforms where this makes off_t a 64-bit type. */ +/* #undef _LARGE_FILES */ + +/* Number of bits in time_t, on hosts where this is settable. */ +/* #undef _TIME_BITS */ + +/* Define to 1 on platforms where this makes time_t a 64-bit type. */ +/* #undef __MINGW_USE_VC2005_COMPAT */ diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..619a72ff200 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -140,6 +140,11 @@ # include #endif +#if USE_NUMACTL +#include +#endif + + #include /// A minimal file used when the server is run without installation INCBIN(resource_embedded_xml, SOURCE_DIR "/programs/server/embedded.xml"); @@ -754,6 +759,28 @@ try setenv("OPENSSL_CONF", config_dir.c_str(), true); /// NOLINT } +#if USE_NUMACTL + if (numa_available() != -1) + { + auto * membind = numa_get_membind(); + if (!numa_bitmask_equal(membind, numa_all_nodes_ptr)) + { + uint64_t total_numa_memory = 0; + auto max_node = numa_max_node(); + for (int i = 0; i <= max_node; ++i) + { + if (numa_bitmask_isbitset(membind, i)) + total_numa_memory += numa_node_size(i, nullptr); + } + + LOG_INFO( + log, + "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", + ReadableSize(total_numa_memory)); + } + } +#endif + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); diff --git a/src/Common/config.h.in b/src/Common/config.h.in index f68701d5d10..6a0090130a3 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -64,6 +64,7 @@ #cmakedefine01 USE_LIBARCHIVE #cmakedefine01 USE_POCKETFFT #cmakedefine01 USE_PROMETHEUS_PROTOBUFS +#cmakedefine01 USE_NUMACTL /// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. /// That's why we use absolute paths. diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 75f61baa854..d22bf674df4 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -173,5 +173,8 @@ endif() if (TARGET ch_contrib::prometheus_protobufs) set(USE_PROMETHEUS_PROTOBUFS 1) endif() +if (TARGET ch_contrib::numactl) + set(USE_NUMACTL 1) +endif() set(SOURCE_DIR ${PROJECT_SOURCE_DIR}) From e181ccd0173c46d31867097532f64df0be3944da Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:53:51 +0800 Subject: [PATCH 0287/1170] update doc --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index ae23387f6e5..db66188b1f5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** From a32c702caa142d15bc3e5bc51ca90240d5d010a9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 18:23:47 +0800 Subject: [PATCH 0288/1170] fix style --- src/Functions/map.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 5319390fb70..a8e5f7ad90e 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -236,7 +236,6 @@ public: ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - std::cout << "data keys is nullable" << std::endl; const NullMap * null_map = nullptr; if (const auto * nullable = checkAndGetColumn(data_keys.get())) { @@ -263,9 +262,7 @@ public: const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); - std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; From 7fedc0ffbee9d04e0352037021a127cea93cbbfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 12:26:37 +0200 Subject: [PATCH 0289/1170] Update base/glibc-compatibility/musl/getauxval.c Co-authored-by: Alexander Gololobov --- base/glibc-compatibility/musl/getauxval.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index b5bd2f114c2..28cb0f8d005 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -99,12 +99,12 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// #8 0x5622c0d6b7cd in _start (./ClickHouse/build_msan/contrib/google-protobuf-cmake/protoc+0x22c7cd) (BuildId: 6411d3c88b898ba3f7d49760555977d3e61f0741) /// The source of the issue above is that, at this point in time during __msan_init, we can't really do much as - /// most global variables aren't initialized or available yet, so we we can't initiate the auxiliary vector. + /// most global variables aren't initialized or available yet, so we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). - /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and it we used + /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and if we used /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and /// unconditionally return "/proc/self/exe" without any preparation. Theoretically this should be fine in /// our case, as we don't load any libraries. That's the theory at least. From beb506a5b8179f5c88a6f5fc90d62b8e74bf0d35 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 12:37:05 +0200 Subject: [PATCH 0290/1170] added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00307_format_xml.reference | 41 ++++++++++++++++++ .../queries/0_stateless/00307_format_xml.sql | 3 ++ .../0_stateless/00309_formats.reference | Bin 18537 -> 18736 bytes tests/queries/0_stateless/00309_formats.sql | 5 +++ 4 files changed, 49 insertions(+) diff --git a/tests/queries/0_stateless/00307_format_xml.reference b/tests/queries/0_stateless/00307_format_xml.reference index 2d9badc5a3e..14e74653d4f 100644 --- a/tests/queries/0_stateless/00307_format_xml.reference +++ b/tests/queries/0_stateless/00307_format_xml.reference @@ -1,3 +1,4 @@ +unnamed columns in tuple @@ -54,3 +55,43 @@ 1 +named columns in tuple + + + + + + s + String + + + time + DateTime + + + tpl + Tuple(String, DateTime) + + + + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + Hello & world + + Hello & world2001-02-03 04:05:06 + + + 1 + diff --git a/tests/queries/0_stateless/00307_format_xml.sql b/tests/queries/0_stateless/00307_format_xml.sql index 29c733bb186..22566112bc7 100644 --- a/tests/queries/0_stateless/00307_format_xml.sql +++ b/tests/queries/0_stateless/00307_format_xml.sql @@ -1,2 +1,5 @@ SET output_format_write_statistics = 0; +SELECT 'unnamed columns in tuple'; SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; +SELECT 'named columns in tuple'; +SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML;` diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index e637ee0363a7b35152a155ae3fa73a4f451d5148..a63720618ba54c6cc456f3356512449322dc2e80 100644 GIT binary patch delta 149 zcmaDkfpNnm#toSsmduQKj47-or3E>uY@wccC1xfpE{P?n;vparjXVX2n2th<0!Ubs ziva?zYfU!f5;0_8gmBMl=W(WRf=%Iq=->n?=fp5&@&sYg$;X8ig*A*!&2-d_3=9l( M4J>sHjny@|06s7$QUCw| delta 9 QcmdlmiSgwG#toSs02V|99{>OV diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index b0939c00a10..691fc6e7ab6 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -9,3 +9,8 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; + +SET enable_named_columns_in_function_tuple = 1; + +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes; From 2988e13050f0ab8a06e36ec8fe745386a214141b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 13:55:01 +0200 Subject: [PATCH 0291/1170] Free bitmask --- base/base/getMemoryAmount.cpp | 1 + programs/server/Server.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index b8162146496..56cddbfd628 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -84,6 +84,7 @@ uint64_t getMemoryAmountOrZero() memory_amount = total_numa_memory; } + numa_bitmask_free(membind); } #endif diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 619a72ff200..b9a7c298f00 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -778,6 +778,8 @@ try "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", ReadableSize(total_numa_memory)); } + + numa_bitmask_free(membind); } #endif From cd06945a03df0f8dbec6ff82332236caf86fbff3 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:05:25 +0000 Subject: [PATCH 0292/1170] Fix crash with Variant + AggregateFunction type --- src/Columns/ColumnAggregateFunction.cpp | 6 +- ...ant_with_aggregate_function_type.reference | 6 ++ ...0_variant_with_aggregate_function_type.sql | 60 +++++++++++++++++++ 3 files changed, 71 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference create mode 100644 tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index e26fe790a8e..9934970c868 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -330,7 +330,11 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_ void ColumnAggregateFunction::expand(const Filter & mask, bool inverted) { - expandDataByMask(data, mask, inverted); + ensureOwnership(); + Arena & arena = createOrGetArena(); + char * default_ptr = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); + func->create(default_ptr); + expandDataByMask(data, mask, inverted, default_ptr); } ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const diff --git a/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference new file mode 100644 index 00000000000..105e8e7d8bd --- /dev/null +++ b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.reference @@ -0,0 +1,6 @@ + 500 +fail 500 + 499 +fail 500 + 500 499 +fail 500 500 diff --git a/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql new file mode 100644 index 00000000000..cb9cdb0b456 --- /dev/null +++ b/tests/queries/0_stateless/03210_variant_with_aggregate_function_type.sql @@ -0,0 +1,60 @@ +SET allow_experimental_variant_type = 1; + +DROP TABLE IF EXISTS source; +CREATE TABLE source +( + Name String, + Value Int64 + +) ENGINE = MergeTree ORDER BY (); + +INSERT INTO source SELECT ['fail', 'success'][number % 2] as Name, number AS Value FROM numbers(1000); + +DROP TABLE IF EXISTS test_agg_variant; +CREATE TABLE test_agg_variant +( + Name String, + Value Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)) +) +ENGINE = MergeTree +ORDER BY (Name); + +INSERT INTO test_agg_variant +SELECT + Name, + t AS Value +FROM +( + SELECT + Name, + arrayJoin([ + uniqExactState(Value)::Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)), + avgState(Value)::Variant(AggregateFunction(uniqExact, Int64), AggregateFunction(avg, Int64)) + ]) AS t + FROM source + GROUP BY Name +); + +SELECT + Name, + uniqExactMerge(Value.`AggregateFunction(uniqExact, Int64)`) AS Value +FROM test_agg_variant +GROUP BY Name; + +SELECT + Name, + avgMerge(Value.`AggregateFunction(avg, Int64)`) AS Value +FROM test_agg_variant +GROUP BY Name; + +SELECT + Name, + uniqExactMerge(Value.`AggregateFunction(uniqExact, Int64)`) AS ValueUniq, + avgMerge(Value.`AggregateFunction(avg, Int64)`) AS ValueAvg +FROM test_agg_variant +GROUP BY Name; + + +DROP TABLE test_agg_variant; +DROP TABLE source; + From c5164fede8665b61c10ec0d7b6873a7cf04aab12 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 13:17:21 +0000 Subject: [PATCH 0293/1170] Fix some test. --- src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6b5b129085d..5972d89bddd 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,7 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); + before_prewhere_sample = prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1976,7 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where->dag.updateHeader(before_where_sample); + before_where_sample = before_where->dag.updateHeader(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); From 0642ed19b7c67e443be110f2a0f2d1f032ddd8d5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 14:17:45 +0000 Subject: [PATCH 0294/1170] Fixing more tests. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index bdb90abd326..e5aeb9686be 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1525,7 +1525,7 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(std::move(*filter_actions_dag)); + query_info.filter_actions_dag = std::make_shared(filter_actions_dag->clone()); buildIndexes( indexes, From bd721950b0401f94be652c11015bd1985c283f3a Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 25 Jul 2024 16:24:17 +0200 Subject: [PATCH 0295/1170] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 18736 -> 18666 bytes tests/queries/0_stateless/00309_formats.sql | 4 ++-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index a63720618ba54c6cc456f3356512449322dc2e80..f3ea45520bb50fb936caf6724e9fedf3cdd00b75 100644 GIT binary patch delta 54 zcmdlmiSgA$#tkJN8myt7c_kJsE{P?nVj-mkIjI_X3Sc1}g%kylsHVzCEg+hFz{3v! D3NRA? delta 99 zcmaDgk#WN$#tkJNdTgPdc_n5hEG~&9sp27}1v#l2c?u9Q9fcGHkgz5f0|Z>xnrz4= o!o|P{;hxo=JjcUt@=alWVGScwGaYp!0|NtH14~^)V|7g~0E0RhwEzGB diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index 691fc6e7ab6..0366cdeea5c 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -12,5 +12,5 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SET enable_named_columns_in_function_tuple = 1; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes; +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From 59f9c125044b6e56a3ded8034478eff79e930018 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 25 Jul 2024 14:37:47 +0000 Subject: [PATCH 0296/1170] Increase sleep time make sure there is a new failure The previous sleep was already adding +5s to make sure the TTL was properly applied, so we'd rather use the same value here instead of just 1s. --- tests/integration/test_storage_s3_queue/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 4348857acd3..2e339a9b5c9 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -871,7 +871,7 @@ def test_max_set_age(started_cluster): node.query(f"SELECT uniq(_path) from {dst_table_name}") ) - time.sleep(max_age + 5) + time.sleep(max_age + max_age / 2) expected_rows *= 2 wait_for_condition(lambda: get_count() == expected_rows) @@ -922,7 +922,7 @@ def test_max_set_age(started_cluster): ) ) - time.sleep(max_age + 1) + time.sleep(max_age + max_age / 2) assert failed_count + 2 <= get_object_storage_failures() From 1973458ae07a5cd519b7069451d2be5822a89bf7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 15:00:37 +0000 Subject: [PATCH 0297/1170] Update PlannerWindowFunctions --- src/Planner/Planner.cpp | 13 ++++++------- src/Planner/PlannerWindowFunctions.cpp | 15 ++------------- src/Planner/PlannerWindowFunctions.h | 2 +- 3 files changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fb721069e6e..968642dc9de 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,19 +933,19 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan, void addWindowSteps(QueryPlan & query_plan, const PlannerContextPtr & planner_context, - const WindowAnalysisResult & window_analysis_result) + WindowAnalysisResult & window_analysis_result) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - const auto & window_descriptions = window_analysis_result.window_descriptions; - auto perm = sortWindowDescriptions(window_descriptions); + auto & window_descriptions = window_analysis_result.window_descriptions; + sortWindowDescriptions(window_descriptions); size_t window_descriptions_size = window_descriptions.size(); for (size_t i = 0; i < window_descriptions_size; ++i) { - const auto & window_description = window_descriptions[perm[i]]; + const auto & window_description = window_descriptions[i]; /** We don't need to sort again if the input from previous window already * has suitable sorting. Also don't create sort steps when there are no @@ -958,9 +958,8 @@ void addWindowSteps(QueryPlan & query_plan, bool need_sort = !window_description.full_sort_description.empty(); if (need_sort && i != 0) { - auto prev = perm[i - 1]; - need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[prev].full_sort_description) - || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[prev].partition_by.size()); + need_sort = !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description) + || (settings.max_threads != 1 && window_description.partition_by.size() != window_descriptions[i - 1].partition_by.size()); } if (need_sort) { diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index ba0e11df76b..225852de5a7 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -122,7 +122,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & return result; } -std::vector sortWindowDescriptions(const std::vector & window_descriptions) +void sortWindowDescriptions(std::vector & window_descriptions) { auto window_description_comparator = [](const WindowDescription & lhs, const WindowDescription & rhs) { @@ -153,18 +153,7 @@ std::vector sortWindowDescriptions(const std::vector return left.size() > right.size(); }; - auto comparator = [&](size_t lhs, size_t rhs) - { - return window_description_comparator(window_descriptions[lhs], window_descriptions[rhs]); - }; - - std::vector perm(window_descriptions.size()); - for (size_t i = 0; i < perm.size(); ++i) - perm[i] = i; - - ::sort(perm.begin(), perm.end(), comparator); - - return perm; + ::sort(window_descriptions.begin(), window_descriptions.end(), window_description_comparator); } } diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h index 3039ecefc4b..1552ef5a71f 100644 --- a/src/Planner/PlannerWindowFunctions.h +++ b/src/Planner/PlannerWindowFunctions.h @@ -15,6 +15,6 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & /** Try to sort window descriptions in such an order that the window with the longest * sort description goes first, and all window that use its prefixes follow. */ -std::vector sortWindowDescriptions(const std::vector & window_descriptions); +void sortWindowDescriptions(std::vector & window_descriptions); } From 5ea867231bafc01b4512989f351106b7afcc14af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 16:55:23 +0200 Subject: [PATCH 0298/1170] Read configuration for clickhouse-local from ~/.clickhouse-local Signed-off-by: Azat Khuzhin --- programs/local/LocalServer.cpp | 16 +++++++-- src/Common/Config/CMakeLists.txt | 1 + src/Common/Config/getLocalConfigPath.cpp | 46 ++++++++++++++++++++++++ src/Common/Config/getLocalConfigPath.h | 12 +++++++ 4 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 src/Common/Config/getLocalConfigPath.cpp create mode 100644 src/Common/Config/getLocalConfigPath.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 48e0cca7b73..ade4e0f49df 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -1,6 +1,7 @@ #include "LocalServer.h" #include +#include #include #include #include @@ -127,10 +128,21 @@ void LocalServer::initialize(Poco::Util::Application & self) { Poco::Util::Application::initialize(self); + const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe) + if (home_path_cstr) + home_path = home_path_cstr; + /// Load config files if exists - if (getClientConfiguration().has("config-file") || fs::exists("config.xml")) + std::string config_path; + if (getClientConfiguration().has("config-file")) + config_path = getClientConfiguration().getString("config-file"); + else if (config_path.empty() && fs::exists("config.xml")) + config_path = "config.xml"; + else if (config_path.empty()) + config_path = getLocalConfigPath(home_path).value_or(""); + + if (fs::exists(config_path)) { - const auto config_path = getClientConfiguration().getString("config-file", "config.xml"); ConfigProcessor config_processor(config_path, false, true); ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 09095ef5acc..2bd32b98bda 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -2,6 +2,7 @@ set (SRCS AbstractConfigurationComparison.cpp ConfigProcessor.cpp getClientConfigPath.cpp + getLocalConfigPath.cpp ConfigReloader.cpp YAMLParser.cpp ConfigHelper.cpp diff --git a/src/Common/Config/getLocalConfigPath.cpp b/src/Common/Config/getLocalConfigPath.cpp new file mode 100644 index 00000000000..afaa7f79026 --- /dev/null +++ b/src/Common/Config/getLocalConfigPath.cpp @@ -0,0 +1,46 @@ +#include + +#include +#include + + +namespace fs = std::filesystem; + +namespace DB +{ + +std::optional getLocalConfigPath(const std::string & home_path) +{ + std::string config_path; + bool found = false; + + std::vector names; + names.emplace_back("./clickhouse-local"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-local/config"); + names.emplace_back("/etc/clickhouse-local/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + + if (found) + return config_path; + + return std::nullopt; +} + +} diff --git a/src/Common/Config/getLocalConfigPath.h b/src/Common/Config/getLocalConfigPath.h new file mode 100644 index 00000000000..14625571d6c --- /dev/null +++ b/src/Common/Config/getLocalConfigPath.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Return path to existing configuration file. +std::optional getLocalConfigPath(const std::string & home_path); + +} From fb271436a1efe969f4de09b14aec942baa145cb9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 15:37:13 +0000 Subject: [PATCH 0299/1170] Remove ActionsDAGPtr completely. --- src/Interpreters/ActionsDAG.h | 3 --- src/Interpreters/evaluateConstantExpression.cpp | 4 ++-- .../optimizePrimaryKeyConditionAndLimit.cpp | 6 +++--- src/Processors/QueryPlan/SourceStepWithFilter.h | 6 +++--- src/Processors/QueryPlan/TotalsHavingStep.h | 3 --- src/Processors/QueryPlan/WindowStep.h | 3 --- src/Storages/MergeTree/KeyCondition.cpp | 14 +++++++------- src/Storages/MergeTree/KeyCondition.h | 2 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 3 +++ src/Storages/SelectQueryInfo.h | 3 --- src/Storages/StorageMerge.cpp | 2 +- 11 files changed, 20 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 76cc9327530..43c1b41a240 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -11,9 +11,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class IExecutableFunction; using ExecutableFunctionPtr = std::shared_ptr; diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 42d6f3d3037..4bfc80af1fe 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -679,9 +679,9 @@ std::optional evaluateExpressionOverConstantCondition( size_t max_elements) { auto inverted_dag = KeyCondition::cloneASTWithInversionPushDown({predicate}, context); - auto matches = matchTrees(expr, *inverted_dag, false); + auto matches = matchTrees(expr, inverted_dag, false); - auto predicates = analyze(inverted_dag->getOutputs().at(0), matches, context, max_elements); + auto predicates = analyze(inverted_dag.getOutputs().at(0), matches, context, max_elements); if (!predicates) return {}; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 63b4e019066..f53212407d2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->prewhere_actions->clone()), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) - source_step_with_filter->addFilter(std::make_unique(storage_prewhere_info->row_level_filter->clone()), storage_prewhere_info->row_level_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) { - source_step_with_filter->addFilter(std::make_unique(filter_step->getExpression().clone()), filter_step->getFilterColumnName()); + source_step_with_filter->addFilter(filter_step->getExpression().clone(), filter_step->getFilterColumnName()); } else if (auto * limit_step = typeid_cast(iter->node->step.get())) { diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index f7a030c0628..6cea5fd7245 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -45,9 +45,9 @@ public: const Names & requiredSourceColumns() const { return required_source_columns; } - void addFilter(ActionsDAGPtr filter_dag, std::string column_name) + void addFilter(ActionsDAG filter_dag, std::string column_name) { - filter_nodes.nodes.push_back(&filter_dag->findInOutputs(column_name)); + filter_nodes.nodes.push_back(&filter_dag.findInOutputs(column_name)); filter_dags.push_back(std::move(filter_dag)); } @@ -86,7 +86,7 @@ protected: private: /// Will be cleared after applyFilters() is called. ActionDAGNodes filter_nodes; - std::vector filter_dags; + std::vector filter_dags; }; } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 927b8d99de3..4b414d41c57 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -6,9 +6,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - enum class TotalsMode : uint8_t; /// Execute HAVING and calculate totals. See TotalsHavingTransform. diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index 47883e5edf6..d79cd7fd45e 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -6,9 +6,6 @@ namespace DB { -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - class WindowTransform; class WindowStep : public ITransformingStep diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 0eb59a47cae..69bffac9160 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -696,22 +696,22 @@ const std::unordered_map KeyConditi {"hilbertEncode", SpaceFillingCurveType::Hilbert} }; -ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) +ActionsDAG KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { - auto res = std::make_unique(); + ActionsDAG res; std::unordered_map to_inverted; for (auto & node : nodes) - node = &DB::cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false); + node = &DB::cloneASTWithInversionPushDown(*node, res, to_inverted, context, false); if (nodes.size() > 1) { auto function_builder = FunctionFactory::instance().get("and", context); - nodes = {&res->addFunction(function_builder, std::move(nodes), "")}; + nodes = {&res.addFunction(function_builder, std::move(nodes), "")}; } - res->getOutputs().swap(nodes); + res.getOutputs().swap(nodes); return res; } @@ -826,9 +826,9 @@ KeyCondition::KeyCondition( * are pushed down and applied (when possible) to leaf nodes. */ auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context); - assert(inverted_dag->getOutputs().size() == 1); + assert(inverted_dag.getOutputs().size() == 1); - const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0]; + const auto * inverted_dag_filter_node = inverted_dag.getOutputs()[0]; RPNBuilder builder(inverted_dag_filter_node, context, [&](const RPNBuilderTreeNode & node, RPNElement & out) { diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index a9e1a589ba5..e9343ec08ea 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -134,7 +134,7 @@ public: DataTypePtr current_type, bool single_point = false); - static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context); + static ActionsDAG cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context); bool matchesExactContinuousRange() const; diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 116edf5b9cb..1d0569e0df6 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -10,6 +10,9 @@ namespace DB { +class ActionsDAG; +using ActionsDAGPtr = std::unique_ptr; + namespace ErrorCodes { extern const int LOGICAL_ERROR; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 60f103fdb70..1c4cb7d92d8 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -18,9 +18,6 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class ActionsDAG; -using ActionsDAGPtr = std::unique_ptr; - struct PrewhereInfo; using PrewhereInfoPtr = std::shared_ptr; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0e1568c8e79..e5de15c1d21 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1279,7 +1279,7 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const { - step->addFilter(std::make_unique(actions_dag.clone()), filter_column_name); + step->addFilter(actions_dag.clone(), filter_column_name); } void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const From 869f6a6f105f50aa4d0e71e6440646b78539f0ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 16:33:12 +0000 Subject: [PATCH 0300/1170] Updating PrewhereInfo --- src/Interpreters/ExpressionAnalyzer.cpp | 9 ++++----- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++-------- src/Planner/PlannerJoinTree.cpp | 8 +++----- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 8 ++++---- .../optimizePrimaryKeyConditionAndLimit.cpp | 2 +- .../Optimizations/optimizeReadInOrder.cpp | 12 +++++------- .../Optimizations/projectionsCommon.cpp | 15 ++++++--------- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++-------- .../QueryPlan/SourceStepWithFilter.cpp | 9 +++------ src/Storages/IStorage.cpp | 3 +-- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 ++-- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 6 +++--- src/Storages/SelectQueryInfo.h | 7 +++---- src/Storages/StorageBuffer.cpp | 7 +++---- 16 files changed, 52 insertions(+), 70 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 5972d89bddd..d25434a515d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -2230,12 +2230,11 @@ void ExpressionAnalysisResult::checkActions() const /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (hasPrewhere()) { - auto check_actions = [](const std::optional & actions) + auto check_actions = [](ActionsDAG & actions) { - if (actions) - for (const auto & node : actions->getNodes()) - if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) - throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action"); + for (const auto & node : actions.getNodes()) + if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) + throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action"); }; check_actions(prewhere_info->prewhere_actions); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 174e6b5b0e0..4fd6f7a2900 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -937,7 +937,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() { { const auto & node - = query_info_copy.prewhere_info->prewhere_actions->findInOutputs(query_info_copy.prewhere_info->prewhere_column_name); + = query_info_copy.prewhere_info->prewhere_actions.findInOutputs(query_info_copy.prewhere_info->prewhere_column_name); added_filter_nodes.nodes.push_back(&node); } @@ -1058,7 +1058,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header); + header = analysis_result.prewhere_info->prewhere_actions.updateHeader(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); } @@ -1521,7 +1521,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.prewhere_info->prewhere_actions->clone(), + expressions.prewhere_info->prewhere_actions.clone(), expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); @@ -2066,7 +2066,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c }); } - auto filter_actions = std::make_shared(prewhere_info.prewhere_actions->clone()); + auto filter_actions = std::make_shared(prewhere_info.prewhere_actions.clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -2157,7 +2157,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() if (prewhere_info) { /// Get some columns directly from PREWHERE expression actions - auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); + auto prewhere_required_columns = prewhere_info->prewhere_actions.getRequiredColumns().getNames(); columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); if (prewhere_info->row_level_filter) @@ -2229,7 +2229,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() if (prewhere_info) { NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end()); - Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns(); + Block prewhere_actions_result = prewhere_info->prewhere_actions.getResultColumns(); /// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards. /// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure. @@ -2268,7 +2268,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() { /// Don't remove columns which are needed to be aliased. for (const auto & name : required_columns) - prewhere_info->prewhere_actions->tryRestoreColumn(name); + prewhere_info->prewhere_actions.tryRestoreColumn(name); /// Add physical columns required by prewhere actions. for (const auto & column : required_columns_from_prewhere) @@ -2326,7 +2326,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle if (analysis_result.hasPrewhere()) { auto & prewhere_info = analysis_result.prewhere_info; - filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name)); + filter_nodes.push_back(&prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name)); if (prewhere_info->row_level_filter) filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name)); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index e9f886ab162..a3db0395ccc 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -437,7 +437,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info std::unordered_set required_output_nodes; - for (const auto * input : prewhere_actions->getInputs()) + for (const auto * input : prewhere_actions.getInputs()) { if (required_columns.contains(input->result_name)) required_output_nodes.insert(input); @@ -446,7 +446,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info if (required_output_nodes.empty()) return; - auto & prewhere_outputs = prewhere_actions->getOutputs(); + auto & prewhere_outputs = prewhere_actions.getOutputs(); for (const auto & output : prewhere_outputs) { auto required_output_node_it = required_output_nodes.find(output); @@ -801,10 +801,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name))) { if (!prewhere_info) - prewhere_info = std::make_shared(); - - if (!prewhere_info->prewhere_actions) { + prewhere_info = std::make_shared(); prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->remove_prewhere_column = filter_info.do_remove_column; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 5711189136c..dc73521210a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -56,7 +56,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) return; const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); - if (storage_prewhere_info && storage_prewhere_info->prewhere_actions) + if (storage_prewhere_info) return; /// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans. @@ -165,16 +165,16 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) { prewhere_info->prewhere_column_name = conditions.front()->result_name; if (prewhere_info->remove_prewhere_column) - prewhere_info->prewhere_actions->getOutputs().push_back(conditions.front()); + prewhere_info->prewhere_actions.getOutputs().push_back(conditions.front()); } else { prewhere_info->remove_prewhere_column = true; FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {}); + const auto * node = &prewhere_info->prewhere_actions.addFunction(func_builder_and, std::move(conditions), {}); prewhere_info->prewhere_column_name = node->result_name; - prewhere_info->prewhere_actions->getOutputs().push_back(node); + prewhere_info->prewhere_actions.getOutputs().push_back(node); } source_step_with_filter->updatePrewhereInfo(prewhere_info); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index f53212407d2..490b79fbf8d 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -18,7 +18,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); if (storage_prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions->clone(), storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions.clone(), storage_prewhere_info->prewhere_column_name); if (storage_prewhere_info->row_level_filter) source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 252420e19fe..99df6da263f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -191,13 +191,11 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi /// Should ignore limit if there is filtering. limit = 0; - if (prewhere_info->prewhere_actions) - { - //std::cerr << "====== Adding prewhere " << std::endl; - appendExpression(dag, *prewhere_info->prewhere_actions); - if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) - appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); - } + //std::cerr << "====== Adding prewhere " << std::endl; + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); + } return; } diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 571d1dd0cc1..7414d479cc9 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -128,15 +128,12 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & return false; } - if (prewhere_info->prewhere_actions) - { - appendExpression(*prewhere_info->prewhere_actions); - if (const auto * filter_expression - = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) - filter_nodes.push_back(filter_expression); - else - return false; - } + appendExpression(prewhere_info->prewhere_actions); + if (const auto * filter_expression + = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + filter_nodes.push_back(filter_expression); + else + return false; } return true; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e5aeb9686be..483876dd293 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -109,8 +109,7 @@ bool restorePrewhereInputs(PrewhereInfo & info, const NameSet & inputs) if (info.row_level_filter) added = added || restoreDAGInputs(*info.row_level_filter, inputs); - if (info.prewhere_actions) - added = added || restoreDAGInputs(*info.prewhere_actions, inputs); + added = added || restoreDAGInputs(info.prewhere_actions, inputs); return added; } @@ -175,9 +174,8 @@ static void updateSortDescriptionForOutputStream( Block original_header = output_stream.header.cloneEmpty(); if (prewhere_info) { - if (prewhere_info->prewhere_actions) { - FindOriginalNodeForOutputName original_column_finder(*prewhere_info->prewhere_actions); + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); for (auto & column : original_header) { const auto * original_node = original_column_finder.find(column.name); @@ -2131,7 +2129,6 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const prefix.push_back(format_settings.indent_char); prefix.push_back(format_settings.indent_char); - if (prewhere_info->prewhere_actions) { format_settings.out << prefix << "Prewhere filter" << '\n'; format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name; @@ -2139,7 +2136,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); expression->describeActions(format_settings.out, prefix); } @@ -2169,12 +2166,11 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_info_map = std::make_unique(); prewhere_info_map->add("Need filter", prewhere_info->need_filter); - if (prewhere_info->prewhere_actions) { std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index b91debc8239..3de9ae37db0 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -34,9 +34,8 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo block.erase(prewhere_info->row_level_column_name); } - if (prewhere_info->prewhere_actions) { - block = prewhere_info->prewhere_actions->updateHeader(block); + block = prewhere_info->prewhere_actions.updateHeader(block); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -102,7 +101,6 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con prefix.push_back(format_settings.indent_char); prefix.push_back(format_settings.indent_char); - if (prewhere_info->prewhere_actions) { format_settings.out << prefix << "Prewhere filter" << '\n'; format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name; @@ -110,7 +108,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con format_settings.out << " (removed)"; format_settings.out << '\n'; - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); expression->describeActions(format_settings.out, prefix); } @@ -132,12 +130,11 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const std::unique_ptr prewhere_info_map = std::make_unique(); prewhere_info_map->add("Need filter", prewhere_info->need_filter); - if (prewhere_info->prewhere_actions) { std::unique_ptr prewhere_filter_map = std::make_unique(); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); - auto expression = std::make_shared(prewhere_info->prewhere_actions->clone()); + auto expression = std::make_shared(prewhere_info->prewhere_actions.clone()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 823a6ae1cbc..755d71df531 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -325,9 +325,8 @@ std::string PrewhereInfo::dump() const ss << "row_level_filter " << row_level_filter->dumpDAG() << "\n"; } - if (prewhere_actions) { - ss << "prewhere_actions " << prewhere_actions->dumpDAG() << "\n"; + ss << "prewhere_actions " << prewhere_actions.dumpDAG() << "\n"; } ss << "remove_prewhere_column " << remove_prewhere_column diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 26595fbb36d..a9b77fb6c03 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -329,7 +329,7 @@ void MergeTreePrefetchedReadPool::fillPerPartStatistics() part_stat.sum_marks += range.end - range.begin; const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info - ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + ? prewhere_info->prewhere_actions.getRequiredColumnsNames() : column_names; part_stat.approx_size_of_mark = getApproximateSizeOfGranule(*read_info.data_part, columns); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 46482bc0959..6d2560bc9c7 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -65,7 +65,7 @@ static size_t calculateMinMarksPerTask( /// Which means in turn that for most of the rows we will read only the columns from prewhere clause. /// So it makes sense to use only them for the estimation. const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info - ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + ? prewhere_info->prewhere_actions.getRequiredColumnsNames() : columns_to_read; const size_t part_compressed_bytes = getApproxSizeOfPart(*part.data_part, columns); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index f1df9e231c4..1a0709faf1c 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -59,7 +59,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( if (prewhere_info) LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}", - (prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("")), + prewhere_info->prewhere_actions.dumpDAG(), (!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string(""))); } @@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr PrewhereExprStep prewhere_step { .type = PrewhereExprStep::Filter, - .actions = std::make_shared(prewhere_info->prewhere_actions->clone(), actions_settings), + .actions = std::make_shared(prewhere_info->prewhere_actions.clone(), actions_settings), .filter_column_name = prewhere_info->prewhere_column_name, .remove_filter_column = prewhere_info->remove_prewhere_column, .need_filter = prewhere_info->need_filter, diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 1d0569e0df6..36ff6c0a4bd 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -216,11 +216,11 @@ const ActionsDAG::Node & addAndTrue( /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere) { - if (!prewhere_info || !prewhere_info->prewhere_actions) + if (!prewhere_info) return true; /// 1. List all condition nodes that are combined with AND into PREWHERE condition - const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + const auto & condition_root = prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name); const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); if (!is_conjunction) return false; @@ -306,7 +306,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } /// 6. Find all outputs of the original DAG - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + auto original_outputs = prewhere_info->prewhere_actions.getOutputs(); /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 NameSet all_output_names; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 1c4cb7d92d8..7ad6a733c6f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -45,7 +45,7 @@ struct PrewhereInfo /// This actions are separate because prewhere condition should not be executed over filtered rows. std::optional row_level_filter; /// Actions which are executed on block in order to get filter column for prewhere step. - std::optional prewhere_actions; + ActionsDAG prewhere_actions; String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; @@ -53,7 +53,7 @@ struct PrewhereInfo bool generated_by_optimizer = false; PrewhereInfo() = default; - explicit PrewhereInfo(std::optional prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(ActionsDAG prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} std::string dump() const; @@ -65,8 +65,7 @@ struct PrewhereInfo if (row_level_filter) prewhere_info->row_level_filter = row_level_filter->clone(); - if (prewhere_actions) - prewhere_info->prewhere_actions = prewhere_actions->clone(); + prewhere_info->prewhere_actions = prewhere_actions.clone(); prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->prewhere_column_name = prewhere_column_name; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index aee4e4683ad..04e6d6676d1 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -319,13 +319,12 @@ void StorageBuffer::read( src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); } - if (src_table_query_info.prewhere_info->prewhere_actions) { src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( actions_dag.clone(), - std::move(*src_table_query_info.prewhere_info->prewhere_actions)); + std::move(src_table_query_info.prewhere_info->prewhere_actions)); - src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); + src_table_query_info.prewhere_info->prewhere_actions.removeUnusedActions(); } } @@ -440,7 +439,7 @@ void StorageBuffer::read( }); } - auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions->clone(), actions_settings); + auto actions = std::make_shared(query_info.prewhere_info->prewhere_actions.clone(), actions_settings); pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( From a06df0729ea398642b715bfd2b121b1db0c5dd6d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 17:10:59 +0000 Subject: [PATCH 0301/1170] Remove the comment. --- src/Interpreters/ActionsDAG.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 85b2b38da17..4aaecc491e0 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3111,7 +3111,6 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs( } FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_) - //: actions(actions_) { const auto & actions_outputs = actions_.getOutputs(); for (const auto * output_node : actions_outputs) From 738d659e3bd8e222ff947e206d03d516c7053052 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jul 2024 17:26:16 +0000 Subject: [PATCH 0302/1170] Do not remove constants from Distributed header if query is executed up to Complete. --- src/Storages/StorageDistributed.cpp | 7 ++- .../02563_analyzer_merge.reference | 1 + .../0_stateless/02563_analyzer_merge.sql | 45 +++++++++++++++++++ 3 files changed, 51 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 07892971ec2..9b417cda177 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -847,8 +847,11 @@ void StorageDistributed::read( /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ - for (auto & column : header) - column.column = column.column->convertToFullColumnIfConst(); + if (processed_stage != QueryProcessingStage::Complete) + { + for (auto & column : header) + column.column = column.column->convertToFullColumnIfConst(); + } modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); diff --git a/tests/queries/0_stateless/02563_analyzer_merge.reference b/tests/queries/0_stateless/02563_analyzer_merge.reference index 8be01c88d6f..2b3cc2d5dfb 100644 --- a/tests/queries/0_stateless/02563_analyzer_merge.reference +++ b/tests/queries/0_stateless/02563_analyzer_merge.reference @@ -1,2 +1,3 @@ 0 Value_0 02563_db test_merge_table_1 1 Value_1 02563_db test_merge_table_2 +91138316-5127-45ac-9c25-4ad8779777b4 160 diff --git a/tests/queries/0_stateless/02563_analyzer_merge.sql b/tests/queries/0_stateless/02563_analyzer_merge.sql index c90f7dcb2a5..217fb7019c4 100644 --- a/tests/queries/0_stateless/02563_analyzer_merge.sql +++ b/tests/queries/0_stateless/02563_analyzer_merge.sql @@ -35,4 +35,49 @@ SELECT id, value, _database, _table FROM 02563_db.test_merge_table ORDER BY id; DROP TABLE 02563_db.test_merge_table; DROP TABLE 02563_db.test_merge_table_1; DROP TABLE 02563_db.test_merge_table_2; + +CREATE TABLE 02563_db.t_1 +( + timestamp DateTime64(9), + a String, + b String +) +ENGINE = MergeTree +PARTITION BY formatDateTime(toStartOfMinute(timestamp), '%Y%m%d%H', 'UTC') +ORDER BY (timestamp, a, b); + +CREATE TABLE 02563_db.dist_t_1 (timestamp DateTime64(9), a String, b String) ENGINE = Distributed('test_shard_localhost', '02563_db', 't_1'); + +CREATE TABLE 02563_db.m ENGINE = Merge('02563_db', '^dist_'); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-13 22:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(30); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-13 23:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(30); + +INSERT INTO 02563_db.t_1 (timestamp, a, b) +select + addMinutes(toDateTime64('2024-07-14 00:00:00', 9, 'UTC'), number), + randomString(5), + randomString(5) +from numbers(100); + + +SELECT '91138316-5127-45ac-9c25-4ad8779777b4', + count() +FROM 02563_db.m; + +DROP TABLE 02563_db.t_1; +DROP TABLE 02563_db.dist_t_1; +DROP TABLE 02563_db.m; + DROP DATABASE 02563_db; From 1ba4790511e1a06af8fb85e01767ce95866ee2a8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 25 Jul 2024 21:18:48 +0200 Subject: [PATCH 0303/1170] Review changes --- .../functions/type-conversion-functions.md | 550 ++++++++++-------- 1 file changed, 310 insertions(+), 240 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 057083d317f..844d957d538 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -51,7 +51,7 @@ SETTINGS cast_keep_nullable = 1 ## toInt8 -Converts an input value to a value of type `Int8`. +Converts an input value to a value of type [`Int8`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -61,10 +61,20 @@ toInt8(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: `SELECT toInt8(128) == -128;`, `SELECT toInt8(128.0) == -128;`, `SELECT toInt8('128') == -128;`. ::: **Returned value** @@ -72,11 +82,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 8-bit integer value. [Int8](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -106,32 +112,33 @@ Result: ## toInt8OrZero -Like [`toInt8`](#toint8), it takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `0`. +Like [`toInt8`](#toint8), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt8OrZero(expr) +toInt8OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise `0`. [Int8](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -160,32 +167,33 @@ Result: ## toInt8OrNull -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns `NULL`. +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt8OrNull(expr) +toInt8OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise `NULL`. [Int8](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -212,7 +220,7 @@ Result: ## toInt8OrDefault -Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int8`. If unsuccessful, returns the default type value. +Like [`toInt8`](#toint8), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int8`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -222,26 +230,28 @@ toInt8OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrDefault('0xc0fe', CAST('-1', 'Int8'));` +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 8-bit integer value if successful, otherwise returns the default value. [Int8](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -268,7 +278,7 @@ Result: ## toInt16 -Converts an input value to a value of type `Int16`. +Converts an input value to a value of type [`Int16`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -278,10 +288,20 @@ toInt16(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: `SELECT toInt16(32768) == -32768;`, `SELECT toInt16(32768) == -32768;`, `SELECT toInt16('32768') == -32768;`. ::: **Returned value** @@ -289,11 +309,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 16-bit integer value. [Int16](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -323,32 +339,33 @@ Result: ## toInt16OrZero -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `0`. +Like [`toInt16`](#toint16), this function converts an input value to a value of type [Int16](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt16OrZero(expr) +toInt16OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise `0`. [Int16](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -377,32 +394,33 @@ Result: ## toInt16OrNull -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns `NULL`. +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt16OrNull(expr) +toInt16OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise `NULL`. [Int16](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -431,7 +449,7 @@ Result: ## toInt16OrDefault -Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int16`. If unsuccessful, returns the default type value. +Like [`toInt16`](#toint16), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int16`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -441,26 +459,28 @@ toInt16OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int16` is unsuccessful. [Int8](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt16OrDefault('0xc0fe', CAST('-1', 'Int16'));` +- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 16-bit integer value if successful, otherwise returns the default value. [Int16](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -485,7 +505,7 @@ Result: ## toInt32 -Converts an input value to a value of type `Int32`. +Converts an input value to a value of type [`Int32`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -495,10 +515,25 @@ toInt32(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: +``` +SELECT toInt32(2147483648) == -2147483648; +SELECT toInt32(2147483648.0) == -2147483648; +SELECT toInt32('2147483648') == -2147483648; +``` ::: **Returned value** @@ -506,11 +541,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 32-bit integer value. [Int32](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -540,32 +571,34 @@ Result: ## toInt32OrZero -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `0`. +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt32OrZero(expr) +toInt32OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: **Returned value** - 32-bit integer value if successful, otherwise `0`. [Int32](../data-types/int-uint.md) :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncate fractional digits of numbers. ::: **Example** @@ -588,35 +621,36 @@ Result: - [`toInt32`](#toint32). - [`toInt32OrNull`](#toint32ornull). - [`toInt32OrDefault`](#toint32ordefault). -- + ## toInt32OrNull -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns `NULL`. +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt32OrNull(expr) +toInt32OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 32-bit integer value if successful, otherwise `NULL`. [Int32](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -643,7 +677,7 @@ Result: ## toInt32OrDefault -Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int32`. If unsuccessful, returns the default type value. +Like [`toInt32`](#toint32), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int32`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -653,24 +687,26 @@ toInt32OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt32OrDefault('0xc0fe', CAST('-1', 'Int32'));` +- If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 32-bit integer value if successful, otherwise returns the default value. [Int32](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. - ::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. ::: **Example** @@ -697,7 +733,7 @@ Result: ## toInt64 -Converts an input value to a value of type `Int64`. +Converts an input value to a value of type [`Int64`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -707,10 +743,26 @@ toInt64(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +For example: + +``` +SELECT toInt64(9223372036854775808) == -9223372036854775808; +SELECT toInt64(9223372036854775808.0) == -9223372036854775808; +SELECT toInt64('9223372036854775808') == --9223372036854775808; +``` ::: **Returned value** @@ -718,11 +770,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -752,32 +800,33 @@ Result: ## toInt64OrZero -Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `0`. +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt64OrZero(expr) +toInt64OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 64-bit integer value if successful, otherwise `0`. [Int64](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -806,32 +855,33 @@ Result: ## toInt64OrNull -Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns `NULL`. +Like [`toInt64`], takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt64OrNull(expr) +toInt64OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -860,7 +910,7 @@ Result: ## toInt64OrDefault -Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int64`. If unsuccessful, returns the default type value. +Like [`toInt64`](#toint64), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int64`](../data-types/nullable.md). If unsuccessful, returns the default type value. **Syntax** @@ -870,24 +920,26 @@ toInt64OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt64OrDefault('0xc0fe', CAST('-1', 'Int64'));` +- If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md) and the result over or under flows. **Returned value** - Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. - ::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. ::: **Example** @@ -916,7 +968,7 @@ Result: ## toInt128 -Converts an input value to a value of type `Int128`. +Converts an input value to a value of type [`Int128`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -926,10 +978,19 @@ toInt128(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md), the result over or under flows. This is not considered an error. ::: **Returned value** @@ -937,11 +998,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 128-bit integer value. [Int128](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -971,7 +1028,7 @@ Result: ## toInt128OrZero -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `0`. +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -981,22 +1038,23 @@ toInt128OrZero(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise `0`. [Int128](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1025,32 +1083,33 @@ Result: ## toInt128OrNull -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns `NULL`. +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt128OrNull(expr) +toInt128OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise `NULL`. [Int128](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1079,7 +1138,7 @@ Result: ## toInt128OrDefault -Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int128`. If unsuccessful, returns the default type value. +Like [`toInt128`](#toint128), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int128`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -1089,26 +1148,28 @@ toInt128OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int128` is unsuccessful. [Int128](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int128'));` +- If the input value cannot be represented within the bounds of [Int128](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 128-bit integer value if successful, otherwise returns the default value. [Int128](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: @@ -1135,7 +1196,7 @@ Result: ## toInt256 -Converts an input value to a value of type `Int256`. +Converts an input value to a value of type [`Int256`](../data-types/int-uint.md). Throws an exception in case of an error. **Syntax** @@ -1145,10 +1206,19 @@ toInt256(expr) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). + +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Unsupported types: +- Float values `NaN` and `Inf` throw an exception. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256('0xc0fe');` :::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. +If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md), the result over or under flows. This is not considered an error. ::: **Returned value** @@ -1156,11 +1226,7 @@ Binary, octal, and hexadecimal representations of numbers are not supported. Lea - 256-bit integer value. [Int256](../data-types/int-uint.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1190,32 +1256,33 @@ Result: ## toInt256OrZero -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `0`. +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** ```sql -toInt256OrZero(expr) +toInt256OrZero(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `0` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrZero('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise `0`. [Int256](../data-types/int-uint.md). :::note -Functions uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1244,32 +1311,33 @@ Result: ## toInt256OrNull -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns `NULL`. +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns [`NULL`](../data-types/nullable.md). **Syntax** ```sql -toInt256OrNull(expr) +toInt256OrNull(x) ``` **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `x` — A String representation of a number. [String](../data-types/string.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- String representations of (U)Int8/16/32/128/256 + +Types for which `\N` is returned: +- String representations of ordinary Float32/64 values. +- String representations of Float values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrNull('0xc0fe');`. +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise `NULL`. [Int256](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note -Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. -::: - -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. +The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. ::: **Example** @@ -1298,7 +1366,7 @@ Result: ## toInt256OrDefault -Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type `Int256`. If unsuccessful, returns the default type value. +Like [`toInt256`](#toint256), takes an argument of type [String](../data-types/string.md) and tries to parse it to type [`Int256`](../data-types/int-uint.md). If unsuccessful, returns the default type value. **Syntax** @@ -1308,26 +1376,28 @@ toInt256OrDefault(expr, def) **Arguments** -- `expr` — Expression returning a number or a string with the decimal representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). - `def` — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). -:::note -Binary, octal, and hexadecimal representations of numbers are not supported. Leading zeroes are stripped. -::: +Supported types: +- (U)Int8/16/32/64/128/256 +- Float* +- String representations of (U)Int8/16/32/128/256 + +Types for which the default value is returned: +- Float values `NaN` and `Inf` return the default value. +- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int256'));` +- If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** - 256-bit integer value if successful, otherwise returns the default value. [Int256](../data-types/int-uint.md). :::note -- Function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. +- The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. - The default value type should be the same as the cast type. ::: -:::danger -An exception is thrown for [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments. Keep in mind [numeric conversions issues](#common-issues-with-data-conversion), when using this function. -::: - **Example** Query: From 3c1004aee4a3b1f3e1b0bd91a1b02c6c9e16c832 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 25 Jul 2024 21:26:14 +0200 Subject: [PATCH 0304/1170] Fix typo --- .../functions/type-conversion-functions.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 844d957d538..4326753216e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -131,7 +131,7 @@ Types for which `0` is returned: - String representations of ordinary Float32/64 values. - String representations of Float values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrZero('0xc0fe');`. -- If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), and the result over or under flows. +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. **Returned value** @@ -186,7 +186,7 @@ Types for which `\N` is returned: - String representations of ordinary Float32/64 values. - String representations of Float values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toInt8OrNull('0xc0fe');`. -- If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), and the result over or under flows. +- If the input value cannot be represented within the bounds of [Int8](../data-types/int-uint.md), and the result over or under flows. **Returned value** @@ -300,7 +300,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt16('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: `SELECT toInt16(32768) == -32768;`, `SELECT toInt16(32768) == -32768;`, `SELECT toInt16('32768') == -32768;`. ::: @@ -527,7 +527,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt32('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [toInt16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int32](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: ``` SELECT toInt32(2147483648) == -2147483648; @@ -755,7 +755,7 @@ Unsupported types: - String representations of binary and hexadecimal values, e.g. `SELECT toInt64('0xc0fe');` :::note -If the input value cannot be represented within the bounds of [Int16](../data-types/int-uint.md), the result over or under flows. This is not considered an error. +If the input value cannot be represented within the bounds of [Int64](../data-types/int-uint.md), the result over or under flows. This is not considered an error. For example: ``` @@ -1386,7 +1386,7 @@ Supported types: Types for which the default value is returned: - Float values `NaN` and `Inf` return the default value. -- String representations of binary and hexadecimal values, e.g. `SELECT toInt128OrDefault('0xc0fe', CAST('-1', 'Int256'));` +- String representations of binary and hexadecimal values, e.g. `SELECT toInt256OrDefault('0xc0fe', CAST('-1', 'Int256'));` - If the input value cannot be represented within the bounds of [Int256](../data-types/int-uint.md) and the result over or under flows. **Returned value** From f2e83f092d1f677c4e0240e749f96766ff6e205c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 25 Jul 2024 21:56:42 +0200 Subject: [PATCH 0305/1170] Patch getauxval for tsan re-exec --- base/glibc-compatibility/CMakeLists.txt | 10 ++++++++++ base/glibc-compatibility/musl/getauxval.c | 4 ++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index c967fa5b11b..8948e25cb8e 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -18,6 +18,16 @@ if (GLIBC_COMPATIBILITY) message (FATAL_ERROR "glibc_compatibility can only be used on x86_64 or aarch64.") endif () + if (SANITIZE STREQUAL thread) + # Disable TSAN instrumentation that conflicts with re-exec due to high ASLR entropy using getauxval + # See longer comment in __auxv_init_procfs + # In the case of tsan we need to make sure getauxval is not instrumented as that would introduce tsan + # internal calls to functions that depend on a state that isn't initialized yet + set_source_files_properties( + musl/getauxval.c + PROPERTIES COMPILE_FLAGS "-mllvm -tsan-instrument-func-entry-exit=false") + endif() + # Need to omit frame pointers to match the performance of glibc set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index 28cb0f8d005..ec2cce1e4aa 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -102,7 +102,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_procfs(unsigned long type) /// most global variables aren't initialized or available yet, so we can't initiate the auxiliary vector. /// Normal glibc / musl getauxval doesn't have this problem since they initiate their auxval vector at the very /// start of __libc_start_main (just keeping track of argv+argc+1), but we don't have such option (otherwise - // this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). + /// this complexity of reading "/proc/self/auxv" or using __environ would not be necessary). /// To avoid this crashes on the re-exec call (see above how it would fail when creating `aux`, and if we used /// __auxv_init_environ then it would SIGSEV on READing `__environ`) we capture this call for `AT_EXECFN` and @@ -237,7 +237,7 @@ static unsigned long NO_SANITIZE_THREAD __auxv_init_environ(unsigned long type) // - __auxv_init_procfs -> __auxv_init_environ -> __getauxval_environ static void * volatile getauxval_func = (void *)__auxv_init_procfs; -unsigned long getauxval(unsigned long type) +unsigned long NO_SANITIZE_THREAD getauxval(unsigned long type) { return ((unsigned long (*)(unsigned long))getauxval_func)(type); } From 3f70977cd660e4617d9bbd68cc229020adc57f98 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 25 Jul 2024 21:02:30 +0000 Subject: [PATCH 0306/1170] try to fix --- ...2572_query_views_log_background_thread.reference | 13 +++++++++---- .../02572_query_views_log_background_thread.sql | 8 ++++++-- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index 22dfaf93781..f867fd0d085 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -1,15 +1,14 @@ -- { echoOn } insert into buffer_02572 values (1); -- ensure that the flush was not direct +select * from buffer_02572; +1 select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; -select * from data_02572; -1 -select * from copy_02572; -1 +select sleepEachRow(1) from numbers(3*2) format Null; system flush logs; select count() > 0, lower(status::String), errorCodeToName(exception_code) from system.query_views_log where @@ -18,3 +17,9 @@ select count() > 0, lower(status::String), errorCodeToName(exception_code) group by 2, 3 ; 1 queryfinish OK +select * from buffer_02572; +1 +select * from data_02572; +1 +select * from copy_02572; +1 diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql index 939c189c5fe..2e9a62b71da 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql @@ -19,13 +19,13 @@ create materialized view mv_02572 to copy_02572 as select * from data_02572; -- { echoOn } insert into buffer_02572 values (1); -- ensure that the flush was not direct +select * from buffer_02572; select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; -select * from data_02572; -select * from copy_02572; +select sleepEachRow(1) from numbers(3*2) format Null; system flush logs; select count() > 0, lower(status::String), errorCodeToName(exception_code) @@ -34,3 +34,7 @@ select count() > 0, lower(status::String), errorCodeToName(exception_code) view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') group by 2, 3 ; + +select * from buffer_02572; +select * from data_02572; +select * from copy_02572; \ No newline at end of file From f0faa111d73c8512c1f88009f0ecfd1a804de45c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 Jul 2024 23:19:58 +0200 Subject: [PATCH 0307/1170] Fix wrong usage of input_format_max_bytes_to_read_for_schema_inference --- src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 1271cdfb7ad..e8956159714 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -253,7 +253,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation; format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; - format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; + format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_bytes_to_read_for_schema_inference; format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference; format_settings.schema_inference_hints = settings.schema_inference_hints; format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable; From 321766d0b8161a794f11835b4650d30b3723835b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 25 Jul 2024 22:51:14 +0000 Subject: [PATCH 0308/1170] Automatic style fix --- tests/performance/scripts/perf.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/performance/scripts/perf.py b/tests/performance/scripts/perf.py index f89784a0e0b..83d66997677 100755 --- a/tests/performance/scripts/perf.py +++ b/tests/performance/scripts/perf.py @@ -349,9 +349,7 @@ for query_index in queries_to_run: try: c.execute("SYSTEM JEMALLOC PURGE") - print( - f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}" - ) + print(f"purging jemalloc arenas\t{conn_index}\t{c.last_query.elapsed}") except KeyboardInterrupt: raise except: From 3f483e547572c47d137d0f2664bd82c0b978ac7b Mon Sep 17 00:00:00 2001 From: heguangnan Date: Fri, 26 Jul 2024 12:09:03 +0800 Subject: [PATCH 0309/1170] fix memory leak when exception happend during count distinct for null key --- src/Interpreters/Aggregator.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e073b7a49b6..543fd8a0bf2 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -3300,6 +3300,17 @@ void NO_INLINE Aggregator::destroyImpl(Table & table) const data = nullptr; }); + + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) + { + if (table.getNullKeyData() != nullptr) + { + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_functions[i]->destroy(table.getNullKeyData() + offsets_of_aggregate_states[i]); + + table.getNullKeyData() = nullptr; + } + } } From 287cce7d211b9386895a4fa898f87405eccb3e96 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 09:20:15 +0200 Subject: [PATCH 0310/1170] Fixes --- .gitmodules | 2 +- contrib/numactl | 2 +- docker/test/performance-comparison/run.sh | 1 + programs/server/Server.cpp | 9 ++++++++- 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index b5d7e1e56b3..7e0b4df4ad1 100644 --- a/.gitmodules +++ b/.gitmodules @@ -374,4 +374,4 @@ url = https://github.com/ClickHouse/double-conversion.git [submodule "contrib/numactl"] path = contrib/numactl - url = https://github.com/numactl/numactl.git + url = https://github.com/ClickHouse/numactl.git diff --git a/contrib/numactl b/contrib/numactl index 3871b1c42fc..8d13d63a05f 160000 --- a/contrib/numactl +++ b/contrib/numactl @@ -1 +1 @@ -Subproject commit 3871b1c42fc71bceadafd745d2eff5dddfc2d67e +Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 diff --git a/docker/test/performance-comparison/run.sh b/docker/test/performance-comparison/run.sh index 7afb5da59b1..6ef781fa4c8 100644 --- a/docker/test/performance-comparison/run.sh +++ b/docker/test/performance-comparison/run.sh @@ -13,6 +13,7 @@ entry="/usr/share/clickhouse-test/performance/scripts/entrypoint.sh" # https://www.kernel.org/doc/Documentation/filesystems/tmpfs.txt # Double-escaped backslashes are a tribute to the engineering wonder of docker -- # it gives '/bin/sh: 1: [bash,: not found' otherwise. +numactl --hardware node=$(( RANDOM % $(numactl --hardware | sed -n 's/^.*available:\(.*\)nodes.*$/\1/p') )); echo Will bind to NUMA node $node; numactl --cpunodebind=$node --membind=$node $entry diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b9a7c298f00..b818ff1f3a2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -775,9 +775,16 @@ try LOG_INFO( log, - "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes {}", + "ClickHouse is bound to a subset of NUMA nodes. Total memory of all available nodes: {}", ReadableSize(total_numa_memory)); } + else + { + LOG_TRACE( + log, + "All NUMA nodes are used. Detected NUMA nodes: {}", + numa_num_configured_nodes()); + } numa_bitmask_free(membind); } From 10b36c09e9ffd96f53d2fdada725f9c999241dd1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 26 Jul 2024 15:38:26 +0800 Subject: [PATCH 0311/1170] support set orc reader time zone name --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/NativeORCBlockInputFormat.cpp | 6 +----- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 10 +++++----- 6 files changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a665326afc..a5220c3017c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1073,7 +1073,7 @@ class IColumn; M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \ - M(Bool, input_format_orc_read_use_writer_time_zone, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ + M(String, input_format_orc_reader_time_zone_name, "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index acd119c159b..457caa76bb6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -69,7 +69,7 @@ static std::initializer_listgetWriterTimezone(); - row_reader_options.setTimezoneName(writer_time_zone); - } + row_reader_options.setTimezoneName(format_settings.orc.reader_time_zone_name); row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); if (format_settings.orc.filter_push_down && sarg) { diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 27530c06237..7e931e16e48 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test SETTINGS session_timezone='Asia/Shanghai'" -$CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" +$CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From ad4e807cf4bb3633616b01e3616844fe2108d59f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 08:12:01 +0000 Subject: [PATCH 0312/1170] Fix stupid crash. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4fd6f7a2900..41306a79198 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2055,9 +2055,9 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & prewhere_info = *query_info.prewhere_info; - auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); if (prewhere_info.row_level_filter) { + auto row_level_actions = std::make_shared(prewhere_info.row_level_filter->clone()); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared(header, From de84f4f045f5ece627ca8295a09a5f2cf1eab6aa Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 26 Jul 2024 08:13:21 +0000 Subject: [PATCH 0313/1170] add proper cast to lagInFrame/leadInFrame --- src/Processors/Transforms/WindowTransform.cpp | 86 ++++++++++++++++--- src/Processors/Transforms/WindowTransform.h | 1 + 2 files changed, 75 insertions(+), 12 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 06ae2bfb25e..006593edeaa 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,6 +16,9 @@ #include #include #include +#include +#include +#include #include #include @@ -75,6 +78,8 @@ public: 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; } }; @@ -1171,6 +1176,9 @@ void WindowTransform::appendChunk(Chunk & chunk) // Initialize output columns. for (auto & ws : workspaces) { + 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.back()->reserve(block.rows); @@ -2358,6 +2366,8 @@ public: template struct WindowFunctionLagLeadInFrame final : public WindowFunction { + FunctionBasePtr func_cast = nullptr; + WindowFunctionLagLeadInFrame(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) : WindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) @@ -2385,18 +2395,71 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction return; } - if (!argument_types[0]->equals(*argument_types[2])) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument type '{}' and the default value type '{}' are different", - argument_types[0]->getName(), - argument_types[2]->getName()); - if (argument_types.size() > 3) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function '{}' accepts at most 3 arguments, {} given", name, argument_types.size()); } + + if (argument_types[0]->equals(*argument_types[2])) + return; + + const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); + if (!supertype) + { + 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, + "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(), + argument_types[2]->getName()); + } + + 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(), + "" + } + }; + + auto get_cast_func = [&arguments] + { + FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::accurate, {}); + return func_builder_cast->build(arguments); + }; + + func_cast = get_cast_func(); + + } + + ColumnPtr castColumn(const Columns & columns, const std::vector & idx) override + { + 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(), + "" + } + }; + + return func_cast->execute(arguments, argument_types[0], columns[idx[2]]->size()); } static DataTypePtr createResultType(const DataTypes & argument_types_, const std::string & name_) @@ -2446,12 +2509,11 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types.size() > 2) { // Column with default values is specified. - // The conversion through Field is inefficient, but we accept - // subtypes of the argument type as a default value (for convenience), - // and it's a pain to write conversion that respects ColumnNothing - // and ColumnConst and so on. - const IColumn & default_column = *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]); } else diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index 43fa6b28019..fe4f79e997c 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -50,6 +50,7 @@ struct WindowTransformBlock { Columns original_input_columns; Columns input_columns; + Columns casted_columns; MutableColumns output_columns; size_t rows = 0; From 3cbb3dc55f6582bc8abc7d5683080702080adcd8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 08:41:44 +0000 Subject: [PATCH 0314/1170] 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 5bf89a433128985944cb5dd6ad6ef40a9658ff52 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 08:42:52 +0000 Subject: [PATCH 0315/1170] Set a different instance dir when using pytest-xdist This allows executing in integration tests in parallel without directory clashes. --- tests/integration/helpers/cluster.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 548b58a17e8..0c8278048bf 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -434,6 +434,11 @@ class ClickHouseCluster: # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r"[^a-z0-9]", "", project_name.lower()) self.instances_dir_name = get_instances_dir(self.name) + xdist_worker = os.getenv("PYTEST_XDIST_WORKER") + if xdist_worker: + self.project_name += f"_{xdist_worker}" + self.instances_dir_name += f"_{xdist_worker}" + self.instances_dir = p.join(self.base_dir, self.instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, "docker.log") self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) From aaa25454b31d854338200b335d7ac6442e959af4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 26 Jul 2024 10:58:45 +0200 Subject: [PATCH 0316/1170] Additional formatting fixes --- .../functions/type-conversion-functions.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4326753216e..87d824ec5bb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -571,7 +571,7 @@ Result: ## toInt32OrZero -Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt32`](#toint32), this function converts an input value to a value of type [Int32](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -767,7 +767,7 @@ SELECT toInt64('9223372036854775808') == --9223372036854775808; **Returned value** -- 64-bit integer value. [Int64](../data-types/int-uint.md). [Int64](../data-types/int-uint.md). +- 64-bit integer value. [Int64](../data-types/int-uint.md). :::note The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -800,7 +800,7 @@ Result: ## toInt64OrZero -Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt64`](#toint64), this function converts an input value to a value of type [Int64](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -878,7 +878,7 @@ Types for which `\N` is returned: **Returned value** -- Integer value of type `Int64` if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). +- 64-bit integer value if successful, otherwise `NULL`. [Int64](../data-types/int-uint.md) / [NULL](../data-types/nullable.md). :::note The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -935,7 +935,7 @@ Types for which the default value is returned: **Returned value** -- Integer value of type `Int64` if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). +- 64-bit integer value if successful, otherwise returns the default value. [Int64](../data-types/int-uint.md). :::note - The function uses [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning it truncates fractional digits of numbers. @@ -1028,7 +1028,7 @@ Result: ## toInt128OrZero -Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt128`](#toint128), this function converts an input value to a value of type [Int128](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** @@ -1256,7 +1256,7 @@ Result: ## toInt256OrZero -Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int8](../data-types/int-uint.md) but returns `0` in case of an error. +Like [`toInt256`](#toint256), this function converts an input value to a value of type [Int256](../data-types/int-uint.md) but returns `0` in case of an error. **Syntax** From 8d13461fb74fc991b73382d04a9bc7a9fd3425fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 09:03:37 +0000 Subject: [PATCH 0317/1170] Another fix. --- src/Storages/StorageDistributed.cpp | 7 ++----- src/Storages/StorageMerge.cpp | 8 ++++++++ 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9b417cda177..07892971ec2 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -847,11 +847,8 @@ void StorageDistributed::read( /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ - if (processed_stage != QueryProcessingStage::Complete) - { - for (auto & column : header) - column.column = column.column->convertToFullColumnIfConst(); - } + for (auto & column : header) + column.column = column.column->convertToFullColumnIfConst(); modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f5bc183931f..9962da3d6de 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -369,6 +369,14 @@ void StorageMerge::read( /// What will be result structure depending on query processed stage in source tables? Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); + if (local_context->getSettingsRef().allow_experimental_analyzer && processed_stage == QueryProcessingStage::Complete) + { + /// Remove constants. + /// For StorageDistributed some functions like `hostName` that are constants only for local queries. + for (auto & column : common_header) + column.column = column.column->convertToFullColumnIfConst(); + } + auto step = std::make_unique( column_names, query_info, From 2519f9ed4252020c6a9fb21ef1410c87f4053200 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 Jul 2024 12:08:16 +0200 Subject: [PATCH 0318/1170] Only support archs --- contrib/numactl-cmake/CMakeLists.txt | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/contrib/numactl-cmake/CMakeLists.txt b/contrib/numactl-cmake/CMakeLists.txt index 5d086366c7f..a72ff11e485 100644 --- a/contrib/numactl-cmake/CMakeLists.txt +++ b/contrib/numactl-cmake/CMakeLists.txt @@ -1,4 +1,14 @@ -option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) +if (NOT ( + OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_LOONGARCH64)) +) + if (ENABLE_NUMACTL) + message (${RECONFIGURE_MESSAGE_LEVEL} + "numactl is disabled implicitly because the OS or architecture is not supported. Use -DENABLE_NUMACTL=0") + endif () + set (ENABLE_NUMACTL OFF) +else() + option (ENABLE_NUMACTL "Enable numactl" ${ENABLE_LIBRARIES}) +endif() if (NOT ENABLE_NUMACTL) message (STATUS "Not using numactl") From 72ebff825c0752e66cefa4f367ce43ff23d77703 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 Jul 2024 12:19:09 +0200 Subject: [PATCH 0319/1170] Reduce max time of 00763_long_lock_buffer_alter_destination_table --- ...ong_lock_buffer_alter_destination_table.sh | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 7e2384cfc52..c12b4426740 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -16,18 +16,39 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Bu ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" ${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" -function thread1() +function thread_alter() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 300 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + ALTER TABLE mt_00763_1 MODIFY column s UInt32; + ALTER TABLE mt_00763_1 MODIFY column s String; + " ||: + done } -function thread2() +function thread_query() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 2000 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + SELECT sum(length(s)) FROM buffer_00763_1; + " 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + done } -thread1 & -thread2 & +export -f thread_alter +export -f thread_query + +TIMEOUT=30 +thread_alter $TIMEOUT & +thread_query $TIMEOUT & wait From 98418120cd3167983b5436834d7c568cb42865af Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 10:21:43 +0000 Subject: [PATCH 0320/1170] Add parallel integration test execution to doc --- tests/integration/README.md | 67 ++++++++++++++++++++++++++++++++++--- 1 file changed, 62 insertions(+), 5 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index cde4cb05aec..5d4fa407e3f 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -16,7 +16,7 @@ Don't use Docker from your system repository. * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` * [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: -``` +```bash sudo -H pip install \ PyMySQL \ avro \ @@ -78,7 +78,7 @@ Notes: * Some tests maybe require a lot of resources (CPU, RAM, etc.). Better not try large tests like `test_distributed_ddl*` on your laptop. You can run tests via `./runner` script and pass pytest arguments as last arg: -``` +```bash $ ./runner --binary $HOME/ClickHouse/programs/clickhouse --odbc-bridge-binary $HOME/ClickHouse/programs/clickhouse-odbc-bridge --base-configs-dir $HOME/ClickHouse/programs/server/ 'test_ssl_cert_authentication -ss' Start tests ====================================================================================================== test session starts ====================================================================================================== @@ -102,7 +102,7 @@ test_ssl_cert_authentication/test.py::test_create_user PASSED ``` Path to binary and configs maybe specified via env variables: -``` +```bash $ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ $ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse $ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge @@ -121,6 +121,63 @@ test_odbc_interaction/test.py ...... [100%] You can just open shell inside a container by overwritting the command: ./runner --command=bash +### Parallel test execution + +On the CI, we run a number of parallel runners (5 at the time of this writing), each on its own +Docker container. These runner containers spawn more containers for the services needed such as +ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized +using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` +to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In other +words: tests are grouped by module for test functions and by class for test methods. This means that +any test within the same module (or any class) will never execute their tests in parallel. They'll +be executed on the same worker one after the other. + +If the test supports parallel and repeated execution, you can run a bunch of them in parallel to +look for flakiness. We use [pytest-repeat](https://pypi.org/project/pytest-repeat/) to set the +number of times we want to execute a test through the `--count` argument. Then, `-n` sets the number +of parallel workers for `pytest-xdist`. + +```bash +$ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ +$ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse +$ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge +$ ./runner 'test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' +Start tests +=============================================================================== test session starts ================================================================================ +platform linux -- Python 3.10.12, pytest-7.4.4, pluggy-1.5.0 -- /usr/bin/python3 +cachedir: .pytest_cache +rootdir: /ClickHouse/tests/integration +configfile: pytest.ini +plugins: reportlog-0.4.0, xdist-3.5.0, random-0.2, repeat-0.9.3, order-1.0.0, timeout-2.2.0 +timeout: 900.0s +timeout method: signal +timeout func_only: False +5 workers [10 items] +scheduling tests via LoadScheduling + +test_storage_s3_queue/test.py::test_max_set_age[9-10] +test_storage_s3_queue/test.py::test_max_set_age[7-10] +test_storage_s3_queue/test.py::test_max_set_age[5-10] +test_storage_s3_queue/test.py::test_max_set_age[1-10] +test_storage_s3_queue/test.py::test_max_set_age[3-10] +[gw3] [ 10%] PASSED test_storage_s3_queue/test.py::test_max_set_age[7-10] +test_storage_s3_queue/test.py::test_max_set_age[8-10] +[gw4] [ 20%] PASSED test_storage_s3_queue/test.py::test_max_set_age[9-10] +test_storage_s3_queue/test.py::test_max_set_age[10-10] +[gw0] [ 30%] PASSED test_storage_s3_queue/test.py::test_max_set_age[1-10] +test_storage_s3_queue/test.py::test_max_set_age[2-10] +[gw1] [ 40%] PASSED test_storage_s3_queue/test.py::test_max_set_age[3-10] +test_storage_s3_queue/test.py::test_max_set_age[4-10] +[gw2] [ 50%] PASSED test_storage_s3_queue/test.py::test_max_set_age[5-10] +test_storage_s3_queue/test.py::test_max_set_age[6-10] +[gw3] [ 60%] PASSED test_storage_s3_queue/test.py::test_max_set_age[8-10] +[gw4] [ 70%] PASSED test_storage_s3_queue/test.py::test_max_set_age[10-10] +[gw0] [ 80%] PASSED test_storage_s3_queue/test.py::test_max_set_age[2-10] +[gw1] [ 90%] PASSED test_storage_s3_queue/test.py::test_max_set_age[4-10] +[gw2] [100%] PASSED test_storage_s3_queue/test.py::test_max_set_age[6-10] +========================================================================== 10 passed in 120.65s (0:02:00) ========================================================================== +``` + ### Rebuilding the docker containers The main container used for integration tests lives in `docker/test/integration/base/Dockerfile`. Rebuild it with @@ -149,7 +206,7 @@ will automagically detect the types of variables and only the small diff of two If tests failing for mysterious reasons, this may help: -``` +```bash sudo service docker stop sudo bash -c 'rm -rf /var/lib/docker/*' sudo service docker start @@ -159,6 +216,6 @@ sudo service docker start On Ubuntu 20.10 and later in host network mode (default) one may encounter problem with nested containers not seeing each other. It happens because legacy and nftables rules are out of sync. Problem can be solved by: -``` +```bash sudo iptables -P FORWARD ACCEPT ``` From 60cca77c8a415142fe6f181b25aaed84232ea3c0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 26 Jul 2024 12:39:35 +0200 Subject: [PATCH 0321/1170] add example for materialize function --- .../functions/other-functions.md | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b7e4094f30e..797607e552a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -346,7 +346,9 @@ Result: ## materialize Turns a constant into a full column containing a single value. -Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. +Full columns and constants are represented differently in memory. +Functions usually execute different code for normal and constant arguments, although the result should typically be the same. +This function can be used to debug this behavior. **Syntax** @@ -354,6 +356,34 @@ Full columns and constants are represented differently in memory. Functions usua materialize(x) ``` +**Parameters** + +- `x` — A constant. [Constant](../functions/index.md/#constants). + +**Returned value** + +- A column containing a single value `x`. + +**Example** + +In the example below the `countMatches` function expects a constant second argument. +This behaviour can be debugged by using the `materialize` function to turn a constant into a full column, +verifying that the function throws an error for a non-constant argument. + +Query: + +```sql +SELECT countMatches('foobarfoo', 'foo'); +SELECT countMatches('foobarfoo', materialize('foo')); +``` + +Result: + +```response +2 +Code: 44. DB::Exception: Received from localhost:9000. DB::Exception: Illegal type of argument #2 'pattern' of function countMatches, expected constant String, got String +``` + ## ignore Accepts any arguments, including `NULL` and does nothing. Always returns 0. From 73e71e7b7a476c7073154a112dd12815bbe49bfe Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 08:45:43 +0000 Subject: [PATCH 0322/1170] 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 3f1dbdfce978bab2b2ce2aedecdbb5afbf54c4a0 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 11:01:10 +0000 Subject: [PATCH 0323/1170] Clarify documentation --- tests/integration/README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index 5d4fa407e3f..ab984b7bd04 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -127,10 +127,10 @@ On the CI, we run a number of parallel runners (5 at the time of this writing), Docker container. These runner containers spawn more containers for the services needed such as ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` -to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In other -words: tests are grouped by module for test functions and by class for test methods. This means that -any test within the same module (or any class) will never execute their tests in parallel. They'll -be executed on the same worker one after the other. +to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the +documentation words: this guarantees that all tests in a file run in the same worker. This means +that any test within the same file will never execute their tests in parallel. They'll be executed +on the same worker one after the other. If the test supports parallel and repeated execution, you can run a bunch of them in parallel to look for flakiness. We use [pytest-repeat](https://pypi.org/project/pytest-repeat/) to set the From 28e991708be1facd87c3760f7929cd5ddc299805 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 13:45:57 +0200 Subject: [PATCH 0324/1170] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 18666 -> 20353 bytes tests/queries/0_stateless/00309_formats.sql | 5 ----- 2 files changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index f3ea45520bb50fb936caf6724e9fedf3cdd00b75..cab311692be229716b58af39079275d3942b01cc 100644 GIT binary patch literal 20353 zcmeHP3wIP%5{6aK9zDL_&p18;axq=E-!KeHeM!>@7a6Ht;zJ#Jk0d+*gt^JbdYKDu4%*X9d$a^ZOYUl#FBHF2yr}q<;@67HiYtm^#Z|>iir*-% zDXuF{6fY}&tN5Mb_lj2(e^C5U@h8Qria#snd+xpO{s$g>=;22meeCfko}78A|MWA@J~z8>|MLgJ!58LIidkNiRoxsq zJUB8u`r@(UFP(V#6D$6wrgio0oJnBji=r;A-&v0K-- z%ZqMZ-|En<>suYVb$zQtx30$|*-G(uTzhnP-g&=^d;Asr39~@AuK#;~vDmn-A16x* zR`tnxBCGmdyM|AAwVs$&ebTP#ldV?ulqNAv(}O7;BueJfKp}=K;v>=KamanSU6UWrDN?_JXVk8gMI`72?`P<#P@i7&<`pS zWF+WF5R#xIK}v#_1ThI}667T4Nf4BvC_z$!rUX$5suE-+=t>Zlpe#XJg0=*43F;E$ zCFn~Kn4mC0VuHp5kqIgjWG3iL5SpMgL28251hENf6XYi7O%R-*I6-oP<^<6RsuN@< z=uQxxpgci(g7yUQ3F;H%C+JUb0Ko$U7Z7|va00;#1UC@;KyU=X69iWfd_iyq!5aj3 z5d48~#s?7ye?<5r!XFX-i10^*KO+1Q;g1M^M8t7~KO+1Q;g1M^MEE1Z9})hD9M2rk z64B_#G0j2EQO#k^am|4Ze?<5r!XFX-i0}tWF!-P!{1M@g2!BNQBf=jM{)q5Lgg+wu z5#f&re?<5r!XFX-i10^*KO+1Q;g1M^#G){NzzgHs7A`ETZCepq_933?i~U2rWWJXqI8KuOcyVRaKQz-H_k$cyu3VDiwye}g+f_bcJ1@D;TzuN5 zD@s+avnf(-W3}#V+EUgwouv-u47MnGiFGeI6M7J0X>pSftOBKaoh3fnm@GvczL`Ta z^L_N{6Fw0i&Ve~T1wJJ{6+SgSO&=q%!s~pgh07?nk!+)EGuh^{Eo58Dw&Hr*^k(>Z zybQ>^Hqtrc{32`%3@;;YAm1kG-$5QXD0j)XNqnD@?1}G`_*RMUmH1|f@0J>DqOcTU z*dnqtku4%yM7D@*5!oWLMaC8xTV$doTV!mJu|>uf8Czs*k+DV27CBqwY>|unY>~4? z&K5aasMw-ni;68Owy4;mVvC9`Dz>QDqGpSlEo!!?*`j8Pnk{O!sM(@si<&JOwrJR* zVT*<>8n$TIqG5}MEgH6H*a8(t*|H6)jlxDpGfP)fUuE8&1TrL@Sn5)r6Ya*MU7eL0wb!Wn76AR5ztX z#+6_}jZ<1=T!|M{I=Mw62Gvg4mvJR%Q1g@)8CT*4l}~AraV2z6|Kt`49@Ic(U&fXA zK_ygLWLya$)I+&Nf(SKH*_Ux8j!+qu78zGU3H4EFk#QxOP$lISi6>M_WnadXkV3sw zT4Y>_DpXCSg~!FbC|e#E)l=EBEh?z8Wm{BHWy|BDk}6xVZKpC4TZt>wRQasL73!+8 zFSZg_sIE$j*tQdl#8s5ILYJI}Nejl>nIuG~}N3YAyc7h8!d)L*4V z#+A526;@hgT!|~xV!1`)3iVjom;05tLRD5;5#VP`KA{>G~DrX+|J4NE4^tt78W7My1Ewbk!9(?&_}d~o*M=)&sS=v zm5YR#+gx;@nz8VMIrYYX7g?<00RC^)91khQIPJmG@OrgrFt^k;LYv_Lo5d@`!DxB3 zGDLsD_K3_;G)I{^%FR(>j!JV>nWNeqHPfRx<;2)BT_0Y1Xf&A9h!Y`)0|nvDH@x8n zHr{gEJg2=CS+6-2Y@=}a^V*~wFA#5d<+f6RtRKlbl5c8*WMo}Q)|X_RN!FWW-AUK{ zDc6tAH={hZdv7)Rc=}3mQ`+9<(5@$S>={hVObTHRbg2bmQ`U{6_yqMxorr-!m=tXtHQD> zEvwS9DlMzhvMMdB(y}TotJ1P6EvwS9sw}I@vZ^er%Cf2~tID#fEUU`0sw}I@vZ^hs z+On!GtJ<=vEvwqHsx7P9vZ^hs+OldatH!cwEUU({YAmb9vT7`=#}TLyo>A3TXJ|u*PXZY@b0cX?~)z)dEQU-ex~gv+M8t>k(P&*cKVkL~(9{aiZ0vklwdB?P?7?an*> zTu#9I-mbsX&!q*_8f<@;81VeWop<`V+<^DVU4N&aOAmPG-1T?*xeS5#(rtg2Bk;7w zop<`VG=X>8U4N&a%M^I;-Sv0+xnzOoK(@b27kFphop<`VjDh#+U4N&aOB#6h-u8EC z1J9b=d8ePt9C&Zv^>_NY_NY{DCT;>+ke)`2)`|ZGV?P@Y@r2-syKefBb6wVB0s=_`E`L+6m)Ezc_5xV{`a(e6A7cfwpXiGp`$XFl3xq_$>F>d{|a@uQ~ Lvprlx%YOd>&)TR8 delta 3052 zcmeIy$4}c(00wZIgtksdZ-w^I3JD}eL2?j6{R4XG1$t-?y;Q0c4wb5?2YL>Bj|8*# z-Xp>6y?59U_TGDMNaN%a?SDX3#j^bK`<|bk?ZfAlN^UDiNE{JVHgaD5e8+19DxExLIy^` zXcz-yVH}Ky2`~{R!DN^MQ}xOSA`L-0%z&9N3ueO{m<#h@J}iKRum~2z5?IQq+n-P( zP{A_D!g5#vD`6F^hBdGj*1>w%02^VGPHumqnY-wY76U{pY=iBv19rkL*bRGNFYJT; zPz?v*AoeGQ5Ddc+I10z$IGlhQI0>iVG@OC6a1PGH1;h6f#3F(vxC~d|DqMr>a070_ zEw~ML;4a*Q`|v>j|8;{y1V``~p1@Oh2G8LIyo6Wq8s5NLcn9yHg%&qjk~HD*O`kk{ zMhS#wBF4@26;n)ok$HM~`}q3#2LuKMhlGZON5~bCOjLACY+QUoVp4KSYFc_mW>$7i zZeD&tVNr2OsZv$OmRD3(RoB$k)i*RYHMg|3wRd!Ob@%l4^{WR4hlWQ+$Hpf#lT*_( zvvczci%ZKZt842Un_JsEyL2uP zs~KC(*lPbltFhlLRQ$(oSyIc$t;|{F>ph;XVgHfpC$+5H%8ajnp%+Y<&DY2OORU_k z(V7&oZmv$Gj7LgIiK)l0pFiA9KYx&VyyL(B=HYJsi3@C{mioxSQew%acGgGsx>HI? tDE$HfH|yd|adXt)D_oo(mI%Z& reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; - -SET enable_named_columns_in_function_tuple = 1; - -SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From d041df80aa112920f28d74ed26a0c8381808dafc Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 26 Jul 2024 12:14:26 +0000 Subject: [PATCH 0325/1170] Add test --- ..._to_read_for_schema_inference_in_cache.reference | 2 ++ ...x_bytes_to_read_for_schema_inference_in_cache.sh | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference create mode 100755 tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference new file mode 100644 index 00000000000..cd109daac52 --- /dev/null +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference @@ -0,0 +1,2 @@ +x Nullable(Int64) +schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh new file mode 100755 index 00000000000..8a77538f592 --- /dev/null +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS input_format_max_bytes_to_read_for_schema_inference=1000; +SELECT additional_format_info from system.schema_inference_cache" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.json + From d06b9ca99669ed3cfdf1beb68c9fbc1b90180536 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Jul 2024 14:47:38 +0200 Subject: [PATCH 0326/1170] Ping CI From aec7848525d2d6bd1cc9e7c573c25bd3b4ac79e7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 26 Jul 2024 13:04:44 +0000 Subject: [PATCH 0327/1170] fix --- src/Processors/Transforms/WindowTransform.cpp | 2 +- .../03210_lag_lead_inframe_types.reference | 20 +++++++++++++++++++ .../03210_lag_lead_inframe_types.sql | 4 ++-- 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 61be0c32a7d..1eac08780e9 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2424,7 +2424,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction argument_types[0]->getName(), argument_types[2]->getName()); } - + const auto from_name = argument_types[2]->getName(); const auto to_name = argument_types[0]->getName(); ColumnsWithTypeAndName arguments diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference index cc3b9a096b9..d4734a85e72 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference @@ -18,3 +18,23 @@ 7 8 9 +0 +1 +2 +2 +2 +2 +2 +2 +2 +2 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql index 5466cfe0fad..f6017ee6690 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql @@ -1,4 +1,4 @@ -SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } -SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); -- { serverError BAD_ARGUMENTS } +SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); +SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); From 42384af0ef38dd326337e8cf18327871924f7359 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 25 Jul 2024 15:42:50 +0000 Subject: [PATCH 0328/1170] Fix crash when the connection is empty --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 6 ++++++ .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 3 +++ 2 files changed, 9 insertions(+) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index e1facec5b40..31779a32c1f 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -245,6 +245,9 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha connection = std::move(result.front().entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", files.size(), connection->getDescription(), @@ -303,6 +306,9 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett auto connection = std::move(result.front().entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + RemoteInserter remote(*connection, timeouts, distributed_header.insert_query, insert_settings, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..15998776d27 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -415,6 +415,9 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto result = pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); auto connection = std::move(result.front().entry); + if (connection.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, connection->getDescription(), From 981135bfb104b5ecfa0f1da5533e3d12f6850838 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 26 Jul 2024 13:38:42 +0000 Subject: [PATCH 0329/1170] Fix style check --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 5 +++-- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 ++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 31779a32c1f..5e7b4b979c7 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } /// Can the batch be split and send files from batch one-by-one instead? @@ -246,7 +247,7 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha compression_expected = connection->getCompression() == Protocol::Compression::Enable; if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", files.size(), @@ -307,7 +308,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); RemoteInserter remote(*connection, timeouts, distributed_header.insert_query, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 15998776d27..2bb0e720c72 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -416,7 +416,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto connection = std::move(result.front().entry); if (connection.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty connection"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index b2ce62caf0a..f8bbc081e55 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -378,6 +378,8 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) auto results = shard_info.pool->getManyCheckedForInsert(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); job.connection_entry = std::move(results.front().entry); + if (job.connection_entry.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty connection"); } else { From eeb012357196db988b01a70b33798fce99bb5deb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 26 Jul 2024 14:48:52 +0000 Subject: [PATCH 0330/1170] fix filling of multilevel Nested --- src/Interpreters/inplaceBlockConversions.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index ce3f25d16f8..945cc62754d 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -316,7 +316,7 @@ static String removeTupleElementsFromSubcolumn(String subcolumn_name, const Name { auto pos = subcolumn_name.find(elem + "."); if (pos != std::string::npos) - subcolumn_name.erase(pos, elem.size()); + subcolumn_name.erase(pos, elem.size() + 1); } if (subcolumn_name.ends_with(".")) @@ -395,7 +395,6 @@ void fillMissingColumns( if (!current_offsets.empty()) { - Names tuple_elements; auto serialization = IDataType::getSerialization(*requested_column); From ff5cd2051fc8bfd609a9040ffba02697283e69af Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 26 Jul 2024 17:10:39 +0200 Subject: [PATCH 0331/1170] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- tests/queries/0_stateless/00307_format_xml.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00307_format_xml.sql b/tests/queries/0_stateless/00307_format_xml.sql index 22566112bc7..a7e0e628945 100644 --- a/tests/queries/0_stateless/00307_format_xml.sql +++ b/tests/queries/0_stateless/00307_format_xml.sql @@ -2,4 +2,4 @@ SET output_format_write_statistics = 0; SELECT 'unnamed columns in tuple'; SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; SELECT 'named columns in tuple'; -SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML;` +SELECT 'Hello & world' AS s, toDateTime('2001-02-03 04:05:06') AS time, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; From 7b4951990edc08a1230c3569339c44ac22036eed Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 26 Jul 2024 15:19:47 +0000 Subject: [PATCH 0332/1170] better place to check and add more test --- src/Interpreters/InterpreterCreateQuery.cpp | 24 -------------- src/Storages/StorageFactory.cpp | 28 ++++++++++++++++ ...206_projection_merge_special_mergetree.sql | 32 +++++++++++++++++++ 3 files changed, 60 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a5f374ba71c..ea10ad59db4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1281,30 +1281,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); - /// Projection is only supported in (Replictaed)MergeTree. - if (create.storage && create.storage->engine) - { - if (std::string_view engine_name(create.storage->engine->name); - !properties.projections.empty() && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") - { - bool projection_support = false; - if (auto * setting = create.storage->settings; setting != nullptr) - { - for (const auto & change : setting->changes) - { - if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) - { - projection_support = true; - break; - } - } - } - if (!projection_support) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Projection is only supported in (Replictaed)MergeTree. Consider drop or rebuild option of deduplicate_merge_projection_mode."); - } - } - /// Check type compatible for materialized dest table and select columns if (create.select && create.is_materialized_view && create.to_table_id && mode <= LoadingStrictnessLevel::CREATE) { diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 060b271d8f4..71f70a807a8 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; extern const int BAD_ARGUMENTS; extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES; + extern const int NOT_IMPLEMENTED; } @@ -196,9 +197,36 @@ StoragePtr StorageFactory::get( [](StorageFeatures features) { return features.supports_skipping_indices; }); if (query.columns_list && query.columns_list->projections && !query.columns_list->projections->children.empty()) + { check_feature( "projections", [](StorageFeatures features) { return features.supports_projections; }); + + /// Now let's handle the merge tree family, projection is fully supported in (Replictaed)MergeTree, + /// but also allowed in non-throw mode with other mergetree family members. + chassert(query.storage->engine); + if (std::string_view engine_name(query.storage->engine->name); + engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") + { + /// default throw mode in deduplicate_merge_projection_mode + bool projection_allowed = false; + if (auto * setting = query.storage->settings; setting != nullptr) + { + for (const auto & change : setting->changes) + { + if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) + { + projection_allowed = true; + break; + } + } + } + if (!projection_allowed) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" + " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode."); + } + } } } diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index 749f906569e..25517fbba30 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -1,5 +1,37 @@ DROP TABLE IF EXISTS tp; +-- test regular merge tree +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = MergeTree order by type; + +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); + +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } + +DROP TABLE tp; + +CREATE TABLE tp ( + type Int32, + eventcnt UInt64, + PROJECTION p (select sum(eventcnt), type group by type) +) engine = MergeTree order by type +SETTINGS deduplicate_merge_projection_mode = 'drop'; + +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); + +OPTIMIZE TABLE tp DEDUPLICATE; + +ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; + +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } + +DROP TABLE tp; + + +-- test irregular merge tree CREATE TABLE tp ( type Int32, eventcnt UInt64, From d42fa0690d1b6ec19755b64740d83327e71a914a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 15:59:23 +0000 Subject: [PATCH 0333/1170] Remove filterBlockWithDAG. --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/System/StorageSystemDroppedTablesParts.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 4 ++-- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 7 +++---- src/Storages/VirtualColumnUtils.h | 2 +- 7 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ecd25e3cf71..d9ab2894dc4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1164,7 +1164,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (valid) { virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_dag), virtual_columns_block, local_context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_dag), local_context), virtual_columns_block); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a37dbfa554c..a6ef0063069 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -488,7 +488,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar return {}; auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts); - VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), virtual_columns_block, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*dag), context), virtual_columns_block); return VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); } diff --git a/src/Storages/System/StorageSystemDroppedTablesParts.cpp b/src/Storages/System/StorageSystemDroppedTablesParts.cpp index defc4ec2d2a..c2601b8ebe3 100644 --- a/src/Storages/System/StorageSystemDroppedTablesParts.cpp +++ b/src/Storages/System/StorageSystemDroppedTablesParts.cpp @@ -75,7 +75,7 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(std::optional f { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter), context), block_to_filter); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index a0c9a5c61bd..7ace8ee24aa 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -124,7 +124,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab /// Filter block_to_filter with column 'database'. if (filter_by_database) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_database), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_by_database), context), block_to_filter); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -204,7 +204,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. if (filter_by_other_columns) - VirtualColumnUtils::filterBlockWithDAG(std::move(*filter_by_other_columns), block_to_filter, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_by_other_columns), context), block_to_filter); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 9ae21ded9ba..943ce9c317a 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -125,7 +125,7 @@ ColumnPtr getFilteredTables( block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); if (dag) - VirtualColumnUtils::filterBlockWithDAG(std::move(*dag), block, context); + VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*dag), context), block); return block.getByPosition(0).column; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index a25b7b5ca49..90c2c7f93c1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -77,11 +77,10 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context) } } -void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context) +ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context) { buildSetsForDAG(dag, context); - auto actions = std::make_shared(std::move(dag)); - filterBlockWithExpression(actions, block); + return std::make_shared(std::move(dag)); } void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block) @@ -384,7 +383,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, { auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); if (dag) - filterBlockWithDAG(std::move(*dag), block, context); + filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 640f9db2fb8..73b7908b75c 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -23,7 +23,7 @@ namespace VirtualColumnUtils void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. -void filterBlockWithDAG(ActionsDAG dag, Block & block, ContextPtr context); +ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context); void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & block); /// Builds sets used by ActionsDAG inplace. From 343f1fa4bae219f7c287cb314ed6e04feb9a0de4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:42:06 +0000 Subject: [PATCH 0334/1170] Check type after optimize_rewrite_aggregate_function_with_if. --- .../RewriteAggregateFunctionWithIfPass.cpp | 32 ++++++++++++++++--- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++- 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index c1adf05ac76..a48e88132a6 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -42,7 +43,7 @@ public: if (lower_name.ends_with("if")) return; - auto & function_arguments_nodes = function_node->getArguments().getNodes(); + const auto & function_arguments_nodes = function_node->getArguments().getNodes(); if (function_arguments_nodes.size() != 1) return; @@ -50,6 +51,8 @@ public: if (!if_node || if_node->getFunctionName() != "if") return; + FunctionNodePtr replaced_node; + auto if_arguments_nodes = if_node->getArguments().getNodes(); auto * first_const_node = if_arguments_nodes[1]->as(); auto * second_const_node = if_arguments_nodes[2]->as(); @@ -75,8 +78,11 @@ public: new_arguments[0] = std::move(if_arguments_nodes[1]); new_arguments[1] = std::move(if_arguments_nodes[0]); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } else if (first_const_node) @@ -104,10 +110,26 @@ public: FunctionFactory::instance().get("not", getContext())->build(not_function->getArgumentColumns())); new_arguments[1] = std::move(not_function); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } + + if (!replaced_node) + return; + + auto prev_type = function_node->getResultType(); + auto curr_type = replaced_node->getResultType(); + if (!prev_type->equals(*curr_type)) + return; + + /// Just in case, CAST compatible aggregate function states. + if (WhichDataType(prev_type).isAggregateFunction() && !DataTypeAggregateFunction::strictEquals(prev_type, curr_type)) + node = createCastFunction(std::move(replaced_node), prev_type, getContext()); + else + node = std::move(replaced_node); } }; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1fe2554988..b1603bb18dd 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,11 +3239,13 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); + std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; + AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - + std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 4833b46a1a86bb1847d2520ea12ea4650c497abc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:43:30 +0000 Subject: [PATCH 0335/1170] Remove debug code --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1603bb18dd..b1fe2554988 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,13 +3239,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); - std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; - AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; + function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 1ba44252cd20ab660d374970257a1ceb438236dd Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 26 Jul 2024 18:33:50 +0000 Subject: [PATCH 0336/1170] turn sql to bash --- ...uery_views_log_background_thread.reference | 25 +----------- ...02572_query_views_log_background_thread.sh | 38 ++++++++++++++++++ ...2572_query_views_log_background_thread.sql | 40 ------------------- 3 files changed, 40 insertions(+), 63 deletions(-) create mode 100755 tests/queries/0_stateless/02572_query_views_log_background_thread.sh delete mode 100644 tests/queries/0_stateless/02572_query_views_log_background_thread.sql diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index f867fd0d085..d7f2272f5b4 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -1,25 +1,4 @@ --- { echoOn } -insert into buffer_02572 values (1); --- ensure that the flush was not direct -select * from buffer_02572; +OK +1 1 -select * from data_02572; -select * from copy_02572; --- we cannot use OPTIMIZE, this will attach query context, so let's wait -SET function_sleep_max_microseconds_per_block = 6000000; -select sleepEachRow(1) from numbers(3*2) format Null; -select sleepEachRow(1) from numbers(3*2) format Null; -system flush logs; -select count() > 0, lower(status::String), errorCodeToName(exception_code) - from system.query_views_log where - view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and - view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3 -; 1 queryfinish OK -select * from buffer_02572; -1 -select * from data_02572; -1 -select * from copy_02572; -1 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 new file mode 100755 index 00000000000..a3e428e75c8 --- /dev/null +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +# INSERT buffer_02572 -> data_02572 -> copy_02572 +# ^^ +# push to system.query_views_log + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists buffer_02572; + drop table if exists data_02572; drop table if exists copy_02572; drop table if exists mv_02572;" + +${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" +${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 materialized view mv_02572 to copy_02572 as select * from data_02572;" + +${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;" + +# we cannot use OPTIMIZE, this will attach query context, so let's wait +for _ in {1..100}; do + $CLICKHOUSE_CLIENT -q "select * from data_02572;" | grep -q "1" && echo 'OK' && break + sleep 0.5 +done + + +${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" + +${CLICKHOUSE_CLIENT} --query="system flush logs;" +${CLICKHOUSE_CLIENT} --query="select count() > 0, lower(status::String), errorCodeToName(exception_code) + from system.query_views_log where + view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and + view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') + group by 2, 3;" \ No newline at end of file diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql deleted file mode 100644 index 2e9a62b71da..00000000000 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ /dev/null @@ -1,40 +0,0 @@ --- INSERT buffer_02572 -> data_02572 -> copy_02572 --- ^^ --- push to system.query_views_log - -drop table if exists buffer_02572; -drop table if exists data_02572; -drop table if exists copy_02572; -drop table if exists mv_02572; - -create table copy_02572 (key Int) engine=Memory(); -create table data_02572 (key Int) engine=Memory(); -create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, - /* never direct flush for flush from background thread */ - /* min_time= */ 3, 3, - 1, 1e9, - 1, 1e9); -create materialized view mv_02572 to copy_02572 as select * from data_02572; - --- { echoOn } -insert into buffer_02572 values (1); --- ensure that the flush was not direct -select * from buffer_02572; -select * from data_02572; -select * from copy_02572; --- we cannot use OPTIMIZE, this will attach query context, so let's wait -SET function_sleep_max_microseconds_per_block = 6000000; -select sleepEachRow(1) from numbers(3*2) format Null; -select sleepEachRow(1) from numbers(3*2) format Null; - -system flush logs; -select count() > 0, lower(status::String), errorCodeToName(exception_code) - from system.query_views_log where - view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and - view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3 -; - -select * from buffer_02572; -select * from data_02572; -select * from copy_02572; \ No newline at end of file From 870ec237bb427243388acbe5bca770241eeb7fbb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Jun 2024 14:14:11 +0200 Subject: [PATCH 0337/1170] Add ability to load dashboards for system.dashboards from config One of the obvious reasons is to allow rendering them with readonly user, which is not possible right now, due to usage of merge() function there. Another one, is to add some custom metrics. Note, that once set, they overrides the default dashboards preset. Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 2 + programs/server/config.xml | 25 ++++++++++ src/Interpreters/Context.cpp | 49 +++++++++++++++++++ src/Interpreters/Context.h | 4 ++ .../System/StorageSystemDashboards.cpp | 26 +++++++--- src/Storages/System/StorageSystemDashboards.h | 2 +- .../test_custom_dashboards/__init__.py | 0 .../configs/config.d/overrides.xml | 15 ++++++ .../test_custom_dashboards/test.py | 35 +++++++++++++ 9 files changed, 149 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_custom_dashboards/__init__.py create mode 100644 tests/integration/test_custom_dashboards/configs/config.d/overrides.xml create mode 100644 tests/integration/test_custom_dashboards/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..f8aea3ad10c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1582,6 +1582,8 @@ try global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); + global_context->setDashboardsConfig(config); + if (global_context->isServerCompletelyStarted()) { /// It does not make sense to reload anything before server has started. diff --git a/programs/server/config.xml b/programs/server/config.xml index 94825a55f67..5dedd78ff2a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1312,6 +1312,31 @@ event_date + INTERVAL 30 DAY + + + + - - create table views_max_insert_threads_null (a UInt64) Engine = Null - create materialized view views_max_insert_threads_mv Engine = Null AS select now() as ts, max(a) from views_max_insert_threads_null group by ts - - insert into views_max_insert_threads_null select * from numbers_mt(3000000000) settings max_threads = 16, max_insert_threads=16 - - drop table if exists views_max_insert_threads_null - drop table if exists views_max_insert_threads_mv - - + + + + + + + + + + From b2d8eaf1e6d67ab76f3e86cd4fd857e9535a9d20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 18:55:08 +0200 Subject: [PATCH 0419/1170] Debug TimerDescriptor --- src/Common/TimerDescriptor.cpp | 26 ++++++++++++++++++++++++-- src/IO/MMappedFileDescriptor.cpp | 2 -- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index b36ea4059cb..9a171ae9487 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -2,9 +2,11 @@ #include #include +#include #include #include +#include namespace DB @@ -89,9 +91,29 @@ void TimerDescriptor::drain() const /// A signal happened, need to retry. if (errno == EINTR) - continue; + { + /** This is to help with debugging. + * + * Sometimes reading from timer_fd blocks, which should not happen, because we opened it in a non-blocking mode. + * But it could be possible if a rogue 3rd-party library closed our file descriptor by mistake + * (for example by double closing due to the lack of exception safety or if it is a crappy code in plain C) + * and then another file descriptor is opened in its place. + * + * Let's try to get a name of this file descriptor and log it. + */ + LoggerPtr log = getLogger("TimerDescriptor"); - throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd"); + static constexpr ssize_t max_link_path_length = 256; + char link_path[max_link_path_length]; + ssize_t link_path_length = readlink(fmt::format("/proc/self/fd/{}", timer_fd).c_str(), link_path, max_link_path_length); + if (-1 == link_path_length) + throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot readlink for a timer_fd {}", timer_fd); + + LOG_TRACE(log, "Received EINTR while trying to drain a TimerDescriptor, fd {}: {}", timer_fd, std::string_view(link_path, link_path_length)); + continue; + } + + throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd {}", timer_fd); } chassert(res == sizeof(buf)); diff --git a/src/IO/MMappedFileDescriptor.cpp b/src/IO/MMappedFileDescriptor.cpp index a7eb8e4ede5..47f80005c9d 100644 --- a/src/IO/MMappedFileDescriptor.cpp +++ b/src/IO/MMappedFileDescriptor.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include From 45db7c85cf25f9b4b27cedd7464a786c53580d3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 29 Jul 2024 16:57:15 +0000 Subject: [PATCH 0420/1170] Remove has_single_port property from plan stream. --- src/Processors/QueryPlan/AggregatingStep.cpp | 2 -- src/Processors/QueryPlan/FillingStep.cpp | 8 +++----- src/Processors/QueryPlan/IQueryPlanStep.h | 6 +----- src/Processors/QueryPlan/ITransformingStep.cpp | 3 --- src/Processors/QueryPlan/ReadNothingStep.cpp | 2 +- 5 files changed, 5 insertions(+), 16 deletions(-) diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f31de80b22d..8a5ed7fde65 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -134,7 +134,6 @@ AggregatingStep::AggregatingStep( { output_stream->sort_description = group_by_sort_description; output_stream->sort_scope = DataStream::SortScope::Global; - output_stream->has_single_port = true; } } @@ -147,7 +146,6 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_, { output_stream->sort_description = group_by_sort_description; output_stream->sort_scope = DataStream::SortScope::Global; - output_stream->has_single_port = true; } explicit_sorting_required_for_aggregation_in_order = false; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 65c9cf11661..81622389ada 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -39,12 +39,13 @@ FillingStep::FillingStep( , interpolate_description(interpolate_description_) , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { - if (!input_stream_.has_single_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); } void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + if (pipeline.getNumStreams() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) @@ -69,9 +70,6 @@ void FillingStep::describeActions(JSONBuilder::JSONMap & map) const void FillingStep::updateOutputStream() { - if (!input_streams.front().has_single_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); - output_stream = createOutputStream( input_streams.front(), FillingTransform::transformHeader(input_streams.front().header, sort_description), getDataStreamTraits()); } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index daca88fcceb..44eb7ea0c59 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -28,9 +28,6 @@ class DataStream public: Block header; - /// QueryPipeline has single port. Totals or extremes ports are not counted. - bool has_single_port = false; - /// Sorting scope. Please keep the mutual order (more strong mode should have greater value). enum class SortScope : uint8_t { @@ -51,8 +48,7 @@ public: bool hasEqualPropertiesWith(const DataStream & other) const { - return has_single_port == other.has_single_port - && sort_description == other.sort_description + return sort_description == other.sort_description && (sort_description.empty() || sort_scope == other.sort_scope); } diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index 9ecfdb0af22..3fa9d1b8308 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -20,9 +20,6 @@ DataStream ITransformingStep::createOutputStream( { DataStream output_stream{.header = std::move(output_header)}; - output_stream.has_single_port = stream_traits.returns_single_stream - || (input_stream.has_single_port && stream_traits.preserves_number_of_streams); - if (stream_traits.preserves_sorting) { output_stream.sort_description = input_stream.sort_description; diff --git a/src/Processors/QueryPlan/ReadNothingStep.cpp b/src/Processors/QueryPlan/ReadNothingStep.cpp index 253f3a5b980..3037172bbd4 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.cpp +++ b/src/Processors/QueryPlan/ReadNothingStep.cpp @@ -6,7 +6,7 @@ namespace DB { ReadNothingStep::ReadNothingStep(Block output_header) - : ISourceStep(DataStream{.header = std::move(output_header), .has_single_port = true}) + : ISourceStep(DataStream{.header = std::move(output_header)}) { } From 412268bf4e64b5c0df3980e5b8ccd2b078cf2177 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 17:05:46 +0000 Subject: [PATCH 0421/1170] Update reference --- ...dynamic_read_subcolumns_small.reference.j2 | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 index 3d814e1205a..be3f4e53990 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 @@ -25,26 +25,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -[[0]] \N \N [] str_10 \N str_10 [] -[[0,1]] \N \N [] +[[0]] \N \N [] str_11 \N str_11 [] -[[0,1,2]] \N \N [] +[[0,1]] \N \N [] str_12 \N str_12 [] -[[0,1,2,3]] \N \N [] +[[0,1,2]] \N \N [] str_13 \N str_13 [] -[[0,1,2,3,4]] \N \N [] +[[0,1,2,3]] \N \N [] str_14 \N str_14 [] -[[0,1,2,3,4,5]] \N \N [] +[[0,1,2,3,4]] \N \N [] str_15 \N str_15 [] -[[0,1,2,3,4,5,6]] \N \N [] +[[0,1,2,3,4,5]] \N \N [] str_16 \N str_16 [] -[[0,1,2,3,4,5,6,7]] \N \N [] +[[0,1,2,3,4,5,6]] \N \N [] str_17 \N str_17 [] -[[0,1,2,3,4,5,6,7,8]] \N \N [] +[[0,1,2,3,4,5,6,7]] \N \N [] str_18 \N str_18 [] -[[0,1,2,3,4,5,6,7,8,9]] \N \N [] +[[0,1,2,3,4,5,6,7,8]] \N \N [] str_19 \N str_19 [] +[[0,1,2,3,4,5,6,7,8,9]] \N \N [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] From 812a2b929938c293441f6a893adf96a00d469351 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 17:24:28 +0000 Subject: [PATCH 0422/1170] formatDateTime[InJodaSyntax]: make format string optional --- .../functions/type-conversion-functions.md | 12 ++++---- src/Functions/parseDateTime.cpp | 30 ++++++++++++------- .../02668_parse_datetime.reference | 6 +++- .../0_stateless/02668_parse_datetime.sql | 7 ++++- ...68_parse_datetime_in_joda_syntax.reference | 6 +++- .../02668_parse_datetime_in_joda_syntax.sql | 7 ++++- 6 files changed, 48 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 7cc2c022143..dc90697bd20 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -3467,13 +3467,13 @@ This function is the opposite operation of function [formatDateTime](../function **Syntax** ``` sql -parseDateTime(str, format[, timezone]) +parseDateTime(str[, format[, timezone]]) ``` **Arguments** -- `str` — the String to be parsed -- `format` — the format string +- `str` — The String to be parsed +- `format` — The format string. Optional. `%Y-%m-%d %H:%i:%s` if not specified. - `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** @@ -3516,13 +3516,13 @@ This function is the opposite operation of function [formatDateTimeInJodaSyntax] **Syntax** ``` sql -parseDateTimeInJodaSyntax(str, format[, timezone]) +parseDateTimeInJodaSyntax(str[, format[, timezone]]) ``` **Arguments** -- `str` — the String to be parsed -- `format` — the format string +- `str` — The String to be parsed +- `format` — The format string. Optional. `yyyy-MM-dd HH:mm:ss` if not specified. - `timezone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). Optional. **Returned value(s)** diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index bdca0151bba..7ca10677be7 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -582,11 +582,11 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args{ - {"time", static_cast(&isString), nullptr, "String"}, - {"format", static_cast(&isString), nullptr, "String"} + {"time", static_cast(&isString), nullptr, "String"} }; FunctionArgumentDescriptors optional_args{ + {"format", static_cast(&isString), nullptr, "String"}, {"timezone", static_cast(&isString), &isColumnConst, "const String"} }; @@ -2029,14 +2029,24 @@ namespace String getFormat(const ColumnsWithTypeAndName & arguments) const { - const auto * format_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!format_column) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second ('format') argument of function {}. Must be constant string.", - arguments[1].column->getName(), - getName()); - return format_column->getValue(); + if (arguments.size() == 1) + { + if constexpr (parse_syntax == ParseSyntax::MySQL) + return "%Y-%m-%d %H:%i:%s"; + else + return "yyyy-MM-dd HH:mm:ss"; + } + else + { + const auto * col_format = checkAndGetColumnConst(arguments[1].column.get()); + if (!col_format) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), + getName()); + return col_format->getValue(); + } } const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index d21a51ce70c..b67ca2d8b76 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -239,7 +239,7 @@ select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTi select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; 1 -- Error handling -select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Fuzzer crash bug #53715 select parseDateTime('', '', toString(number)) from numbers(13); -- { serverError ILLEGAL_COLUMN } @@ -270,3 +270,7 @@ select parseDateTime('8 13, 2022, 7:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); 2022-08-13 07:58:32 select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); 2022-08-13 07:58:32 +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); +1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 02ac0c5f35c..7b3aed60a4a 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -162,7 +162,7 @@ select sTr_To_DaTe('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTi select str_to_date('10:04:11 invalid 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') IS NULL; -- Error handling -select parseDateTime('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTime('12 AM', '%h %p', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Fuzzer crash bug #53715 @@ -187,4 +187,9 @@ select parseDateTime('08 13, 2022, 07:58:32', '%m %e, %G, %k:%i:%s', 'UTC'); select parseDateTime('8 13, 2022, 7:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC'); +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); + + -- { echoOff } diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference index 9fbf105dc41..6f560577ab5 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.reference @@ -354,5 +354,9 @@ select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; 1 -- Error handling -select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); +1 diff --git a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql index f5810d3d4c3..28d14607ba6 100644 --- a/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql +++ b/tests/queries/0_stateless/02668_parse_datetime_in_joda_syntax.sql @@ -239,6 +239,11 @@ select parseDateTimeInJodaSyntaxOrNull('2001 366 2000', 'yyyy D yyyy', 'UTC') = select parseDateTimeInJodaSyntaxOrNull('2001 invalid 366 2000', 'yyyy D yyyy', 'UTC') IS NULL; -- Error handling -select parseDateTimeInJodaSyntax('12 AM'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select parseDateTimeInJodaSyntax(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select parseDateTimeInJodaSyntax('12 AM', 'h a', 'UTC', 'a fourth argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +-- The format string argument is optional +set session_timezone = 'UTC'; -- don't randomize the session timezone +select parseDateTimeInJodaSyntax('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); + -- { echoOff } From 1c9d60ca972eab618b82704e14f1a680daed9c04 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 16:36:45 +0000 Subject: [PATCH 0423/1170] Refactoring --- programs/client/Client.h | 6 +- programs/local/LocalServer.h | 4 +- src/Client/ClientApplicationBase.cpp | 539 +++++++++++++++++++++++++++ src/Client/ClientApplicationBase.h | 54 +++ src/Client/ClientBase.cpp | 362 +----------------- src/Client/ClientBase.h | 73 +++- 6 files changed, 659 insertions(+), 379 deletions(-) create mode 100644 src/Client/ClientApplicationBase.cpp create mode 100644 src/Client/ClientApplicationBase.h diff --git a/programs/client/Client.h b/programs/client/Client.h index 9571440d6ba..7fdf77031ab 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -1,14 +1,16 @@ #pragma once -#include +#include namespace DB { -class Client : public ClientBase +class Client : public ClientApplicationBase { public: + using Arguments = ClientApplicationBase::Arguments; + Client() = default; void initialize(Poco::Util::Application & self) override; diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index ae9980311e1..b18a7a90961 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -21,7 +21,7 @@ namespace DB /// Lightweight Application for clickhouse-local /// No networking, no extra configs and working directories, no pid and status files, no dictionaries, no logging. /// Quiet mode by default -class LocalServer : public ClientBase, public Loggers +class LocalServer : public ClientApplicationBase, public Loggers { public: LocalServer() = default; diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp new file mode 100644 index 00000000000..59c98983694 --- /dev/null +++ b/src/Client/ClientApplicationBase.cpp @@ -0,0 +1,539 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include +#include "config.h" + +#include +#include +#include +#include +#include + +using namespace std::literals; + +namespace CurrentMetrics +{ + extern const Metric MemoryTracking; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_SET_SIGNAL_HANDLER; + extern const int UNRECOGNIZED_ARGUMENTS; +} + +static ClientInfo::QueryKind parseQueryKind(const String & query_kind) +{ + if (query_kind == "initial_query") + return ClientInfo::QueryKind::INITIAL_QUERY; + if (query_kind == "secondary_query") + return ClientInfo::QueryKind::SECONDARY_QUERY; + if (query_kind == "no_query") + return ClientInfo::QueryKind::NO_QUERY; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query kind {}", query_kind); +} + +/// This signal handler is set only for SIGINT and SIGQUIT. +void interruptSignalHandler(int signum) +{ + if (ClientApplicationBase::getInstance().tryStopQuery()) + safeExit(128 + signum); +} + +ClientApplicationBase::~ClientApplicationBase() = default; +ClientApplicationBase::ClientApplicationBase() : ClientBase(STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO, std::cin, std::cout, std::cerr) {} + +ClientApplicationBase & ClientApplicationBase::getInstance() +{ + return dynamic_cast(Poco::Util::Application::instance()); +} + +void ClientApplicationBase::setupSignalHandler() +{ + ClientApplicationBase::getInstance().stopQuery(); + + struct sigaction new_act; + memset(&new_act, 0, sizeof(new_act)); + + new_act.sa_handler = interruptSignalHandler; + new_act.sa_flags = 0; + +#if defined(OS_DARWIN) + sigemptyset(&new_act.sa_mask); +#else + if (sigemptyset(&new_act.sa_mask)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); +#endif + + if (sigaction(SIGINT, &new_act, nullptr)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); + + if (sigaction(SIGQUIT, &new_act, nullptr)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); +} + + +namespace +{ + +/// Define transparent hash to we can use +/// std::string_view with the containers +struct TransparentStringHash +{ + using is_transparent = void; + size_t operator()(std::string_view txt) const + { + return std::hash{}(txt); + } +}; + +/* + * This functor is used to parse command line arguments and replace dashes with underscores, + * allowing options to be specified using either dashes or underscores. + */ +class OptionsAliasParser +{ +public: + explicit OptionsAliasParser(const boost::program_options::options_description& options) + { + options_names.reserve(options.options().size()); + for (const auto& option : options.options()) + options_names.insert(option->long_name()); + } + + /* + * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options + * Implements boost::program_options::ext_parser logic + */ + std::pair operator()(const std::string & token) const + { + if (!token.starts_with("--")) + return {}; + std::string arg = token.substr(2); + + // divide token by '=' to separate key and value if options style=long_allow_adjacent + auto pos_eq = arg.find('='); + std::string key = arg.substr(0, pos_eq); + + if (options_names.contains(key)) + // option does not require any changes, because it is already correct + return {}; + + std::replace(key.begin(), key.end(), '-', '_'); + if (!options_names.contains(key)) + // after replacing '-' with '_' argument is still unknown + return {}; + + std::string value; + if (pos_eq != std::string::npos && pos_eq < arg.size()) + value = arg.substr(pos_eq + 1); + + return {key, value}; + } + +private: + std::unordered_set options_names; +}; + +} + +/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests +#if defined(__clang__) +#pragma clang optimize on +#endif +void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +{ + if (allow_repeated_settings) + addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); + else + addProgramOptions(cmd_settings, options_description.main_description.value()); + + if (allow_merge_tree_settings) + { + /// Add merge tree settings manually, because names of some settings + /// may clash. Query settings have higher priority and we just + /// skip ambiguous merge tree settings. + auto & main_options = options_description.main_description.value(); + + std::unordered_set> main_option_names; + for (const auto & option : main_options.options()) + main_option_names.insert(option->long_name()); + + for (const auto & setting : cmd_merge_tree_settings.all()) + { + const auto add_setting = [&](const std::string_view name) + { + if (auto it = main_option_names.find(name); it != main_option_names.end()) + return; + + if (allow_repeated_settings) + addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); + else + addProgramOption(cmd_merge_tree_settings, main_options, name, setting); + }; + + const auto & setting_name = setting.getName(); + + add_setting(setting_name); + + const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); + if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) + { + for (const auto alias : it->second) + { + add_setting(alias); + } + } + } + } + + /// Parse main commandline options. + auto parser = po::command_line_parser(arguments) + .options(options_description.main_description.value()) + .extra_parser(OptionsAliasParser(options_description.main_description.value())) + .allow_unregistered(); + po::parsed_options parsed = parser.run(); + + /// Check unrecognized options without positional options. + auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); + if (!unrecognized_options.empty()) + { + auto hints = this->getHints(unrecognized_options[0]); + if (!hints.empty()) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", + unrecognized_options[0], toString(hints)); + + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); + } + + /// Check positional options. + for (const auto & op : parsed.options) + { + if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") + && !op.original_tokens[0].empty() && !op.value.empty()) + { + /// Two special cases for better usability: + /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" + /// These are relevant for interactive usage - user-friendly, but questionable in general. + /// In case of ambiguity or for scripts, prefer using proper options. + + const auto & token = op.original_tokens[0]; + po::variable_value value(boost::any(op.value), false); + + const char * option; + if (token.contains(' ')) + option = "query"; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + + if (!options.emplace(option, value).second) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); + } + } + + po::store(parsed, options); +} + +void ClientApplicationBase::addMultiquery(std::string_view query, Arguments & common_arguments) const +{ + common_arguments.emplace_back("--multiquery"); + common_arguments.emplace_back("-q"); + common_arguments.emplace_back(query); +} + +Poco::Util::LayeredConfiguration & ClientApplicationBase::getClientConfiguration() +{ + return config(); +} + +void ClientApplicationBase::init(int argc, char ** argv) +{ + namespace po = boost::program_options; + + /// Don't parse options with Poco library, we prefer neat boost::program_options. + stopOptionsProcessing(); + + stdin_is_a_tty = isatty(STDIN_FILENO); + stdout_is_a_tty = isatty(STDOUT_FILENO); + stderr_is_a_tty = isatty(STDERR_FILENO); + terminal_width = getTerminalWidth(); + + std::vector external_tables_arguments; + Arguments common_arguments = {""}; /// 0th argument is ignored. + std::vector hosts_and_ports_arguments; + + if (argc) + argv0 = argv[0]; + readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments); + + /// Support for Unicode dashes + /// Interpret Unicode dashes as default double-hyphen + for (auto & arg : common_arguments) + { + // replace em-dash(U+2014) + boost::replace_all(arg, "—", "--"); + // replace en-dash(U+2013) + boost::replace_all(arg, "–", "--"); + // replace mathematical minus(U+2212) + boost::replace_all(arg, "−", "--"); + } + + + OptionsDescription options_description; + options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width)); + + /// Common options for clickhouse-client and clickhouse-local. + options_description.main_description->add_options() + ("help", "print usage summary, combine with --verbose to display all options") + ("verbose", "print query and other debugging info") + ("version,V", "print version information and exit") + ("version-clean", "print version in machine-readable format and exit") + + ("config-file,C", po::value(), "config-file path") + + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") + ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") + ("multiquery,n", "Obsolete, does nothing") + ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") + ("database,d", po::value(), "database") + ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") + ("query_id", po::value(), "query_id") + + ("history_file", po::value(), "path to history file") + + ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") + + ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") + ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") + ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") + + ("echo", "in batch mode, print query before execution") + + ("log-level", po::value(), "log level") + ("server_logs_file", po::value(), "put server logs into specified file") + + ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") + + ("format,f", po::value(), "default output format (and input format for clickhouse-local)") + ("output-format", po::value(), "default output format (this option has preference over --format)") + + ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") + ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") + + ("ignore-error", "do not stop processing when an error occurs") + ("stacktrace", "print stack traces of exceptions") + ("hardware-utilization", "print hardware utilization information in progress bar") + ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") + ("profile-events-delay-ms", po::value()->default_value(profile_events.delay_ms), "Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet)") + ("processed-rows", "print the number of locally processed rows") + + ("interactive", "Process queries-file or --query query and start interactive mode") + ("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)") + ; + + addOptions(options_description); + + OptionsDescription options_description_non_verbose = options_description; + + auto getter = [](const auto & op) + { + String op_long_name = op->long_name(); + return "--" + String(op_long_name); + }; + + if (options_description.main_description) + { + const auto & main_options = options_description.main_description->options(); + std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); + } + + if (options_description.external_description) + { + const auto & external_options = options_description.external_description->options(); + std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); + } + + po::variables_map options; + parseAndCheckOptions(options_description, options, common_arguments); + po::notify(options); + + if (options.count("version") || options.count("V")) + { + showClientVersion(); + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + if (options.count("version-clean")) + { + output_stream << VERSION_STRING; + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + if (options.count("verbose")) + getClientConfiguration().setBool("verbose", true); + + /// Output of help message. + if (options.count("help") + || (options.count("host") && options["host"].as() == "elp")) /// If user writes -help instead of --help. + { + if (getClientConfiguration().getBool("verbose", false)) + printHelpMessage(options_description, true); + else + printHelpMessage(options_description_non_verbose, false); + exit(0); // NOLINT(concurrency-mt-unsafe) + } + + /// Common options for clickhouse-client and clickhouse-local. + + /// Output execution time to stderr in batch mode. + if (options.count("time")) + getClientConfiguration().setBool("print-time-to-stderr", true); + if (options.count("memory-usage")) + { + const auto & memory_usage_mode = options["memory-usage"].as(); + if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); + getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); + } + + if (options.count("query")) + queries = options["query"].as>(); + if (options.count("query_id")) + getClientConfiguration().setString("query_id", options["query_id"].as()); + if (options.count("database")) + getClientConfiguration().setString("database", options["database"].as()); + if (options.count("config-file")) + getClientConfiguration().setString("config-file", options["config-file"].as()); + if (options.count("queries-file")) + queries_files = options["queries-file"].as>(); + if (options.count("multiline")) + getClientConfiguration().setBool("multiline", true); + if (options.count("ignore-error")) + getClientConfiguration().setBool("ignore-error", true); + if (options.count("format")) + getClientConfiguration().setString("format", options["format"].as()); + if (options.count("output-format")) + getClientConfiguration().setString("output-format", options["output-format"].as()); + if (options.count("vertical")) + getClientConfiguration().setBool("vertical", true); + if (options.count("stacktrace")) + getClientConfiguration().setBool("stacktrace", true); + if (options.count("print-profile-events")) + getClientConfiguration().setBool("print-profile-events", true); + if (options.count("profile-events-delay-ms")) + getClientConfiguration().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as()); + /// Whether to print the number of processed rows at + if (options.count("processed-rows")) + getClientConfiguration().setBool("print-num-processed-rows", true); + if (options.count("progress")) + { + switch (options["progress"].as()) + { + case DEFAULT: + getClientConfiguration().setString("progress", "default"); + break; + case OFF: + getClientConfiguration().setString("progress", "off"); + break; + case TTY: + getClientConfiguration().setString("progress", "tty"); + break; + case ERR: + getClientConfiguration().setString("progress", "err"); + break; + } + } + if (options.count("echo")) + getClientConfiguration().setBool("echo", true); + if (options.count("disable_suggestion")) + getClientConfiguration().setBool("disable_suggestion", true); + if (options.count("wait_for_suggestions_to_load")) + getClientConfiguration().setBool("wait_for_suggestions_to_load", true); + if (options.count("suggestion_limit")) + getClientConfiguration().setInt("suggestion_limit", options["suggestion_limit"].as()); + if (options.count("highlight")) + getClientConfiguration().setBool("highlight", options["highlight"].as()); + if (options.count("history_file")) + getClientConfiguration().setString("history_file", options["history_file"].as()); + if (options.count("interactive")) + getClientConfiguration().setBool("interactive", true); + if (options.count("pager")) + getClientConfiguration().setString("pager", options["pager"].as()); + + if (options.count("log-level")) + Poco::Logger::root().setLevel(options["log-level"].as()); + if (options.count("server_logs_file")) + server_logs_file = options["server_logs_file"].as(); + + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); + query_kind = parseQueryKind(options["query_kind"].as()); + profile_events.print = options.count("print-profile-events"); + profile_events.delay_ms = options["profile-events-delay-ms"].as(); + + processOptions(options_description, options, external_tables_arguments, hosts_and_ports_arguments); + { + std::unordered_set alias_names; + alias_names.reserve(options_description.main_description->options().size()); + for (const auto& option : options_description.main_description->options()) + alias_names.insert(option->long_name()); + argsToConfig(common_arguments, getClientConfiguration(), 100, &alias_names); + } + + clearPasswordFromCommandLine(argc, argv); + + /// Limit on total memory usage + std::string max_client_memory_usage = getClientConfiguration().getString("max_memory_usage_in_client", "0" /*default value*/); + if (max_client_memory_usage != "0") + { + UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); + + total_memory_tracker.setHardLimit(max_client_memory_usage_int); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + } + + /// Print stacktrace in case of crash + HandledSignals::instance().setupTerminateHandler(); + HandledSignals::instance().setupCommonDeadlySignalHandlers(); + /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't + /// have an option for client to shutdown gracefully. + + fatal_channel_ptr = new Poco::SplitterChannel; + fatal_console_channel_ptr = new Poco::ConsoleChannel; + fatal_channel_ptr->addChannel(fatal_console_channel_ptr); + if (options.count("client_logs_file")) + { + fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as()); + fatal_channel_ptr->addChannel(fatal_file_channel_ptr); + } + + fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); + signal_listener = std::make_unique(nullptr, fatal_log); + signal_listener_thread.start(*signal_listener); + +#if USE_GWP_ASAN + GWPAsan::initFinished(); +#endif + +} + + +} diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h new file mode 100644 index 00000000000..217fa29c3f4 --- /dev/null +++ b/src/Client/ClientApplicationBase.h @@ -0,0 +1,54 @@ +#pragma once + + +#include +#include +#include +#include +#include +#include + +#include + +namespace po = boost::program_options; + +namespace DB +{ + +void interruptSignalHandler(int signum); + +/** + * The base class for client appliucations such as + * clickhouse-client or clickhouse-local. + * The main purpose and responsibility of it is dealing with + * application-specific stuff such as command line arguments parsing + * and setting up signal handlers, so queries will be cancelled after + * Ctrl+C is pressed. + */ +class ClientApplicationBase : public ClientBase, public Poco::Util::Application, public IHints<2> +{ +public: + using ClientBase::processOptions; + using Arguments = ClientBase::Arguments; + + static ClientApplicationBase & getInstance(); + + ClientApplicationBase(); + ~ClientApplicationBase() override; + + void init(int argc, char ** argv); + std::vector getAllRegisteredNames() const override { return cmd_options; } + +protected: + Poco::Util::LayeredConfiguration & getClientConfiguration() override; + void setupSignalHandler() override; + void addMultiquery(std::string_view query, Arguments & common_arguments) const; + +private: + void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); + + std::vector cmd_options; +}; + + +} diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a88eed25db1..9cf3b955d26 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -5,7 +5,6 @@ #include #include -#include #include #include #include @@ -17,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -158,17 +156,6 @@ std::istream& operator>> (std::istream & in, ProgressOption & progress) return in; } -static ClientInfo::QueryKind parseQueryKind(const String & query_kind) -{ - if (query_kind == "initial_query") - return ClientInfo::QueryKind::INITIAL_QUERY; - if (query_kind == "secondary_query") - return ClientInfo::QueryKind::SECONDARY_QUERY; - if (query_kind == "no_query") - return ClientInfo::QueryKind::NO_QUERY; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query kind {}", query_kind); -} - static void incrementProfileEventsBlock(Block & dst, const Block & src) { if (!dst) @@ -269,36 +256,6 @@ static void incrementProfileEventsBlock(Block & dst, const Block & src) dst.setColumns(std::move(mutable_columns)); } - -std::atomic exit_after_signals = 0; - -class QueryInterruptHandler : private boost::noncopyable -{ -public: - /// Store how much interrupt signals can be before stopping the query - /// by default stop after the first interrupt signal. - static void start(Int32 signals_before_stop = 1) { exit_after_signals.store(signals_before_stop); } - - /// Set value not greater then 0 to mark the query as stopped. - static void stop() { exit_after_signals.store(0); } - - /// Return true if the query was stopped. - /// Query was stopped if it received at least "signals_before_stop" interrupt signals. - static bool try_stop() { return exit_after_signals.fetch_sub(1) <= 0; } - static bool cancelled() { return exit_after_signals.load() <= 0; } - - /// Return how much interrupt signals remain before stop. - static Int32 cancelled_status() { return exit_after_signals.load(); } -}; - -/// This signal handler is set for SIGINT and SIGQUIT. -void interruptSignalHandler(int signum) -{ - if (QueryInterruptHandler::try_stop()) - safeExit(128 + signum); -} - - /// To cancel the query on local format error. class LocalFormatError : public DB::Exception { @@ -345,31 +302,6 @@ ClientBase::ClientBase( terminal_width = getTerminalWidth(in_fd, err_fd); } -void ClientBase::setupSignalHandler() -{ - QueryInterruptHandler::stop(); - - struct sigaction new_act; - memset(&new_act, 0, sizeof(new_act)); - - new_act.sa_handler = interruptSignalHandler; - new_act.sa_flags = 0; - -#if defined(OS_DARWIN) - sigemptyset(&new_act.sa_mask); -#else - if (sigemptyset(&new_act.sa_mask)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); -#endif - - if (sigaction(SIGINT, &new_act, nullptr)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); - - if (sigaction(SIGQUIT, &new_act, nullptr)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); -} - - ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements) { std::unique_ptr parser; @@ -1113,8 +1045,8 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa { try { - QueryInterruptHandler::start(signals_before_stop); - SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + query_interrupt_handler.start(signals_before_stop); + SCOPE_EXIT({ query_interrupt_handler.stop(); }); connection->sendQuery( connection_parameters.timeouts, @@ -1178,13 +1110,13 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b /// to avoid losing sync. if (!cancelled) { - if (partial_result_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1) + if (partial_result_on_first_cancel && query_interrupt_handler.cancelled_status() == signals_before_stop - 1) { connection->sendCancel(); /// First cancel reading request was sent. Next requests will only be with a full cancel partial_result_on_first_cancel = false; } - else if (QueryInterruptHandler::cancelled()) + else if (query_interrupt_handler.cancelled()) { cancelQuery(); } @@ -1563,8 +1495,8 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars return; } - QueryInterruptHandler::start(); - SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + query_interrupt_handler.start(); + SCOPE_EXIT({ query_interrupt_handler.stop(); }); connection->sendQuery( connection_parameters.timeouts, @@ -1775,7 +1707,7 @@ try Block block; while (executor.pull(block)) { - if (!cancelled && QueryInterruptHandler::cancelled()) + if (!cancelled && query_interrupt_handler.cancelled()) { cancelQuery(); executor.cancel(); @@ -2857,7 +2789,6 @@ void ClientBase::runLibFuzzer() void ClientBase::runLibFuzzer() {} #endif - void ClientBase::clearTerminal() { /// Clear from cursor until end of screen. @@ -2867,288 +2798,9 @@ void ClientBase::clearTerminal() output_stream << "\033[0J" "\033[?25h"; } - void ClientBase::showClientVersion() { output_stream << VERSION_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; } -void ClientBase::init(int argc, char ** argv) -{ - namespace po = boost::program_options; - - /// Don't parse options with Poco library, we prefer neat boost::program_options. - stopOptionsProcessing(); - - stdin_is_a_tty = isatty(STDIN_FILENO); - stdout_is_a_tty = isatty(STDOUT_FILENO); - stderr_is_a_tty = isatty(STDERR_FILENO); - terminal_width = getTerminalWidth(); - - std::vector external_tables_arguments; - Arguments common_arguments = {""}; /// 0th argument is ignored. - std::vector hosts_and_ports_arguments; - - if (argc) - argv0 = argv[0]; - readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments); - - /// Support for Unicode dashes - /// Interpret Unicode dashes as default double-hyphen - for (auto & arg : common_arguments) - { - // replace em-dash(U+2014) - boost::replace_all(arg, "—", "--"); - // replace en-dash(U+2013) - boost::replace_all(arg, "–", "--"); - // replace mathematical minus(U+2212) - boost::replace_all(arg, "−", "--"); - } - - - OptionsDescription options_description; - options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width)); - - /// Common options for clickhouse-client and clickhouse-local. - options_description.main_description->add_options() - ("help", "print usage summary, combine with --verbose to display all options") - ("verbose", "print query and other debugging info") - ("version,V", "print version information and exit") - ("version-clean", "print version in machine-readable format and exit") - - ("config-file,C", po::value(), "config-file path") - - ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") - ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") - ("multiquery,n", "Obsolete, does nothing") - ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") - ("database,d", po::value(), "database") - ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") - ("query_id", po::value(), "query_id") - - ("history_file", po::value(), "path to history file") - - ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") - - ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") - ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") - ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") - ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") - - ("echo", "in batch mode, print query before execution") - - ("log-level", po::value(), "log level") - ("server_logs_file", po::value(), "put server logs into specified file") - - ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") - - ("format,f", po::value(), "default output format (and input format for clickhouse-local)") - ("output-format", po::value(), "default output format (this option has preference over --format)") - - ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") - ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") - - ("ignore-error", "do not stop processing when an error occurs") - ("stacktrace", "print stack traces of exceptions") - ("hardware-utilization", "print hardware utilization information in progress bar") - ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") - ("profile-events-delay-ms", po::value()->default_value(profile_events.delay_ms), "Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet)") - ("processed-rows", "print the number of locally processed rows") - - ("interactive", "Process queries-file or --query query and start interactive mode") - ("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)") - ; - - addOptions(options_description); - - OptionsDescription options_description_non_verbose = options_description; - - auto getter = [](const auto & op) - { - String op_long_name = op->long_name(); - return "--" + String(op_long_name); - }; - - if (options_description.main_description) - { - const auto & main_options = options_description.main_description->options(); - std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); - } - - if (options_description.external_description) - { - const auto & external_options = options_description.external_description->options(); - std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); - } - - po::variables_map options; - parseAndCheckOptions(options_description, options, common_arguments); - po::notify(options); - - if (options.count("version") || options.count("V")) - { - showClientVersion(); - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - if (options.count("version-clean")) - { - output_stream << VERSION_STRING; - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - if (options.count("verbose")) - getClientConfiguration().setBool("verbose", true); - - /// Output of help message. - if (options.count("help") - || (options.count("host") && options["host"].as() == "elp")) /// If user writes -help instead of --help. - { - if (getClientConfiguration().getBool("verbose", false)) - printHelpMessage(options_description, true); - else - printHelpMessage(options_description_non_verbose, false); - exit(0); // NOLINT(concurrency-mt-unsafe) - } - - /// Common options for clickhouse-client and clickhouse-local. - - /// Output execution time to stderr in batch mode. - if (options.count("time")) - getClientConfiguration().setBool("print-time-to-stderr", true); - if (options.count("memory-usage")) - { - const auto & memory_usage_mode = options["memory-usage"].as(); - if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); - getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); - } - - if (options.count("query")) - queries = options["query"].as>(); - if (options.count("query_id")) - getClientConfiguration().setString("query_id", options["query_id"].as()); - if (options.count("database")) - getClientConfiguration().setString("database", options["database"].as()); - if (options.count("config-file")) - getClientConfiguration().setString("config-file", options["config-file"].as()); - if (options.count("queries-file")) - queries_files = options["queries-file"].as>(); - if (options.count("multiline")) - getClientConfiguration().setBool("multiline", true); - if (options.count("ignore-error")) - getClientConfiguration().setBool("ignore-error", true); - if (options.count("format")) - getClientConfiguration().setString("format", options["format"].as()); - if (options.count("output-format")) - getClientConfiguration().setString("output-format", options["output-format"].as()); - if (options.count("vertical")) - getClientConfiguration().setBool("vertical", true); - if (options.count("stacktrace")) - getClientConfiguration().setBool("stacktrace", true); - if (options.count("print-profile-events")) - getClientConfiguration().setBool("print-profile-events", true); - if (options.count("profile-events-delay-ms")) - getClientConfiguration().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as()); - /// Whether to print the number of processed rows at - if (options.count("processed-rows")) - getClientConfiguration().setBool("print-num-processed-rows", true); - if (options.count("progress")) - { - switch (options["progress"].as()) - { - case DEFAULT: - getClientConfiguration().setString("progress", "default"); - break; - case OFF: - getClientConfiguration().setString("progress", "off"); - break; - case TTY: - getClientConfiguration().setString("progress", "tty"); - break; - case ERR: - getClientConfiguration().setString("progress", "err"); - break; - } - } - if (options.count("echo")) - getClientConfiguration().setBool("echo", true); - if (options.count("disable_suggestion")) - getClientConfiguration().setBool("disable_suggestion", true); - if (options.count("wait_for_suggestions_to_load")) - getClientConfiguration().setBool("wait_for_suggestions_to_load", true); - if (options.count("suggestion_limit")) - getClientConfiguration().setInt("suggestion_limit", options["suggestion_limit"].as()); - if (options.count("highlight")) - getClientConfiguration().setBool("highlight", options["highlight"].as()); - if (options.count("history_file")) - getClientConfiguration().setString("history_file", options["history_file"].as()); - if (options.count("interactive")) - getClientConfiguration().setBool("interactive", true); - if (options.count("pager")) - getClientConfiguration().setString("pager", options["pager"].as()); - - if (options.count("log-level")) - Poco::Logger::root().setLevel(options["log-level"].as()); - if (options.count("server_logs_file")) - server_logs_file = options["server_logs_file"].as(); - - query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); - query_kind = parseQueryKind(options["query_kind"].as()); - profile_events.print = options.count("print-profile-events"); - profile_events.delay_ms = options["profile-events-delay-ms"].as(); - - processOptions(options_description, options, external_tables_arguments, hosts_and_ports_arguments); - { - std::unordered_set alias_names; - alias_names.reserve(options_description.main_description->options().size()); - for (const auto& option : options_description.main_description->options()) - alias_names.insert(option->long_name()); - argsToConfig(common_arguments, getClientConfiguration(), 100, &alias_names); - } - - clearPasswordFromCommandLine(argc, argv); - - /// Limit on total memory usage - std::string max_client_memory_usage = getClientConfiguration().getString("max_memory_usage_in_client", "0" /*default value*/); - if (max_client_memory_usage != "0") - { - UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); - - total_memory_tracker.setHardLimit(max_client_memory_usage_int); - total_memory_tracker.setDescription("(total)"); - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - } - - /// Print stacktrace in case of crash - HandledSignals::instance().setupTerminateHandler(); - HandledSignals::instance().setupCommonDeadlySignalHandlers(); - /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't - /// have an option for client to shutdown gracefully. - - fatal_channel_ptr = new Poco::SplitterChannel; - fatal_console_channel_ptr = new Poco::ConsoleChannel; - fatal_channel_ptr->addChannel(fatal_console_channel_ptr); - if (options.count("client_logs_file")) - { - fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as()); - fatal_channel_ptr->addChannel(fatal_file_channel_ptr); - } - - fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); - signal_listener = std::make_unique(nullptr, fatal_log); - signal_listener_thread.start(*signal_listener); - -#if USE_GWP_ASAN - GWPAsan::initFinished(); -#endif - -} - } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 7689744a373..557ac30d27c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -1,26 +1,30 @@ #pragma once -#include -#include "Common/NamePrompter.h" -#include -#include + +#include +#include #include +#include #include #include -#include #include #include -#include #include #include #include #include -#include -#include -#include -#include -#include +#include +#include #include +#include +#include + +#include + +#include +#include +#include +#include namespace po = boost::program_options; @@ -64,9 +68,16 @@ std::istream& operator>> (std::istream & in, ProgressOption & progress); class InternalTextLogs; class WriteBufferFromFileDescriptor; -class ClientBase : public Poco::Util::Application, public IHints<2> +/** + * The base class which encapsulates the core functionality of a client. + * Can be used in a standalone application (clickhouse-client or clickhouse-local), + * or be embedded into server. + * Always keep in mind that there can be several instances of this class within + * a process. Thus, it cannot keep its state in global shared variables or even use them. + * The best example - std::cin, std::cout and std::cerr. + */ +class ClientBase { - public: using Arguments = std::vector; @@ -79,12 +90,11 @@ public: std::ostream & output_stream_ = std::cout, std::ostream & error_stream_ = std::cerr ); + virtual ~ClientBase(); - ~ClientBase() override; + bool tryStopQuery() { return query_interrupt_handler.tryStop(); } + void stopQuery() { return query_interrupt_handler.stop(); } - void init(int argc, char ** argv); - - std::vector getAllRegisteredNames() const override { return cmd_options; } ASTPtr parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements); protected: @@ -114,7 +124,7 @@ protected: ASTPtr parsed_query, std::optional echo_query_ = {}, bool report_error = false); static void adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth, uint32_t max_parser_backtracks); - static void setupSignalHandler(); + virtual void setupSignalHandler() = 0; bool executeMultiQuery(const String & all_queries_text); MultiQueryProcessingStage analyzeMultiQueryText( @@ -188,7 +198,6 @@ private: String prompt() const; void resetOutput(); - void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); void updateSuggest(const ASTPtr & ast); @@ -196,6 +205,31 @@ private: bool addMergeTreeSettings(ASTCreateQuery & ast_create); protected: + + class QueryInterruptHandler : private boost::noncopyable + { + public: + /// Store how much interrupt signals can be before stopping the query + /// by default stop after the first interrupt signal. + void start(Int32 signals_before_stop = 1) { exit_after_signals.store(signals_before_stop); } + + /// Set value not greater then 0 to mark the query as stopped. + void stop() { exit_after_signals.store(0); } + + /// Return true if the query was stopped. + /// Query was stopped if it received at least "signals_before_stop" interrupt signals. + bool tryStop() { return exit_after_signals.fetch_sub(1) <= 0; } + bool cancelled() { return exit_after_signals.load() <= 0; } + + /// Return how much interrupt signals remain before stop. + Int32 cancelled_status() { return exit_after_signals.load(); } + + private: + std::atomic exit_after_signals = 0; + }; + + QueryInterruptHandler query_interrupt_handler; + static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); bool processMultiQueryFromFile(const String & file_name); @@ -239,7 +273,6 @@ protected: std::vector queries; /// Queries passed via '--query' std::vector queries_files; /// If not empty, queries will be read from these files std::vector interleave_queries_files; /// If not empty, run queries from these files before processing every file from 'queries_files'. - std::vector cmd_options; bool stdin_is_a_tty = false; /// stdin is a terminal. bool stdout_is_a_tty = false; /// stdout is a terminal. From 9186e647eb672283b8cb95d2fc152e0994f6df6f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 17:05:09 +0000 Subject: [PATCH 0424/1170] Fix style --- src/Client/ClientBase.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9cf3b955d26..56685f9d3f4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -88,12 +88,6 @@ namespace fs = std::filesystem; using namespace std::literals; - -namespace CurrentMetrics -{ - extern const Metric MemoryTracking; -} - namespace DB { From af2c9fcaaf4e2f38c9db105c246ee24b095b256f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 18:18:42 +0000 Subject: [PATCH 0425/1170] Skip file --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 380656cd1ca..3c959617d02 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -318,6 +318,7 @@ std_cerr_cout_excludes=( src/Interpreters/Context.cpp # IProcessor::dump() src/Processors/IProcessor.cpp + src/Client/ClientApplicationBase.cpp src/Client/ClientBase.cpp src/Client/LineReader.cpp src/Client/QueryFuzzer.cpp From 8ba85074e74f403b3d5106f6ef811019075cefb4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 19 Jul 2024 12:25:17 +0000 Subject: [PATCH 0426/1170] Fix build --- src/Client/ClientApplicationBase.cpp | 16 +++++++++++++++- src/Client/ClientApplicationBase.h | 11 +++++++++++ src/Client/ClientBase.cpp | 14 +------------- src/Client/ClientBase.h | 10 ---------- 4 files changed, 27 insertions(+), 24 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 59c98983694..6b19898ef5c 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include "config.h" @@ -51,7 +52,20 @@ void interruptSignalHandler(int signum) safeExit(128 + signum); } -ClientApplicationBase::~ClientApplicationBase() = default; +ClientApplicationBase::~ClientApplicationBase() +{ + try + { + writeSignalIDtoSignalPipe(SignalListener::StopThread); + signal_listener_thread.join(); + HandledSignals::instance().reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + ClientApplicationBase::ClientApplicationBase() : ClientBase(STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO, std::cin, std::cout, std::cerr) {} ClientApplicationBase & ClientApplicationBase::getInstance() diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h index 217fa29c3f4..771bb948cb7 100644 --- a/src/Client/ClientApplicationBase.h +++ b/src/Client/ClientApplicationBase.h @@ -6,6 +6,10 @@ #include #include #include +#include +#include +#include + #include #include @@ -48,6 +52,13 @@ private: void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); std::vector cmd_options; + + LoggerPtr fatal_log; + Poco::AutoPtr fatal_channel_ptr; + Poco::AutoPtr fatal_console_channel_ptr; + Poco::AutoPtr fatal_file_channel_ptr; + Poco::Thread signal_listener_thread; + std::unique_ptr signal_listener; }; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 56685f9d3f4..85dfb767e75 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -258,19 +258,7 @@ public: }; -ClientBase::~ClientBase() -{ - try - { - writeSignalIDtoSignalPipe(SignalListener::StopThread); - signal_listener_thread.join(); - HandledSignals::instance().reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} +ClientBase::~ClientBase() = default; ClientBase::ClientBase( int in_fd_, diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 557ac30d27c..304d8c4b890 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -9,9 +9,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -253,13 +250,6 @@ protected: /// Client context is a context used only by the client to parse queries, process query parameters and to connect to clickhouse-server. ContextMutablePtr client_context; - LoggerPtr fatal_log; - Poco::AutoPtr fatal_channel_ptr; - Poco::AutoPtr fatal_console_channel_ptr; - Poco::AutoPtr fatal_file_channel_ptr; - Poco::Thread signal_listener_thread; - std::unique_ptr signal_listener; - bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool delayed_interactive = false; From 49dc30d5c28392d361ce0ef1e18f7db73841617f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 24 Jul 2024 21:42:36 +0000 Subject: [PATCH 0427/1170] Small adjustement --- src/Client/ClientApplicationBase.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 6b19898ef5c..0306468d084 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -48,8 +48,13 @@ static ClientInfo::QueryKind parseQueryKind(const String & query_kind) /// This signal handler is set only for SIGINT and SIGQUIT. void interruptSignalHandler(int signum) { - if (ClientApplicationBase::getInstance().tryStopQuery()) - safeExit(128 + signum); + /// Signal handler might be called even before the setup is fully finished + /// and client application started to process the query. + /// Because of that we have to manually check it. + if (auto * instance = ClientApplicationBase::instanceRawPtr(); instance) + if (auto * base = dynamic_cast(instance); base) + if (base->tryStopQuery()) + safeExit(128 + signum); } ClientApplicationBase::~ClientApplicationBase() From c7c1f10720cd194d85de6d81156cbd37304ab52b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 24 Jul 2024 21:45:16 +0000 Subject: [PATCH 0428/1170] Added new method --- base/poco/Util/include/Poco/Util/Application.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index c8d18e1bce9..d1a2021eb67 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -261,6 +261,11 @@ namespace Util /// /// Throws a NullPointerException if no Application instance exists. + static Application * instanceRawPtr(); + /// Returns a raw pointer to the Application sigleton. + /// + /// The caller should check whether the result is nullptr. + const Poco::Timestamp & startTime() const; /// Returns the application start time (UTC). @@ -448,6 +453,12 @@ namespace Util } + inline Application * Application::instanceRawPtr() + { + return _pInstance; + } + + inline const Poco::Timestamp & Application::startTime() const { return _startTime; From 6f068639db627944aaab978c79866ad5a2a234e7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Jul 2024 16:38:26 +0000 Subject: [PATCH 0429/1170] Better --- src/Client/ClientApplicationBase.h | 1 - src/Client/ClientBase.h | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientApplicationBase.h b/src/Client/ClientApplicationBase.h index 771bb948cb7..3663271dd25 100644 --- a/src/Client/ClientApplicationBase.h +++ b/src/Client/ClientApplicationBase.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 304d8c4b890..175ebe97075 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -2,6 +2,7 @@ #include +#include #include #include #include @@ -9,9 +10,13 @@ #include #include #include +#include +#include +#include #include #include #include + #include #include #include From a457db34b216ea987a4875a3cbeb5363878d5d5a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Jul 2024 16:58:41 +0000 Subject: [PATCH 0430/1170] Fixed a typo --- base/poco/Util/include/Poco/Util/Application.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index d1a2021eb67..786e331fe73 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -262,7 +262,7 @@ namespace Util /// Throws a NullPointerException if no Application instance exists. static Application * instanceRawPtr(); - /// Returns a raw pointer to the Application sigleton. + /// Returns a raw pointer to the Application singleton. /// /// The caller should check whether the result is nullptr. From 2f255dc68d2cffb8bc17efc153a9e75a9166675d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 01:11:40 +0200 Subject: [PATCH 0431/1170] Fix clang-tidy --- 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 175ebe97075..1a23b6b1363 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -95,7 +95,7 @@ public: virtual ~ClientBase(); bool tryStopQuery() { return query_interrupt_handler.tryStop(); } - void stopQuery() { return query_interrupt_handler.stop(); } + void stopQuery() { query_interrupt_handler.stop(); } ASTPtr parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements); From 9b4accebb3d19789e69b422ae2f235149e453f94 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 15:47:52 +0000 Subject: [PATCH 0432/1170] Fix build --- src/Client/ClientApplicationBase.cpp | 162 ------------------------ src/Client/ClientBaseOptimizedParts.cpp | 4 +- 2 files changed, 2 insertions(+), 164 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 0306468d084..4aa8b6c0bbe 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -102,168 +102,6 @@ void ClientApplicationBase::setupSignalHandler() throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); } - -namespace -{ - -/// Define transparent hash to we can use -/// std::string_view with the containers -struct TransparentStringHash -{ - using is_transparent = void; - size_t operator()(std::string_view txt) const - { - return std::hash{}(txt); - } -}; - -/* - * This functor is used to parse command line arguments and replace dashes with underscores, - * allowing options to be specified using either dashes or underscores. - */ -class OptionsAliasParser -{ -public: - explicit OptionsAliasParser(const boost::program_options::options_description& options) - { - options_names.reserve(options.options().size()); - for (const auto& option : options.options()) - options_names.insert(option->long_name()); - } - - /* - * Parses arguments by replacing dashes with underscores, and matches the resulting name with known options - * Implements boost::program_options::ext_parser logic - */ - std::pair operator()(const std::string & token) const - { - if (!token.starts_with("--")) - return {}; - std::string arg = token.substr(2); - - // divide token by '=' to separate key and value if options style=long_allow_adjacent - auto pos_eq = arg.find('='); - std::string key = arg.substr(0, pos_eq); - - if (options_names.contains(key)) - // option does not require any changes, because it is already correct - return {}; - - std::replace(key.begin(), key.end(), '-', '_'); - if (!options_names.contains(key)) - // after replacing '-' with '_' argument is still unknown - return {}; - - std::string value; - if (pos_eq != std::string::npos && pos_eq < arg.size()) - value = arg.substr(pos_eq + 1); - - return {key, value}; - } - -private: - std::unordered_set options_names; -}; - -} - -/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests -#if defined(__clang__) -#pragma clang optimize on -#endif -void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) -{ - if (allow_repeated_settings) - addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); - else - addProgramOptions(cmd_settings, options_description.main_description.value()); - - if (allow_merge_tree_settings) - { - /// Add merge tree settings manually, because names of some settings - /// may clash. Query settings have higher priority and we just - /// skip ambiguous merge tree settings. - auto & main_options = options_description.main_description.value(); - - std::unordered_set> main_option_names; - for (const auto & option : main_options.options()) - main_option_names.insert(option->long_name()); - - for (const auto & setting : cmd_merge_tree_settings.all()) - { - const auto add_setting = [&](const std::string_view name) - { - if (auto it = main_option_names.find(name); it != main_option_names.end()) - return; - - if (allow_repeated_settings) - addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting); - else - addProgramOption(cmd_merge_tree_settings, main_options, name, setting); - }; - - const auto & setting_name = setting.getName(); - - add_setting(setting_name); - - const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases(); - if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end()) - { - for (const auto alias : it->second) - { - add_setting(alias); - } - } - } - } - - /// Parse main commandline options. - auto parser = po::command_line_parser(arguments) - .options(options_description.main_description.value()) - .extra_parser(OptionsAliasParser(options_description.main_description.value())) - .allow_unregistered(); - po::parsed_options parsed = parser.run(); - - /// Check unrecognized options without positional options. - auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); - if (!unrecognized_options.empty()) - { - auto hints = this->getHints(unrecognized_options[0]); - if (!hints.empty()) - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", - unrecognized_options[0], toString(hints)); - - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); - } - - /// Check positional options. - for (const auto & op : parsed.options) - { - if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--") - && !op.original_tokens[0].empty() && !op.value.empty()) - { - /// Two special cases for better usability: - /// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1" - /// These are relevant for interactive usage - user-friendly, but questionable in general. - /// In case of ambiguity or for scripts, prefer using proper options. - - const auto & token = op.original_tokens[0]; - po::variable_value value(boost::any(op.value), false); - - const char * option; - if (token.contains(' ')) - option = "query"; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - - if (!options.emplace(option, value).second) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token); - } - } - - po::store(parsed, options); -} - void ClientApplicationBase::addMultiquery(std::string_view query, Arguments & common_arguments) const { common_arguments.emplace_back("--multiquery"); diff --git a/src/Client/ClientBaseOptimizedParts.cpp b/src/Client/ClientBaseOptimizedParts.cpp index 421843a0e79..297b8e7ce51 100644 --- a/src/Client/ClientBaseOptimizedParts.cpp +++ b/src/Client/ClientBaseOptimizedParts.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -80,7 +80,7 @@ private: } -void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +void ClientApplicationBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) { if (allow_repeated_settings) addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value()); From 6f16ca02a74a81956b7524958c5f94eb7a2c7bf5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 16:14:31 +0000 Subject: [PATCH 0433/1170] Fixed Style Check --- src/Client/ClientApplicationBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 4aa8b6c0bbe..9f133616d2e 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -31,7 +31,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int CANNOT_SET_SIGNAL_HANDLER; - extern const int UNRECOGNIZED_ARGUMENTS; } static ClientInfo::QueryKind parseQueryKind(const String & query_kind) From 75728ac56d83b85e476162a745686837cb194b73 Mon Sep 17 00:00:00 2001 From: Halersson Paris <142428374+halersson@users.noreply.github.com> Date: Mon, 29 Jul 2024 14:42:58 -0300 Subject: [PATCH 0434/1170] Fix typo --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index e837d4d5e20..bc5e8292192 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -420,7 +420,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() int num_row_groups = metadata->num_row_groups(); row_group_batches.reserve(num_row_groups); - auto adative_chunk_size = [&](int row_group_idx) -> size_t + auto adaptive_chunk_size = [&](int row_group_idx) -> size_t { size_t total_size = 0; auto row_group_meta = metadata->RowGroup(row_group_idx); @@ -457,7 +457,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().row_groups_idxs.push_back(row_group); row_group_batches.back().total_rows += metadata->RowGroup(row_group)->num_rows(); row_group_batches.back().total_bytes_compressed += metadata->RowGroup(row_group)->total_compressed_size(); - auto rows = adative_chunk_size(row_group); + auto rows = adaptive_chunk_size(row_group); row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; } } From 06b3185e57953c43531e4281f22bc03a8cd424e7 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 29 Jul 2024 20:05:29 +0200 Subject: [PATCH 0435/1170] fixes --- src/Interpreters/InterpreterCreateQuery.cpp | 14 +- .../test_restore_external_engines/test.py | 133 ++++++++++++++---- 2 files changed, 111 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a5eb3a83365..6e689c59c09 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1031,13 +1031,6 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*to_engine, getContext()->getSettingsRef().default_table_engine.value); } - /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to - /// Null table engine. - else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) - { - if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) - setNullTableEngine(*create.storage); - } return; } } @@ -1050,6 +1043,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); } + /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to + /// Null table engine. + else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) + { + if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) + setNullTableEngine(*create.storage); + } return; } diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py index eb88da6b61f..cf189f2a6ed 100644 --- a/tests/integration/test_restore_external_engines/test.py +++ b/tests/integration/test_restore_external_engines/test.py @@ -7,52 +7,75 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] -node1 = cluster.add_instance("replica1", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) -node2 = cluster.add_instance("replica2", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) -node3 = cluster.add_instance("replica3", with_zookeeper=True, with_mysql8=True, main_configs=configs, external_dirs=["/backups/"]) +node1 = cluster.add_instance( + "replica1", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node2 = cluster.add_instance( + "replica2", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node3 = cluster.add_instance( + "replica3", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) nodes = [node1, node2, node3] backup_id_counter = 0 + def new_backup_name(): global backup_id_counter backup_id_counter += 1 return f"Disk('backups', '{backup_id_counter}/')" + def cleanup_nodes(nodes, dbname): for node in nodes: node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") + def fill_nodes(nodes, dbname): cleanup_nodes(nodes, dbname) for node in nodes: - node.query(f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')") + node.query( + f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')" + ) + def drop_mysql_table(conn, tableName): with conn.cursor() as cursor: cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") + def get_mysql_conn(cluster): conn = pymysql.connect( - user="root", password="clickhouse", host=cluster.mysql8_ip, port=cluster.mysql8_port + user="root", + password="clickhouse", + host=cluster.mysql8_ip, + port=cluster.mysql8_port, ) return conn + def fill_tables(cluster, dbname): fill_nodes(nodes, dbname) conn = get_mysql_conn(cluster) with conn.cursor() as cursor: - cursor.execute( - "DROP DATABASE IF EXISTS clickhouse" - ) - cursor.execute( - "CREATE DATABASE clickhouse" - ) - cursor.execute( - "DROP TABLE IF EXISTS clickhouse.inference_table" - ) + cursor.execute("DROP DATABASE IF EXISTS clickhouse") + cursor.execute("CREATE DATABASE clickhouse") + cursor.execute("DROP TABLE IF EXISTS clickhouse.inference_table") cursor.execute( "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" ) @@ -66,17 +89,30 @@ def fill_tables(cluster, dbname): node1.query( f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" ) - node1.query(f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})") + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})" + ) node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" - assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") == expected - assert node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") == expected + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") + == expected + ) + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") + == expected + ) assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" - assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" - assert node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") == "100\n" + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" + ) + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") + == "100\n" + ) assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" @@ -92,6 +128,7 @@ def start_cluster(): finally: cluster.shutdown() + def test_restore_table(start_cluster): fill_tables(cluster, "replicated") backup_name = new_backup_name() @@ -107,12 +144,26 @@ def test_restore_table(start_cluster): assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true") + node3.query( + f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true" + ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine") == "1\t100\n" - assert node1.query("SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function") == "1\t100\n" - assert node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" + ) + == "1\t100\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function" + ) + == "1\t100\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + ) cleanup_nodes(nodes, "replicated") @@ -132,12 +183,36 @@ def test_restore_table_null(start_cluster): assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - node3.query(f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1") + node3.query( + f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1" + ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine") == "0\t0\n" - assert node1.query("SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function") == "0\t0\n" - assert node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" - assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'") == "Null\nNull\n" - assert node1.query("SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'") == "MergeTree\n" + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" + ) + == "0\t0\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function" + ) + == "0\t0\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'" + ) + == "Null\nNull\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'" + ) + == "MergeTree\n" + ) cleanup_nodes(nodes, "replicated2") From cb056cf3a5080cbff61f6efd070733ae2061d5b8 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Mon, 29 Jul 2024 12:05:44 -0700 Subject: [PATCH 0436/1170] Add camelCase aliases for percentRank() and denseRank() for percent_rank() and dense_rank() --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 2b255c5120e..a1b46c8e36c 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2721,20 +2721,24 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("dense_rank", {[](const std::string & name, + factory.registerFunction("denseRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("percent_rank", {[](const std::string & name, + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + + factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { From 9811a2e71b825a55c376edfb38303c817493cd9e Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 28 Jul 2024 17:32:32 -0700 Subject: [PATCH 0437/1170] Add test 03213_denseRank_percentRank_alias --- ...3213_denseRank_percentRank_alias.reference | 45 ++++++++++++++ .../03213_denseRank_percentRank_alias.sql | 59 +++++++++++++++++++ 2 files changed, 104 insertions(+) create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference new file mode 100644 index 00000000000..b49e179df68 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference @@ -0,0 +1,45 @@ +---- denseRank() ---- +0 0 0 1 1 1 1 +3 0 0 2 2 2 2 +1 0 1 3 3 3 3 +4 0 1 4 4 4 4 +2 0 2 5 5 5 5 +6 1 0 1 1 1 1 +9 1 0 2 2 2 2 +7 1 1 3 3 3 3 +5 1 2 4 4 4 4 +8 1 2 5 5 5 5 +12 2 0 1 1 1 1 +10 2 1 2 2 2 2 +13 2 1 3 3 3 3 +11 2 2 4 4 4 4 +14 2 2 5 5 5 5 +15 3 0 1 1 1 1 +18 3 0 2 2 2 2 +16 3 1 3 3 3 3 +19 3 1 4 4 4 4 +17 3 2 5 5 5 5 +21 4 0 1 1 1 1 +24 4 0 2 2 2 2 +22 4 1 3 3 3 3 +20 4 2 4 4 4 4 +23 4 2 5 5 5 5 +27 5 0 1 1 1 1 +25 5 1 2 2 2 2 +28 5 1 3 3 3 3 +26 5 2 4 4 4 4 +29 5 2 5 5 5 5 +30 6 0 1 1 1 1 +---- percentRank() ---- +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 diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql new file mode 100644 index 00000000000..ff841294eb1 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql @@ -0,0 +1,59 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/67042 +-- Reference generated using percent_rank() and dense_rank() + +-- From ClickHouse/tests/queries/0_stateless/01591_window_functions.sql (for deterministic query) +SELECT '---- denseRank() ----'; +select number, p, o, + count(*) over w, + rank() over w, + denseRank() over w, + row_number() over w +from (select number, intDiv(number, 5) p, mod(number, 3) o + from numbers(31) order by o, number) t +window w as (partition by p order by o, number) +order by p, o, number +settings max_block_size = 2; + +-- Modifed from ClickHouse/tests/queries/0_stateless/01592_window_functions.sql (for deterministic query) +SELECT '---- percentRank() ----'; + +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, + percentRank() 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; From fea03cf46ff29aa398b08d86ae77361fe85d7c40 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 29 Jul 2024 21:07:24 +0200 Subject: [PATCH 0438/1170] Build results fix --- tests/ci/ci.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index e30062c32ff..935fe472e50 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -766,7 +766,9 @@ def _upload_build_artifacts( int(job_report.duration), GITHUB_JOB_API_URL(), head_ref=pr_info.head_ref, - pr_number=pr_info.number, + # PRInfo fetches pr number for release branches as well - set pr_number to 0 for release + # so that build results are not mistakenly treated as feature branch builds + pr_number=pr_info.number if pr_info.is_pr else 0, ) report_url = ci_cache.upload_build_report(build_result) print(f"Report file has been uploaded to [{report_url}]") From 3df2d88cf13ad552058a6958630741d7cdab9d3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 21:09:11 +0200 Subject: [PATCH 0439/1170] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 07b37835dda..620b7c99bac 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,7 +45,6 @@ * Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). #### Performance Improvement -* Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). * Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). * Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). * Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). From 756bde1158c4b3e6e65d324436291d53b9e25fbb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:27:15 +0100 Subject: [PATCH 0440/1170] rm file --- tests/performance/views_max_insert_threads.xml | 11 ----------- 1 file changed, 11 deletions(-) delete mode 100644 tests/performance/views_max_insert_threads.xml diff --git a/tests/performance/views_max_insert_threads.xml b/tests/performance/views_max_insert_threads.xml deleted file mode 100644 index 473bcd02ab8..00000000000 --- a/tests/performance/views_max_insert_threads.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - - - - - - From 2308a362a0aca716e2e50d3eb5283bdfd575e023 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 29 Jul 2024 19:31:46 +0000 Subject: [PATCH 0441/1170] Disable 02932_refreshable_materialized_views --- .../02932_refreshable_materialized_views.sh | 304 +----------------- 1 file changed, 1 insertion(+), 303 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 9081035579d..6df3c391ddb 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -1,306 +1,4 @@ #!/usr/bin/env bash # Tags: atomic-database -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh -# scheduling is done in UTC. -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" - -$CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" - - -# Basic refreshing. -$CLICKHOUSE_CLIENT -nq " - create materialized view a - refresh after 2 second - engine Memory - empty - as select number as x from numbers(2) union all select rand64() as x" -$CLICKHOUSE_CLIENT -nq "select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes"; -$CLICKHOUSE_CLIENT -nq "show create a" -# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] -do - sleep 0.1 -done -start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" -# Check table contents. -$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" -# Wait for table contents to change. -res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" -while : -do - res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res2" == "$res1" ] || break - sleep 0.1 -done -# Wait for another change. -while : -do - res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res3" == "$res2" ] || break - sleep 0.1 -done -# Check that the two changes were at least 1 second apart, in particular that we're not refreshing -# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer -# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. -$CLICKHOUSE_CLIENT -nq " - select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); - select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" - -# Create a source table from which views will read. -$CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1" - -# Switch to fake clock, change refresh schedule, change query. -$CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table a modify refresh every 2 year; - alter table a modify query select x*2 as x from src; - select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; - show create a;" -# Advance time to trigger the refresh. -$CLICKHOUSE_CLIENT -nq " - select '<5: no refresh>', count() from a; - system test view a set fake time '2052-02-03 04:05:06';" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<6: refreshed>', * from a; - select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" - -# Create a dependent view, refresh it once. -$CLICKHOUSE_CLIENT -nq " - create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; - show create b; - system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2052-11-11 11:11:11' ] -do - sleep 0.1 -done -# Next refresh shouldn't start until the dependency refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<8: refreshed>', * from b; - select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; - system test view b set fake time '2054-01-24 23:22:21';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] -do - sleep 0.1 -done -# Update source table (by dropping and re-creating it - to test that tables are looked up by name -# rather than uuid), kick off refresh of the dependency. -$CLICKHOUSE_CLIENT -nq " - select '<10: waiting>', view, status, remaining_dependencies, next_refresh_time from refreshes; - drop table src; - create table src (x Int16) engine Memory as select 2; - system test view a set fake time '2054-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] -do - sleep 0.1 -done -# Both tables should've refreshed. -$CLICKHOUSE_CLIENT -nq " - select '<11: chain-refreshed a>', * from a; - select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception from refreshes;" - -# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to -# catch up to the same cycle. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2059-01-01 00:00:00'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2061-01-01 00:00:00'; - system test view a set fake time '2057-01-01 00:00:00';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] -do - sleep 0.1 -done - -$CLICKHOUSE_CLIENT -nq " - select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; - truncate src; - insert into src values (3); - system test view a set fake time '2060-02-02 02:02:02';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<15: chain-refreshed a>', * from a; - select '<16: chain-refreshed b>', * from b; - select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" - -# Get to WaitingForDependencies state and remove the depencency. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2062-03-03 03:03:03'" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table b modify refresh every 2 year" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; - show create b;" - -# Select from a table that doesn't exist, get an exception. -$CLICKHOUSE_CLIENT -nq " - drop table a; - drop table b; - create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; - drop table src;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'c' -- $LINENO" | xargs`" != 'Exception' ] -do - sleep 0.1 -done -# Check exception, create src, expect successful refresh. -$CLICKHOUSE_CLIENT -nq " - select '<19: exception>', exception ilike '%UNKNOWN_TABLE%' ? '1' : exception from refreshes where view = 'c'; - create table src (x Int64) engine Memory as select 1; - system refresh view c;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -# Rename table. -$CLICKHOUSE_CLIENT -nq " - select '<20: unexception>', * from c; - rename table c to d; - select '<21: rename>', * from d; - select '<22: rename>', view, last_refresh_result from refreshes;" - -# Do various things during a refresh. -# First make a nonempty view. -$CLICKHOUSE_CLIENT -nq " - drop table d; - truncate src; - insert into src values (1) - create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -# Stop refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<23: simple refresh>', * from e; - system stop view e;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Disabled' ] -do - sleep 0.1 -done -# Make refreshes slow, wait for a slow refresh to start. (We stopped refreshes first to make sure -# we wait for a slow refresh, not a previous fast one.) -$CLICKHOUSE_CLIENT -nq " - insert into src select * from numbers(1000) settings max_block_size=1; - system start view e;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] -do - sleep 0.1 -done -# Rename. -$CLICKHOUSE_CLIENT -nq " - rename table e to f; - select '<24: rename during refresh>', * from f; - select '<25: rename during refresh>', view, status from refreshes where view = 'f'; - alter table f modify refresh after 10 year;" - -# Cancel. -$CLICKHOUSE_CLIENT -nq " - system cancel view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Cancelled' ] -do - sleep 0.1 -done - -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" = 'Running' ] -do - sleep 0.1 -done - -# Check that another refresh doesn't immediately start after the cancelled one. -$CLICKHOUSE_CLIENT -nq " - select '<27: cancelled>', view, status from refreshes where view = 'f'; - system refresh view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Running' ] -do - sleep 0.1 -done -# Drop. -$CLICKHOUSE_CLIENT -nq " - drop table f; - select '<28: drop during refresh>', view, status from refreshes;" - -# Try OFFSET and RANDOMIZE FOR. -$CLICKHOUSE_CLIENT -nq " - create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42; - show create g; - system test view g set fake time '2050-02-03 15:30:13';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - with '2050-02-10 04:00:00'::DateTime as expected - select '<29: randomize>', abs(next_refresh_time::Int64 - expected::Int64) <= 3600*(24*4+1), next_refresh_time != expected from refreshes;" - -# Send data 'TO' an existing table. -$CLICKHOUSE_CLIENT -nq " - drop table g; - create table dest (x Int64) engine MergeTree order by x; - truncate src; - insert into src values (1); - create materialized view h refresh every 1 second to dest empty as select x*10 as x from src; - show create h;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<30: to existing table>', * from dest; - insert into src values (2);" -while [ "`$CLICKHOUSE_CLIENT -nq "select count() from dest -- $LINENO" | xargs`" != '2' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<31: to existing table>', * from dest; - drop table dest; - drop table src; - drop table h;" - -# EMPTY -$CLICKHOUSE_CLIENT -nq " - create materialized view i refresh after 1 year engine Memory empty as select number as x from numbers(2); - create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2)" -while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<32: empty>', view, status, last_refresh_result from refreshes order by view; - drop table i; - drop table j" - -$CLICKHOUSE_CLIENT -nq " - drop table refreshes;" +# TODO: Re-add this test in https://github.com/ClickHouse/ClickHouse/pull/58934 From f3d5859c8bd5d1fb43f0d636d6cff3062e4ca267 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 29 Jul 2024 19:35:37 +0000 Subject: [PATCH 0442/1170] Fix new test --- ...dynamic_read_subcolumns_small.reference.j2 | 192 +++++++++--------- ...03036_dynamic_read_subcolumns_small.sql.j2 | 16 +- 2 files changed, 104 insertions(+), 104 deletions(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 index be3f4e53990..d6add681f51 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.reference.j2 @@ -25,26 +25,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -115,6 +115,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -134,7 +135,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -295,26 +295,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -475,26 +475,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -655,7 +655,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -735,6 +734,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] Array(Array(Dynamic)) Array(Variant(String, UInt64)) None @@ -762,26 +762,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -852,6 +852,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -871,7 +872,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -1032,26 +1032,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -1212,26 +1212,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -1392,7 +1392,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -1472,6 +1471,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] Array(Array(Dynamic)) Array(Variant(String, UInt64)) None @@ -1499,26 +1499,26 @@ UInt64 7 7 \N [] 8 8 \N [] 9 9 \N [] -str_10 \N str_10 [] [[0]] \N \N [] -str_11 \N str_11 [] +str_10 \N str_10 [] [[0,1]] \N \N [] -str_12 \N str_12 [] +str_11 \N str_11 [] [[0,1,2]] \N \N [] -str_13 \N str_13 [] +str_12 \N str_12 [] [[0,1,2,3]] \N \N [] -str_14 \N str_14 [] +str_13 \N str_13 [] [[0,1,2,3,4]] \N \N [] -str_15 \N str_15 [] +str_14 \N str_14 [] [[0,1,2,3,4,5]] \N \N [] -str_16 \N str_16 [] +str_15 \N str_15 [] [[0,1,2,3,4,5,6]] \N \N [] -str_17 \N str_17 [] +str_16 \N str_16 [] [[0,1,2,3,4,5,6,7]] \N \N [] -str_18 \N str_18 [] +str_17 \N str_17 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] -str_19 \N str_19 [] +str_18 \N str_18 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] +str_19 \N str_19 [] [20] \N \N [20] ['str_21','str_21'] \N \N ['str_21','str_21'] [22,22,22] \N \N [22,22,22] @@ -1589,6 +1589,7 @@ str_79 \N str_79 [] 7 \N [] 8 \N [] 9 \N [] +\N \N [] \N str_10 [] \N \N [] \N str_11 [] @@ -1608,7 +1609,6 @@ str_79 \N str_79 [] \N str_18 [] \N \N [] \N str_19 [] -\N \N [] \N \N [20] \N \N ['str_21','str_21'] \N \N [22,22,22] @@ -1769,26 +1769,26 @@ str_79 \N str_79 [] 7 7 \N [] 0 [] 8 8 \N [] 0 [] 9 9 \N [] 0 [] -str_10 \N \N [] 0 [] [[0]] \N \N [] 0 [] -str_11 \N \N [] 0 [] +str_10 \N \N [] 0 [] [[0,1]] \N \N [] 0 [] -str_12 \N \N [] 0 [] +str_11 \N \N [] 0 [] [[0,1,2]] \N \N [] 0 [] -str_13 \N \N [] 0 [] +str_12 \N \N [] 0 [] [[0,1,2,3]] \N \N [] 0 [] -str_14 \N \N [] 0 [] +str_13 \N \N [] 0 [] [[0,1,2,3,4]] \N \N [] 0 [] -str_15 \N \N [] 0 [] +str_14 \N \N [] 0 [] [[0,1,2,3,4,5]] \N \N [] 0 [] -str_16 \N \N [] 0 [] +str_15 \N \N [] 0 [] [[0,1,2,3,4,5,6]] \N \N [] 0 [] -str_17 \N \N [] 0 [] +str_16 \N \N [] 0 [] [[0,1,2,3,4,5,6,7]] \N \N [] 0 [] -str_18 \N \N [] 0 [] +str_17 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8]] \N \N [] 0 [] -str_19 \N \N [] 0 [] +str_18 \N \N [] 0 [] [[0,1,2,3,4,5,6,7,8,9]] \N \N [] 0 [] +str_19 \N \N [] 0 [] [20] \N \N [20] 1 [20] ['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] [22,22,22] \N \N [22,22,22] 3 [22,22,22] @@ -1949,26 +1949,26 @@ str_79 \N \N [] 0 [] 7 0 [] [] 8 0 [] [] 9 0 [] [] -str_10 0 [] [] [[0]] 0 [] [] -str_11 0 [] [] +str_10 0 [] [] [[0,1]] 0 [] [] -str_12 0 [] [] +str_11 0 [] [] [[0,1,2]] 0 [] [] -str_13 0 [] [] +str_12 0 [] [] [[0,1,2,3]] 0 [] [] -str_14 0 [] [] +str_13 0 [] [] [[0,1,2,3,4]] 0 [] [] -str_15 0 [] [] +str_14 0 [] [] [[0,1,2,3,4,5]] 0 [] [] -str_16 0 [] [] +str_15 0 [] [] [[0,1,2,3,4,5,6]] 0 [] [] -str_17 0 [] [] +str_16 0 [] [] [[0,1,2,3,4,5,6,7]] 0 [] [] -str_18 0 [] [] +str_17 0 [] [] [[0,1,2,3,4,5,6,7,8]] 0 [] [] -str_19 0 [] [] +str_18 0 [] [] [[0,1,2,3,4,5,6,7,8,9]] 0 [] [] +str_19 0 [] [] [20] 0 [] [20] ['str_21','str_21'] 0 [] [NULL,NULL] [22,22,22] 0 [] [22,22,22] @@ -2129,7 +2129,6 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] -[] [] [] [1] [[0]] [[[]]] [] [] [] [2] [[0,1]] [[[],[]]] @@ -2209,3 +2208,4 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +[] [] [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 index 1ed836fbeee..3253d7a6c68 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_small.sql.j2 @@ -29,14 +29,14 @@ select count() from test where not empty(d.`Array(Array(Dynamic))`); select count() from test where d is NULL; select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); -select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id; -select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id; -select d.Int8, d.Date, d.`Array(String)` from test order by id; -select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id; -select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id; -select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id; +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.Int8, d.Date, d.`Array(String)` from test order by id, d; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id, d; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id, d; drop table test; From 36c57ca50bf54180dc1a68bfe097112f3dc13a6f Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 29 Jul 2024 19:50:28 +0000 Subject: [PATCH 0443/1170] only check in the create mode to prevent failure to start --- src/Storages/StorageFactory.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 71f70a807a8..a059d624cd8 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -202,11 +202,12 @@ StoragePtr StorageFactory::get( "projections", [](StorageFeatures features) { return features.supports_projections; }); - /// Now let's handle the merge tree family, projection is fully supported in (Replictaed)MergeTree, - /// but also allowed in non-throw mode with other mergetree family members. + /// Now let's handle the merge tree family. Note we only handle in the mode of CREATE due to backward compatibility. + /// Otherwise, it would fail to start in the case of existing projections with special mergetree. + /// Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other mergetree family members. chassert(query.storage->engine); - if (std::string_view engine_name(query.storage->engine->name); - engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") + if (std::string_view engine_name(query.storage->engine->name); mode == LoadingStrictnessLevel::CREATE + && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") { /// default throw mode in deduplicate_merge_projection_mode bool projection_allowed = false; From 6317979825794882905bc02b3a18dd82cfd8ec1c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:53:11 +0100 Subject: [PATCH 0444/1170] 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 2cae0cb5ecedc2fd041def829b35bdf4dbb50f2f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 29 Jul 2024 20:29:15 +0000 Subject: [PATCH 0445/1170] force_connected flag for connection establisher --- src/Client/ConnectionEstablisher.cpp | 4 ++-- src/Client/ConnectionEstablisher.h | 4 +++- src/QueryPipeline/RemoteQueryExecutor.cpp | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 8cebe7a6183..f96546846c7 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -33,12 +33,12 @@ ConnectionEstablisher::ConnectionEstablisher( { } -void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::string & fail_message) +void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::string & fail_message, bool force_connected) { try { ProfileEvents::increment(ProfileEvents::DistributedConnectionTries); - result.entry = pool->get(*timeouts, settings, /* force_connected = */ true); + result.entry = pool->get(*timeouts, settings, force_connected); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); UInt64 server_revision = 0; diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index a3a01e63246..304ec4d34b4 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -24,7 +24,9 @@ public: const QualifiedTableName * table_to_check = nullptr); /// Establish connection and save it in result, write possible exception message in fail_message. - void run(TryResult & result, std::string & fail_message); + /// The connection is returned from the pool, it can be stale. Use force_connected flag + /// to ensure that connection is working one + void run(TryResult & result, std::string & fail_message, bool force_connected = false); /// Set async callback that will be called when reading from socket blocks. void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b08f2002f64..09ea6a9fb3c 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -89,12 +89,12 @@ RemoteQueryExecutor::RemoteQueryExecutor( auto table_name = main_table.getQualifiedName(); ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name); - connection_establisher.run(result, fail_message); + connection_establisher.run(result, fail_message, /*force_connected=*/ true); } else { ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr); - connection_establisher.run(result, fail_message); + connection_establisher.run(result, fail_message, /*force_connected=*/ true); } std::vector connection_entries; From 09619e6006f122fb3a8352328a07f42bfd284d17 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 29 Jul 2024 20:57:21 +0000 Subject: [PATCH 0446/1170] consider the case of alter table add projection --- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++ ...ojection_merge_special_mergetree.reference | 1 + ...206_projection_merge_special_mergetree.sql | 26 +++++++++++++++++-- 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 78a551591a6..677c4a92cda 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3211,6 +3211,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context queryToString(mutation_commands.ast())); } + /// Block the case of alter table add projection for special merge trees. + if (std::any_of(commands.begin(), commands.end(), [](const AlterCommand & c) { return c.type == AlterCommand::ADD_PROJECTION; })) + { + if (auto storage_name = getName(); storage_name != "MergeTree" && storage_name != "ReplicatedMergeTree" + && settings_from_storage->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" + " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode."); + } + commands.apply(new_metadata, local_context); if (AlterCommands::hasFullTextIndex(new_metadata) && !settings.allow_experimental_full_text_index) diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference index e69de29bb2d..1a9cc2b7fbf 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.reference @@ -0,0 +1 @@ +p diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index 25517fbba30..e0a4f4f8cec 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -59,8 +59,7 @@ OPTIMIZE TABLE tp FINAL; -- expecting no projection SYSTEM FLUSH LOGS; SELECT - name, - part_name + name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'tp') AND (active = 1); @@ -81,4 +80,27 @@ ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +DROP TABLE tp; + +-- test alter add projection case +CREATE TABLE tp ( + type Int32, + eventcnt UInt64 +) engine = ReplacingMergeTree order by type; + +ALTER TABLE tp ADD PROJECTION p (SELECT sum(eventcnt), type GROUP BY type); -- { serverError NOT_IMPLEMENTED } + +ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'drop'; + +ALTER TABLE tp ADD PROJECTION p (SELECT sum(eventcnt), type GROUP BY type); + +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); + +SYSTEM FLUSH LOGS; +-- expecting projection p +SELECT + name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'tp') AND (active = 1); + DROP TABLE tp; \ No newline at end of file From 8dfe4a93f6c1afde8475984a899cd5604f415d78 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 29 Jul 2024 22:07:10 +0200 Subject: [PATCH 0447/1170] Rewrite function get_broken_projections_info() without using system.errors --- .../test_broken_projections/test.py | 86 +++++++++---------- 1 file changed, 42 insertions(+), 44 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 2cbbee5563f..9493937d936 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -148,23 +148,22 @@ def break_part(node, table, part): bash(node, f"rm '{part_path}/columns.txt'") -def get_broken_projections_info(node, table, active=True): +def get_broken_projections_info(node, table, part=None, projection=None, active=True): + parent_name_filter = f" AND parent_name = '{part}'" if part else "" + name_filter = f" AND name = '{projection}'" if projection else "" return node.query( f""" - SELECT parent_name, name, errors.name FROM - ( - SELECT parent_name, name, exception_code + SELECT parent_name, name, exception FROM system.projection_parts WHERE table='{table}' AND database=currentDatabase() AND is_broken = 1 AND active = {active} - ) AS parts_info - INNER JOIN system.errors AS errors - ON parts_info.exception_code = errors.code + {parent_name_filter} + {name_filter} ORDER BY parent_name, name """ - ).strip() + ) def get_projections_info(node, table): @@ -312,8 +311,8 @@ def test_broken_ignored(cluster): # Projection 'proj1' from part all_2_2_0 will now appear in broken parts info # because it was marked broken during "check table" query. - assert "all_2_2_0\tproj1\tFILE_DOESNT_EXIST" in get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj1" ) # Check table query will also show a list of parts which have broken projections. @@ -323,14 +322,14 @@ def test_broken_ignored(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0 will now appear in broken parts info. - assert "all_2_2_0\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) # Second select works, because projection is now marked as broken. @@ -340,7 +339,7 @@ def test_broken_ignored(cluster): break_projection(node, table_name, "proj2", "all_3_3_0", "data") # It will not yet appear in broken projections info. - assert "all_3_3_0" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, part="all_3_3_0") insert(node, table_name, 20, 5) insert(node, table_name, 25, 5) @@ -371,8 +370,8 @@ def test_broken_ignored(cluster): node, table_name ) - assert "all_3_3_0" in get_broken_projections_info(node, table_name, active=False) - assert "all_2_2_0" in get_broken_projections_info(node, table_name, active=True) + assert get_broken_projections_info(node, table_name, part="all_3_3_0", active=False) + assert get_broken_projections_info(node, table_name, part="all_2_2_0", active=True) # 0 because of all_2_2_0 check(node, table_name, 0) @@ -396,8 +395,8 @@ def test_materialize_broken_projection(cluster): break_projection(node, table_name, "proj1", "all_1_1_0", "metadata") reattach(node, table_name) - assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" ) assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( node, table_name @@ -406,8 +405,8 @@ def test_materialize_broken_projection(cluster): break_projection(node, table_name, "proj2", "all_1_1_0", "data") reattach(node, table_name) - assert "all_1_1_0\tproj2\tFILE_DOESNT_EXIST" in get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj2" ) assert "Part all_1_1_0 has a broken projection proj2" in check_table_full( node, table_name @@ -469,8 +468,8 @@ def test_broken_projections_in_backups_2(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "part") check(node, table_name, 0, "proj2", "ErrnoException") - assert "all_2_2_0\tproj2\tFILE_DOESNT_EXIST" == get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) assert "FILE_DOESNT_EXIST" in node.query_and_get_error( @@ -524,8 +523,8 @@ def test_broken_projections_in_backups_3(cluster): assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( node, table_name ) - assert "all_1_1_0\tproj1\tFILE_DOESNT_EXIST" == get_broken_projections_info( - node, table_name + assert "FILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" ) backup_name = f"b4-{get_random_string()}" @@ -545,8 +544,11 @@ def test_broken_projections_in_backups_3(cluster): ) check(node, table_name, 0) - assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( - node, table_name + assert ( + "Projection directory proj1.proj does not exist while loading projections" + in get_broken_projections_info( + node, table_name, part="all_1_1_0", projection="proj1" + ) ) @@ -569,7 +571,7 @@ def test_check_part_thread(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST", do_check_command=False) @@ -606,15 +608,15 @@ def test_broken_on_start(cluster): break_projection(node, table_name, "proj2", "all_2_2_0", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" # We will mark projection as broken. check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0 will now appear in broken parts info. - assert "all_2_2_0\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0", projection="proj2" ) # Second select works, because projection is now marked as broken. @@ -623,7 +625,7 @@ def test_broken_on_start(cluster): node.restart_clickhouse() # It will not yet appear in broken projections info. - assert "proj2" in get_broken_projections_info(node, table_name) + assert get_broken_projections_info(node, table_name, projection="proj2") # Select works check(node, table_name, 0) @@ -654,7 +656,7 @@ def test_mutation_with_broken_projection(cluster): node, table_name ) - assert "" == get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name) check(node, table_name, 1) @@ -662,21 +664,21 @@ def test_mutation_with_broken_projection(cluster): break_projection(node, table_name, "proj2", "all_2_2_0_4", "data") # It will not yet appear in broken projections info. - assert "proj2" not in get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name, projection="proj2") # Select now fails with error "File doesn't exist" # We will mark projection as broken. check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") # Projection 'proj2' from part all_2_2_0_4 will now appear in broken parts info. - assert "all_2_2_0_4\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( - node, table_name + assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name, part="all_2_2_0_4", projection="proj2" ) # Second select works, because projection is now marked as broken. check(node, table_name, 0) - assert "all_2_2_0_4" in get_broken_projections_info(node, table_name) + assert get_broken_projections_info(node, table_name, part="all_2_2_0_4") node.query( f"ALTER TABLE {table_name} DELETE WHERE _part == 'all_0_0_0_4' SETTINGS mutations_sync = 1" @@ -690,14 +692,10 @@ def test_mutation_with_broken_projection(cluster): # Still broken because it was hardlinked. broken = get_broken_projections_info(node, table_name) - assert ( - "all_2_2_0_5" in broken or "" == broken - ) # second could be because of a merge. + if broken: # can be not broken because of a merge. + assert get_broken_projections_info(node, table_name, part="all_2_2_0_5") - if "" == broken: - check(node, table_name, 1) - else: - check(node, table_name, 0) + check(node, table_name, not broken) node.query( f"ALTER TABLE {table_name} DELETE WHERE c == 13 SETTINGS mutations_sync = 1" @@ -710,6 +708,6 @@ def test_mutation_with_broken_projection(cluster): ) # Not broken anymore. - assert "" == get_broken_projections_info(node, table_name) + assert not get_broken_projections_info(node, table_name) check(node, table_name, 1) From 18327bdf9c7a08d0f88683d51f930ef83b23127f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 29 Jul 2024 22:17:02 +0000 Subject: [PATCH 0448/1170] 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 f94bebb0530b7a9fdd3db104ad4261a467fafad3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 29 Jul 2024 22:36:59 +0000 Subject: [PATCH 0449/1170] fix --- docs/en/sql-reference/window-functions/lagInFrame.md | 2 +- docs/en/sql-reference/window-functions/leadInFrame.md | 2 +- src/Processors/Transforms/WindowTransform.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index 049e095c10f..de6e9005baa 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -23,7 +23,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Parameters** - `x` — Column name. - `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). -- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - default value of column type when omitted). **Returned value** diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index fc1b92cc266..4a82c03f6e6 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -23,7 +23,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Parameters** - `x` — Column name. - `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). -- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - default value of column type when omitted). **Returned value** diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1eac08780e9..f76e2d64368 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2408,7 +2408,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction if (argument_types[0]->equals(*argument_types[2])) return; - const auto supertype = getLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); + const auto supertype = tryGetLeastSupertype(DataTypes{argument_types[0], argument_types[2]}); if (!supertype) { throw Exception(ErrorCodes::BAD_ARGUMENTS, From 2aafd711463d30cb7803a054f434268335817db8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 19:08:07 +0200 Subject: [PATCH 0450/1170] Moved settings to ServerSettings and made the table drop even faster (cherry picked from commit e1eb542dcc2b9fbc6a470a3cd9a183e79c86d7c7) --- programs/local/LocalServer.cpp | 5 +++ programs/server/Server.cpp | 5 +++ src/Core/ServerSettings.cpp | 2 +- src/Core/ServerSettings.h | 9 ++++ src/IO/SharedThreadPools.cpp | 10 +++++ src/IO/SharedThreadPools.h | 3 ++ src/Interpreters/DatabaseCatalog.cpp | 50 ++++++++++------------- src/Interpreters/DatabaseCatalog.h | 15 ------- src/Interpreters/InterpreterDropQuery.cpp | 19 +++++++-- 9 files changed, 70 insertions(+), 48 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 88d5a0253d1..250c5e3b6c8 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -184,6 +184,11 @@ void LocalServer::initialize(Poco::Util::Application & self) cleanup_threads, 0, // We don't need any threads one all the parts will be deleted cleanup_threads); + + getDatabaseCatalogDropTablesThreadPool().initialize( + server_settings.database_catalog_drop_table_concurrency, + 0, // We don't need any threads if there are no DROP queries. + server_settings.database_catalog_drop_table_concurrency); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 16888015f8b..dd56114de0f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1035,6 +1035,11 @@ try 0, // We don't need any threads once all the tables will be created max_database_replicated_create_table_thread_pool_size); + getDatabaseCatalogDropTablesThreadPool().initialize( + server_settings.database_catalog_drop_table_concurrency, + 0, // We don't need any threads if there are no DROP queries. + server_settings.database_catalog_drop_table_concurrency); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index fbf86d3e9ad..6c498014996 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1,4 +1,4 @@ -#include "ServerSettings.h" +#include #include namespace DB diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..f2f78f70e91 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -66,6 +66,15 @@ namespace DB M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \ \ + /* Database Catalog */ \ + M(UInt64, database_atomic_delay_before_drop_table_sec, 8 * 60, "The delay during which a dropped table can be restored using the UNDROP statement. If DROP TABLE ran with a SYNC modifier, the setting is ignored.", 0) \ + M(UInt64, database_catalog_unused_dir_hide_timeout_sec, 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and this directory was not modified for last database_catalog_unused_dir_hide_timeout_sec seconds, the task will 'hide' this directory by removing all access rights. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'immediately'.", 0) \ + M(UInt64, database_catalog_unused_dir_rm_timeout_sec, 30 * 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and it was previously 'hidden' (see database_catalog_unused_dir_hide_timeout_sec) and this directory was not modified for last database_catalog_unused_dir_rm_timeout_sec seconds, the task will remove this directory. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'never'.", 0) \ + M(UInt64, database_catalog_unused_dir_cleanup_period_sec, 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. Sets scheduling period of the task. Zero means 'never'.", 0) \ + M(UInt64, database_catalog_drop_error_cooldown_sec, 5, "In case if drop table failed, ClickHouse will wait for this timeout before retrying the operation.", 0) \ + M(UInt64, database_catalog_drop_table_concurrency, 16, "The size of the threadpool used for dropping tables.", 0) \ + \ + \ M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index 3606ddd984c..cda7bc01bbf 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -23,6 +23,9 @@ namespace CurrentMetrics extern const Metric MergeTreeUnexpectedPartsLoaderThreads; extern const Metric MergeTreeUnexpectedPartsLoaderThreadsActive; extern const Metric MergeTreeUnexpectedPartsLoaderThreadsScheduled; + extern const Metric DatabaseCatalogThreads; + extern const Metric DatabaseCatalogThreadsActive; + extern const Metric DatabaseCatalogThreadsScheduled; extern const Metric DatabaseReplicatedCreateTablesThreads; extern const Metric DatabaseReplicatedCreateTablesThreadsActive; extern const Metric DatabaseReplicatedCreateTablesThreadsScheduled; @@ -166,4 +169,11 @@ StaticThreadPool & getDatabaseReplicatedCreateTablesThreadPool() return instance; } +/// ThreadPool used for dropping tables. +StaticThreadPool & getDatabaseCatalogDropTablesThreadPool() +{ + static StaticThreadPool instance("DropTablesThreadPool", CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, CurrentMetrics::DatabaseCatalogThreadsScheduled); + return instance; +} + } diff --git a/src/IO/SharedThreadPools.h b/src/IO/SharedThreadPools.h index 50adc70c9a0..06ccebd20b2 100644 --- a/src/IO/SharedThreadPools.h +++ b/src/IO/SharedThreadPools.h @@ -69,4 +69,7 @@ StaticThreadPool & getUnexpectedPartsLoadingThreadPool(); /// ThreadPool used for creating tables in DatabaseReplicated. StaticThreadPool & getDatabaseReplicatedCreateTablesThreadPool(); +/// ThreadPool used for dropping tables. +StaticThreadPool & getDatabaseCatalogDropTablesThreadPool(); + } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bb2dd158710..f64f8a06f38 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include #include #include @@ -189,13 +191,6 @@ StoragePtr TemporaryTableHolder::getTable() const void DatabaseCatalog::initializeAndLoadTemporaryDatabase() { - drop_delay_sec = getContext()->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec); - unused_dir_hide_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_hide_timeout_sec", unused_dir_hide_timeout_sec); - unused_dir_rm_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_rm_timeout_sec", unused_dir_rm_timeout_sec); - unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); - drop_error_cooldown_sec = getContext()->getConfigRef().getInt64("database_catalog_drop_error_cooldown_sec", drop_error_cooldown_sec); - drop_table_concurrency = getContext()->getConfigRef().getInt64("database_catalog_drop_table_concurrency", drop_table_concurrency); - auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); } @@ -203,7 +198,7 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase() void DatabaseCatalog::createBackgroundTasks() { /// It has to be done before databases are loaded (to avoid a race condition on initialization) - if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && unused_dir_cleanup_period_sec) + if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec) { auto cleanup_task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalogCleanupStoreDirectoryTask", [this]() { this->cleanupStoreDirectoryTask(); }); @@ -224,7 +219,7 @@ void DatabaseCatalog::startupBackgroundTasks() { (*cleanup_task)->activate(); /// Do not start task immediately on server startup, it's not urgent. - (*cleanup_task)->scheduleAfter(unused_dir_hide_timeout_sec * 1000); + (*cleanup_task)->scheduleAfter(static_cast(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec) * 1000); } (*drop_task)->activate(); @@ -1038,15 +1033,12 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() LOG_INFO(log, "Found {} partially dropped tables. Will load them and retry removal.", dropped_metadata.size()); - ThreadPool pool(CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, CurrentMetrics::DatabaseCatalogThreadsScheduled); + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); for (const auto & elem : dropped_metadata) { - pool.scheduleOrThrowOnError([&]() - { - this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first); - }); + runner([this, &elem](){ this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first); }); } - pool.wait(); + runner.waitForAllToFinishAndRethrowFirstError(); } String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const @@ -1135,7 +1127,13 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } else { - tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec}); + tables_marked_dropped.push_back + ({ + table_id, + table, + dropped_metadata_path, + drop_time + static_cast(getContext()->getServerSettings().database_atomic_delay_before_drop_table_sec) + }); if (first_async_drop_in_queue == tables_marked_dropped.end()) --first_async_drop_in_queue; } @@ -1289,13 +1287,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); for (const auto & item : tables_to_drop) { @@ -1332,7 +1324,7 @@ void DatabaseCatalog::dropTablesParallel(std::vectordrop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; + table_iterator->drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + getContext()->getServerSettings().database_catalog_drop_error_cooldown_sec; if (first_async_drop_in_queue == tables_marked_dropped.end()) --first_async_drop_in_queue; @@ -1342,7 +1334,7 @@ void DatabaseCatalog::dropTablesParallel(std::vectorscheduleAfter(unused_dir_cleanup_period_sec * 1000); + (*cleanup_task)->scheduleAfter(static_cast(getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec) * 1000); } bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir) @@ -1742,7 +1734,7 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP time_t current_time = time(nullptr); if (st.st_mode & (S_IRWXU | S_IRWXG | S_IRWXO)) { - if (current_time <= max_modification_time + unused_dir_hide_timeout_sec) + if (current_time <= max_modification_time + static_cast(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec)) return false; LOG_INFO(log, "Removing access rights for unused directory {} from disk {} (will remove it when timeout exceed)", unused_dir, disk_name); @@ -1758,6 +1750,8 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP } else { + auto unused_dir_rm_timeout_sec = static_cast(getContext()->getServerSettings().database_catalog_unused_dir_rm_timeout_sec); + if (!unused_dir_rm_timeout_sec) return false; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 23e38a6445e..83a302f117d 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -354,23 +354,8 @@ private: mutable std::mutex tables_marked_dropped_mutex; std::unique_ptr drop_task; - static constexpr time_t default_drop_delay_sec = 8 * 60; - time_t drop_delay_sec = default_drop_delay_sec; std::condition_variable wait_table_finally_dropped; - std::unique_ptr cleanup_task; - static constexpr time_t default_unused_dir_hide_timeout_sec = 60 * 60; /// 1 hour - time_t unused_dir_hide_timeout_sec = default_unused_dir_hide_timeout_sec; - static constexpr time_t default_unused_dir_rm_timeout_sec = 30 * 24 * 60 * 60; /// 30 days - time_t unused_dir_rm_timeout_sec = default_unused_dir_rm_timeout_sec; - static constexpr time_t default_unused_dir_cleanup_period_sec = 24 * 60 * 60; /// 1 day - time_t unused_dir_cleanup_period_sec = default_unused_dir_cleanup_period_sec; - - static constexpr time_t default_drop_error_cooldown_sec = 5; - time_t drop_error_cooldown_sec = default_drop_error_cooldown_sec; - - static constexpr size_t default_drop_table_concurrency = 10; - size_t drop_table_concurrency = default_drop_table_concurrency; std::unique_ptr reload_disks_task; std::mutex reload_disks_mutex; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index bad3e5277db..d8056ddd1a3 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -424,18 +425,28 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, auto table_context = Context::createCopy(getContext()); table_context->setInternalQuery(true); /// Do not hold extra shared pointers to tables - std::vector> tables_to_drop; + std::vector> tables_to_drop; // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { auto table_ptr = iterator->table(); - table_ptr->flushAndPrepareForShutdown(); - tables_to_drop.push_back({iterator->name(), table_ptr->isDictionary()}); + tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); } + /// Prepare tables for shutdown in parallel. + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); + for (const auto & [name, _] : tables_to_drop) + { + auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); + runner([my_table_ptr = std::move(table_ptr)](){ + my_table_ptr->flushAndPrepareForShutdown(); + }); + } + runner.waitForAllToFinishAndRethrowFirstError(); + for (const auto & table : tables_to_drop) { - query_for_table.setTable(table.first); + query_for_table.setTable(table.first.getTableName()); query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; From 1427b16689601d7dd29d26de99b233c132905fde Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 26 Jul 2024 21:44:15 +0200 Subject: [PATCH 0451/1170] Fixed style --- src/Interpreters/DatabaseCatalog.cpp | 3 --- src/Interpreters/InterpreterDropQuery.cpp | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f64f8a06f38..30b151eb81d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -50,9 +50,6 @@ namespace CurrentMetrics { extern const Metric TablesToDropQueueSize; - extern const Metric DatabaseCatalogThreads; - extern const Metric DatabaseCatalogThreadsActive; - extern const Metric DatabaseCatalogThreadsScheduled; } namespace DB diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index d8056ddd1a3..ef560ec3405 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -438,7 +438,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, for (const auto & [name, _] : tables_to_drop) { auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); - runner([my_table_ptr = std::move(table_ptr)](){ + runner([my_table_ptr = std::move(table_ptr)]() + { my_table_ptr->flushAndPrepareForShutdown(); }); } From 1096a4ff33497c64eb786f6dbb603b18ccf804b1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 13:37:36 +0000 Subject: [PATCH 0452/1170] Fixed occasional LOGICAL_ERROR --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 30b151eb81d..eaf8cf1cc82 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1336,7 +1336,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector Date: Tue, 30 Jul 2024 00:39:16 +0200 Subject: [PATCH 0453/1170] Dont throw --- src/Interpreters/DatabaseCatalog.cpp | 30 ++++++++++------------------ 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index eaf8cf1cc82..98526e5c1cd 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1329,26 +1329,10 @@ void DatabaseCatalog::dropTablesParallel(std::vector Date: Mon, 29 Jul 2024 23:10:13 +0000 Subject: [PATCH 0454/1170] Fix Dwarf range list parsing in stack symbolizer --- src/Common/Dwarf.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 8439c01b22c..1f22e3b05aa 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -1559,8 +1559,7 @@ bool Dwarf::isAddrInRangeList(const CompilationUnit & cu, auto sp_start = addr_.substr(*cu.addr_base + index_start * sizeof(uint64_t)); auto start = read(sp_start); - auto sp_end = addr_.substr(*cu.addr_base + index_start * sizeof(uint64_t) + length); - auto end = read(sp_end); + auto end = start + length; if (start != end && address >= start && address < end) { return true; From 4a42ddc18e57576d119b1416f1da06b9ec292fce Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 27 Jul 2024 02:36:54 +0000 Subject: [PATCH 0455/1170] Make Dwarf::findAddress() fallback slow path less slow --- src/Common/Dwarf.cpp | 42 ++++++++++++++++++++++++++++++++++++++---- src/Common/Dwarf.h | 3 ++- 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 8439c01b22c..eda479607b1 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -1029,7 +1029,8 @@ bool Dwarf::findLocation( const LocationInfoMode mode, CompilationUnit & cu, LocationInfo & info, - std::vector & inline_frames) const + std::vector & inline_frames, + bool assume_in_cu_range) const { Die die = getDieAtOffset(cu, cu.first_die); // Partial compilation unit (DW_TAG_partial_unit) is not supported. @@ -1041,6 +1042,11 @@ bool Dwarf::findLocation( std::optional main_file_name; std::optional base_addr_cu; + std::optional low_pc; + std::optional high_pc; + std::optional is_high_pc_addr; + std::optional range_offset; + forEachAttribute(cu, die, [&](const Attribute & attr) { switch (attr.spec.name) // NOLINT(bugprone-switch-missing-default-case) @@ -1058,18 +1064,46 @@ bool Dwarf::findLocation( // File name of main file being compiled main_file_name = std::get(attr.attr_value); break; - case DW_AT_low_pc: case DW_AT_entry_pc: // 2.17.1: historically DW_AT_low_pc was used. DW_AT_entry_pc was // introduced in DWARF3. Support either to determine the base address of // the CU. base_addr_cu = std::get(attr.attr_value); break; + case DW_AT_ranges: + range_offset = std::get(attr.attr_value); + break; + case DW_AT_low_pc: + low_pc = std::get(attr.attr_value); + base_addr_cu = std::get(attr.attr_value); + break; + case DW_AT_high_pc: + // The value of the DW_AT_high_pc attribute can be + // an address (DW_FORM_addr*) or an offset (DW_FORM_data*). + is_high_pc_addr = attr.spec.form == DW_FORM_addr || // + attr.spec.form == DW_FORM_addrx || // + attr.spec.form == DW_FORM_addrx1 || // + attr.spec.form == DW_FORM_addrx2 || // + attr.spec.form == DW_FORM_addrx3 || // + attr.spec.form == DW_FORM_addrx4; + high_pc = std::get(attr.attr_value); + break; } // Iterate through all attributes until find all above. return true; }); + /// Check if the address falls inside this unit's address ranges. + if (!assume_in_cu_range && ((low_pc && high_pc) || range_offset)) { + bool pc_match = low_pc && high_pc && is_high_pc_addr && address >= *low_pc + && (address < (*is_high_pc_addr ? *high_pc : *low_pc + *high_pc)); + bool range_match = range_offset && isAddrInRangeList(cu, address, base_addr_cu, range_offset.value(), cu.addr_size); + if (!pc_match && !range_match) + { + return false; + } + } + if (main_file_name) { info.has_main_file = true; @@ -1442,7 +1476,7 @@ bool Dwarf::findAddress( { return false; } - findLocation(address, mode, unit, locationInfo, inline_frames); + findLocation(address, mode, unit, locationInfo, inline_frames, /*assume_in_cu_range*/ true); return locationInfo.has_file_and_line; } else if (mode == LocationInfoMode::FAST) @@ -1471,7 +1505,7 @@ bool Dwarf::findAddress( { continue; } - findLocation(address, mode, unit, locationInfo, inline_frames); + findLocation(address, mode, unit, locationInfo, inline_frames, /*assume_in_cu_range*/ false); } return locationInfo.has_file_and_line; diff --git a/src/Common/Dwarf.h b/src/Common/Dwarf.h index da18b3affa0..d754191bfa9 100644 --- a/src/Common/Dwarf.h +++ b/src/Common/Dwarf.h @@ -283,7 +283,8 @@ private: LocationInfoMode mode, CompilationUnit & cu, LocationInfo & info, - std::vector & inline_frames) const; + std::vector & inline_frames, + bool assume_in_cu_range) const; /** * Finds a subprogram debugging info entry that contains a given address among From b0629726a05701341e5d96207fe5e3743cd9345b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 03:23:19 +0200 Subject: [PATCH 0456/1170] Update src/Common/Dwarf.cpp --- src/Common/Dwarf.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index eda479607b1..f17219e9bf5 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -1094,7 +1094,8 @@ bool Dwarf::findLocation( }); /// Check if the address falls inside this unit's address ranges. - if (!assume_in_cu_range && ((low_pc && high_pc) || range_offset)) { + if (!assume_in_cu_range && ((low_pc && high_pc) || range_offset)) + { bool pc_match = low_pc && high_pc && is_high_pc_addr && address >= *low_pc && (address < (*is_high_pc_addr ? *high_pc : *low_pc + *high_pc)); bool range_match = range_offset && isAddrInRangeList(cu, address, base_addr_cu, range_offset.value(), cu.addr_size); From 8f920d064ccca8ed8d9341b10d54ebef7500484c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 04:05:43 +0200 Subject: [PATCH 0457/1170] Fix inconsistent formatting of CODEC and STATISTICS --- src/Parsers/ASTFunction.cpp | 4 +++- src/Parsers/ASTFunction.h | 2 ++ src/Parsers/ExpressionElementParsers.cpp | 2 ++ src/Parsers/FunctionSecretArgumentsFinderAST.h | 4 +++- src/Parsers/IAST.h | 1 + src/Storages/StatisticsDescription.cpp | 1 + 6 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 230d4c778e8..cd9e910d45a 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -285,6 +285,8 @@ static bool formatNamedArgWithHiddenValue(IAST * arg, const IAST::FormatSettings void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.expression_list_prepend_whitespace = false; + if (kind == Kind::CODEC || kind == Kind::STATISTICS || kind == Kind::BACKUP_NAME) + frame.allow_operators = false; FormatStateStacked nested_need_parens = frame; FormatStateStacked nested_dont_need_parens = frame; nested_need_parens.need_parens = true; @@ -308,7 +310,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format /// Should this function to be written as operator? bool written = false; - if (arguments && !parameters && nulls_action == NullsAction::EMPTY) + if (arguments && !parameters && frame.allow_operators && nulls_action == NullsAction::EMPTY) { /// Unary prefix operators. if (arguments->children.size() == 1) diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index be2b6beae54..1b4a5928d1c 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -58,6 +58,8 @@ public: TABLE_ENGINE, DATABASE_ENGINE, BACKUP_NAME, + CODEC, + STATISTICS, }; Kind kind = Kind::ORDINARY_FUNCTION; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 865d07faaa7..9927acdcf17 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -696,6 +696,7 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto function_node = std::make_shared(); function_node->name = "CODEC"; + function_node->kind = ASTFunction::Kind::CODEC; function_node->arguments = expr_list_args; function_node->children.push_back(function_node->arguments); @@ -723,6 +724,7 @@ bool ParserStatisticsType::parseImpl(Pos & pos, ASTPtr & node, Expected & expect auto function_node = std::make_shared(); function_node->name = "STATISTICS"; + function_node->kind = ASTFunction::Kind::STATISTICS; function_node->arguments = stat_type; function_node->children.push_back(function_node->arguments); node = function_node; diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 5b77485afb0..94da30922cc 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -33,7 +33,9 @@ public: { case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; case ASTFunction::Kind::WINDOW_FUNCTION: break; - case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::LAMBDA_FUNCTION: break; + case ASTFunction::Kind::CODEC: break; + case ASTFunction::Kind::STATISTICS: break; case ASTFunction::Kind::TABLE_ENGINE: findTableEngineSecretArguments(); break; case ASTFunction::Kind::DATABASE_ENGINE: findDatabaseEngineSecretArguments(); break; case ASTFunction::Kind::BACKUP_NAME: findBackupNameSecretArguments(); break; diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d70c1cd0b6c..e2cf7579667 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -256,6 +256,7 @@ public: bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) bool surround_each_list_element_with_parens = false; + bool allow_operators = true; /// Format some functions, such as "plus", "in", etc. as operators. size_t list_element_index = 0; const IAST * current_select = nullptr; }; diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 9c5fd3604b2..63c849e3806 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -193,6 +193,7 @@ ASTPtr ColumnStatisticsDescription::getAST() const { auto function_node = std::make_shared(); function_node->name = "STATISTICS"; + function_node->kind = ASTFunction::Kind::STATISTICS; function_node->arguments = std::make_shared(); for (const auto & [type, desc] : types_to_desc) { From c1e7b7be89f5c5e39318ab093b00fa6cd8114ff3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 04:07:46 +0200 Subject: [PATCH 0458/1170] Add a test --- ...nconsistent_formatting_of_codecs_statistics.reference | 2 ++ ...03214_inconsistent_formatting_of_codecs_statistics.sh | 9 +++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference create mode 100755 tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh diff --git a/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference new file mode 100644 index 00000000000..7213baa3e5b --- /dev/null +++ b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.reference @@ -0,0 +1,2 @@ +ALTER TABLE t MODIFY COLUMN `c` CODEC(in(1, 2)) +ALTER TABLE t MODIFY COLUMN `c` STATISTICS(plus(1, 2)) diff --git a/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh new file mode 100755 index 00000000000..c3f8d89b9a4 --- /dev/null +++ b/tests/queries/0_stateless/03214_inconsistent_formatting_of_codecs_statistics.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Ensure that these (possibly incorrect) queries can at least be parsed back after formatting. +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t MODIFY COLUMN c CODEC(in(1, 2))" | $CLICKHOUSE_FORMAT --oneline +$CLICKHOUSE_FORMAT --oneline --query "ALTER TABLE t MODIFY COLUMN c STATISTICS(plus(1, 2))" | $CLICKHOUSE_FORMAT --oneline From cb6b6329c8e763f61f70797a95dab8ef24fd47d1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 30 Jul 2024 10:45:36 +0800 Subject: [PATCH 0459/1170] add session timezone settings --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 7e931e16e48..7d1da0c1579 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -8,5 +8,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" $CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" $CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone SETTINGS session_timezone='Asia/Shanghai'" $CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From dd5819ab6ab2df3231737a4808d0445ad5345555 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:16:09 +0200 Subject: [PATCH 0460/1170] Changelog sanity --- CHANGELOG.md | 55 +++++++++++++++++++++++++--------------------------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 620b7c99bac..06f7bcdd84e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,7 @@ * Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). * Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Function `tuple` will now try to construct named tuples in query (controlled by `enable_named_columns_in_function_tuple`). Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). +* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). #### New Feature * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). @@ -32,13 +33,14 @@ * Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). * Introduce `logger.console_log_level` server config to control the log level to the console (if enabled). [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). * Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). -* Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). +* Add `--memory-usage` option to client in non-interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). * Make an interactive client for clickhouse-disks, add local disk from the local directory. [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). * When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). +* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). #### Experimental Feature * Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). -* Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* Support rocksdb as backend storage of clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). * Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). * Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). * Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). @@ -46,8 +48,8 @@ #### Performance Improvement * Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). -* Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). -* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). +* Sizes of hash tables created by join (`parallel_hash` algorithm) are collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). * Improve performance of loading `plain_rewritable` metadata. [#65634](https://github.com/ClickHouse/ClickHouse/pull/65634) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Attaching tables on read-only disks will use fewer resources by not loading outdated parts. [#65635](https://github.com/ClickHouse/ClickHouse/pull/65635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). @@ -59,11 +61,11 @@ * DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). #### Improvement +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). * The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). -* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). -* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). +* Allow matching column names in a case-insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). @@ -71,7 +73,6 @@ * Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). * Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). -* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). * `arrayMin`/`arrayMax` can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). * Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). * Do not create format settings for each row when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). @@ -80,36 +81,35 @@ * Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add new option to config `` which allow to specify how often clickhouse will reload config. [#65545](https://github.com/ClickHouse/ClickHouse/pull/65545) ([alesapin](https://github.com/alesapin)). * Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). -* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). * Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). * Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). * `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add a script to backup your files to ClickHouse. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* PostgreSQL source support cancel. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). -* Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* PostgreSQL source to support query cancellations. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). +* Make `allow_experimental_analyzer` be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). -* Allow to use `concat` function with empty arguments ``` sql :) select concat();. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). -* Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). -* Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). -* Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). -* Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). -* Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). -* Add settings to ignore ON CLUSTER clause in queries for named collection management with replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Allow to use `concat` function with empty arguments `:) select concat();`. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). +* Allow controlling named collections in `clickhouse-local`. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve Azure-related profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). +* Support ORC file read by writer's time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). +* Add settings to control connections to PostgreSQL. The setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. The setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). +* Reduce inaccuracy of `input_wait_elapsed_us`/`elapsed_us` in the `system.processors_profile_log`. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). +* Improve ProfileEvents for the filesystem cache. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). +* Add settings to ignore the `ON CLUSTER` clause in queries for named collection management with the replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Function `generateSnowflakeID` now allows to specify a machine ID as a parameter to prevent collisions in large clusters. [#66374](https://github.com/ClickHouse/ClickHouse/pull/66374) ([ZAWA_ll](https://github.com/Zawa-ll)). -* Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Disable suspending on `Ctrl+Z` in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add option for validating the primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). #### Bug Fix (user-visible misbehavior in an official stable release) -* Fix unexpected size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). * Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix unexpected sizes of `LowCardinality` columns in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). * Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). -* Fix the VALID UNTIL clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the `VALID UNTIL` clause in the user definition resetting after a restart. [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix the remaining time column in `SHOW MERGES`. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). -* Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). -* Fix bug with cancellation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fixed crash while using `MaterializedMySQL` (which is an unsupported, experimental feature) with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). +* Fix logical error when `PREWHERE` expression read no columns and table has no adaptive index granularity (very old table). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug with the cancellation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). * Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). * Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). * Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). @@ -178,9 +178,6 @@ * Fix `indexHint` function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). * Fix AST formatting of 'create table b empty as a'. [#64951](https://github.com/ClickHouse/ClickHouse/pull/64951) ([Michael Kolupaev](https://github.com/al13n321)). -#### Build/Testing/Packaging Improvement -* Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). - ### ClickHouse release 24.6, 2024-07-01 #### Backward Incompatible Change From 3a7ffb3284003d853974baf12cf442bdc1105143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:19:10 +0200 Subject: [PATCH 0461/1170] Changelog sanity --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 06f7bcdd84e..9d1a63cb3a9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -156,7 +156,7 @@ * Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). * Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix error reporting while copying to S3 or AzureBlobStorage. [#66295](https://github.com/ClickHouse/ClickHouse/pull/66295) ([Vitaly Baranov](https://github.com/vitlibar)). -* Prevent watchdog from keeping descriptors of unlinked(rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Prevent watchdog from keeping descriptors of unlinked (rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). * Fix the bug that logicalexpressionoptimizerpass lost logical type of constant. [#66344](https://github.com/ClickHouse/ClickHouse/pull/66344) ([pn](https://github.com/chloro-pn)). * Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too aggressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. [#66402](https://github.com/ClickHouse/ClickHouse/pull/66402) ([vdimir](https://github.com/vdimir)). From 368b9a058379c2e7902fd9ee7a21b664f0500df9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 05:29:36 +0200 Subject: [PATCH 0462/1170] Changelog sanity --- CHANGELOG.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9d1a63cb3a9..722ae4f8268 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -39,11 +39,11 @@ * Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). #### Experimental Feature -* Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). -* Support rocksdb as backend storage of clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* Change binary serialization of the `Variant` data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). +* Support on-disk backend storage for clickhouse-keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). * Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). -* Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Support null map subcolumn for `Variant` and `Dynamic` subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix reading `Dynamic` subcolumns from altered `Memory` table. Previously if `max_types` parameter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). * Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). #### Performance Improvement From 4df37538820f4874f54852e2418cf560c9da9ecc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 03:50:28 +0000 Subject: [PATCH 0463/1170] Fix bloom filter index breaking some queries --- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 115 ++++++++++-------- .../00908_bloom_filter_index.reference | 2 + .../0_stateless/00908_bloom_filter_index.sh | 4 + .../00945_bloom_filter_index.reference | 5 + .../0_stateless/00945_bloom_filter_index.sql | 9 ++ 5 files changed, 83 insertions(+), 52 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index c6a00751f25..dc314ce53d4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -371,67 +371,78 @@ bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTre bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) { - bool maybe_useful = false; + if (!node.isFunction()) + return false; - if (node.isFunction()) + const auto function = node.toFunctionNode(); + auto arguments_size = function.getArgumentsSize(); + auto function_name = function.getFunctionName(); + + if (parent == nullptr) { - const auto function = node.toFunctionNode(); - auto arguments_size = function.getArgumentsSize(); - auto function_name = function.getFunctionName(); - + /// Recurse a little bit for indexOf(). for (size_t i = 0; i < arguments_size; ++i) { auto argument = function.getArgumentAt(i); if (traverseFunction(argument, out, &node)) - maybe_useful = true; - } - - if (arguments_size != 2) - return false; - - auto lhs_argument = function.getArgumentAt(0); - auto rhs_argument = function.getArgumentAt(1); - - if (functionIsInOrGlobalInOperator(function_name)) - { - if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) - { - if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) - { - if (prepared_set->hasExplicitSetElements()) - { - const auto prepared_info = getPreparedSetInfo(prepared_set); - if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) - maybe_useful = true; - } - } - } - } - else if (function_name == "equals" || - function_name == "notEquals" || - function_name == "has" || - function_name == "mapContains" || - function_name == "indexOf" || - function_name == "hasAny" || - function_name == "hasAll") - { - Field const_value; - DataTypePtr const_type; - - if (rhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } - else if (lhs_argument.tryGetConstant(const_value, const_type)) - { - if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) - maybe_useful = true; - } + return true; } } - return maybe_useful; + if (arguments_size != 2) + return false; + + /// indexOf() should be inside comparison function, e.g. greater(indexOf(key, 42), 0). + /// Other conditions should be at top level, e.g. equals(key, 42), not equals(equals(key, 42), 1). + if ((function_name == "indexOf") != (parent != nullptr)) + return false; + + auto lhs_argument = function.getArgumentAt(0); + auto rhs_argument = function.getArgumentAt(1); + + if (functionIsInOrGlobalInOperator(function_name)) + { + if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) + { + if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) + { + if (prepared_set->hasExplicitSetElements()) + { + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) + return true; + } + } + } + return false; + } + + if (function_name == "equals" || + function_name == "notEquals" || + function_name == "has" || + function_name == "mapContains" || + function_name == "indexOf" || + function_name == "hasAny" || + function_name == "hasAll") + { + Field const_value; + DataTypePtr const_type; + + if (rhs_argument.tryGetConstant(const_value, const_type)) + { + if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) + return true; + } + else if (lhs_argument.tryGetConstant(const_value, const_type) && (function_name == "equals" || function_name == "notEquals")) + { + if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) + return true; + } + + return false; + } + + return false; } bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.reference b/tests/queries/0_stateless/00908_bloom_filter_index.reference index c0cbd6c0335..e2e13a9ed12 100644 --- a/tests/queries/0_stateless/00908_bloom_filter_index.reference +++ b/tests/queries/0_stateless/00908_bloom_filter_index.reference @@ -28,6 +28,8 @@ "rows_read": 3, 8 aбвгдеёж "rows_read": 2, +13 +1 1 column-oriented 2 column-oriented "rows_read": 4, diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.sh b/tests/queries/0_stateless/00908_bloom_filter_index.sh index 88fc7944236..25a6567b894 100755 --- a/tests/queries/0_stateless/00908_bloom_filter_index.sh +++ b/tests/queries/0_stateless/00908_bloom_filter_index.sh @@ -103,6 +103,10 @@ $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT * FROM bloom_filte $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT * FROM bloom_filter_idx WHERE (s, lower(s)) IN (('aбвгдеёж', 'aбвгдеёж'), ('abc', 'cba')) ORDER BY k" $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT * FROM bloom_filter_idx WHERE (s, lower(s)) IN (('aбвгдеёж', 'aбвгдеёж'), ('abc', 'cba')) ORDER BY k FORMAT JSON" | grep "rows_read" +# Weird conditions not supported by the index. +$CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT count() FROM bloom_filter_idx WHERE (s = 'asd') = (s = 'asd')" +$CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT count() FROM bloom_filter_idx WHERE has(['asd', 'some string'], s)" + # TOKEN BF $CLICKHOUSE_CLIENT -n --query=" diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.reference b/tests/queries/0_stateless/00945_bloom_filter_index.reference index c0c2254648e..e6751fe4762 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.reference +++ b/tests/queries/0_stateless/00945_bloom_filter_index.reference @@ -14,6 +14,11 @@ 0 2 2 +18 +100 +100 +3 +100 1 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 4c26988574a..2b7feacbd98 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -25,6 +25,15 @@ WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM single_column_bl WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 6; WITH ((1, (1, 1)), (2, (2, 2))) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN liter_prepared_set SETTINGS max_rows_to_read = 6; +-- Check that indexHint() works (but it doesn't work with COUNT()). +SELECT SUM(ignore(*) + 1) FROM single_column_bloom_filter WHERE indexHint(i32 in (3, 15, 50)); + +-- The index doesn't understand expressions like these, but it shouldn't break the query. +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32 = 200) = (i32 = 200); +SELECT SUM(ignore(*) + 1) FROM single_column_bloom_filter WHERE indexHint((i32 = 200) != (i32 = 200)); +SELECT COUNT() FROM single_column_bloom_filter WHERE indexOf([10, 20, 30], i32) != 0; +SELECT COUNT() FROM single_column_bloom_filter WHERE has([100, 200, 300], 200); + DROP TABLE IF EXISTS single_column_bloom_filter; From f90b88c978d87225a5cf5f66136714ff5535d69c Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 30 Jul 2024 05:57:07 +0000 Subject: [PATCH 0464/1170] Reduced comprexity of the test --- ...2832_alter_max_sessions_for_user.reference | 7 +-- .../02832_alter_max_sessions_for_user.sh | 62 ++++++++----------- 2 files changed, 30 insertions(+), 39 deletions(-) diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference index f80f8738ff8..c2e103d61cb 100644 --- a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference @@ -1,8 +1,7 @@ -test_alter_profile case: max_session_count 1 alter_sessions_count 1 -test_alter_profile case: max_session_count 2 alter_sessions_count 1 +test_alter_profile case: max_sessions_for_user 1 +USER_SESSION_LIMIT_EXCEEDED +test_alter_profile case: max_sessions_for_user 2 USER_SESSION_LIMIT_EXCEEDED -test_alter_profile case: max_session_count 1 alter_sessions_count 2 -test_alter_profile case: max_session_count 2 alter_sessions_count 2 READONLY READONLY READONLY diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh index 87fbffdb1e6..55f9e3e97a4 100755 --- a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SESSION_ID_PREFIX="02832_alter_max_sessions_session_$$" QUERY_ID_PREFIX="02832_alter_max_sessions_query_$$" PROFILE="02832_alter_max_sessions_profile_$$" USER="02832_alter_max_sessions_user_$$" @@ -17,48 +16,41 @@ ${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" ${CLICKHOUSE_CLIENT} -q $"CREATE SETTINGS PROFILE ${PROFILE}" ${CLICKHOUSE_CLIENT} -q $"CREATE USER '${USER}' SETTINGS PROFILE '${PROFILE}'" -function run_sessions_set() +function wait_for_query_to_start() { - local sessions_count="$1" - local session_check="$2" - for ((i = 1 ; i <= ${sessions_count} ; i++)); do - local session_id="${SESSION_ID_PREFIX}_${i}" - local query_id="${QUERY_ID_PREFIX}_${i}" - # Write only expected error text - # More than alter_sessions_count queries will not start. - ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&query_id=${query_id}&session_id=${session_id}&session_check=${session_check}&session_timeout=600&function_sleep_max_microseconds_per_block=120000000" --data-binary "SELECT sleep(120)" | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' & + while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.2; done +} + +function test_alter_max_sessions_for_user() +{ + local max_sessions_for_user="$1" + echo $"test_alter_profile case: max_sessions_for_user ${max_sessions_for_user}" + + # Step 0: Set max_sessions_for_user. + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_sessions_for_user}" + + # Step 1: Simulaneously run `max_sessions_for_user` queries. These queries should run without any problems. + for ((i = 1 ; i <= max_sessions_for_user ; i++)); do + local query_id="${QUERY_ID_PREFIX}_${i}_${max_sessions_for_user}" + ${CLICKHOUSE_CLIENT} --max_block_size 1 --query_id $query_id --user $USER --function_sleep_max_microseconds_per_block=120000000 -q "SELECT sleepEachRow(0.1) FROM numbers(1200)" &>/dev/null & + wait_for_query_to_start $query_id done - for ((i = 1 ; i <= ${sessions_count} ; i++)); do - local query_id="${QUERY_ID_PREFIX}_${i}" - $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id' SYNC" >/dev/null + # Step 2: Run another `max_sessions_for_user` + 1 query. That query should fail. + local query_id="${QUERY_ID_PREFIX}_should_fail" + ${CLICKHOUSE_CLIENT} --query_id $query_id --user $USER -q "SELECT 1" 2>&1 | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' + + # Step 3: Stop running queries launched at step 1. + for ((i = 1 ; i <= max_sessions_for_user ; i++)); do + local query_id="${QUERY_ID_PREFIX}_${i}_${max_sessions_for_user}" + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id' ASYNC" >/dev/null done wait } -function test_alter_profile() -{ - local max_session_count="$1" - local alter_sessions_count="$2" - echo $"test_alter_profile case: max_session_count ${max_session_count} alter_sessions_count ${alter_sessions_count}" - - ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}" - - # Create sessions with $max_session_count restriction - run_sessions_set $max_session_count 0 - - # Update restriction to $alter_sessions_count - ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}" - - # Simultaneous sessions should use max settings from profile ($alter_sessions_count) - run_sessions_set $max_session_count 1 -} - -test_alter_profile 1 1 -test_alter_profile 2 1 -test_alter_profile 1 2 -test_alter_profile 2 2 +test_alter_max_sessions_for_user 1 +test_alter_max_sessions_for_user 2 ${CLICKHOUSE_CLIENT} -q "SELECT 1 SETTINGS max_sessions_for_user = 1" 2>&1 | grep -m 1 -o 'READONLY' | head -1 ${CLICKHOUSE_CLIENT} -q $"SET max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 From 7a53a14940ae1be299305548f0d024de7f279fe3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 09:29:16 +0200 Subject: [PATCH 0465/1170] Update 03213_deep_json.sql --- tests/queries/0_stateless/03213_deep_json.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03213_deep_json.sql b/tests/queries/0_stateless/03213_deep_json.sql index 4f79d99eb26..2a9476381ff 100644 --- a/tests/queries/0_stateless/03213_deep_json.sql +++ b/tests/queries/0_stateless/03213_deep_json.sql @@ -1,5 +1,5 @@ -- The default limit works. -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } -- Even if we relax the limit, it is also safe. SET input_format_json_max_depth = 100000; -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } From 6ab67323862391e520ac571f609f52b582248da5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 07:36:38 +0000 Subject: [PATCH 0466/1170] Delete .reference --- ...2_refreshable_materialized_views.reference | 44 ------------------- 1 file changed, 44 deletions(-) delete mode 100644 tests/queries/0_stateless/02932_refreshable_materialized_views.reference diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference deleted file mode 100644 index 2eb41590af1..00000000000 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ /dev/null @@ -1,44 +0,0 @@ -<1: created view> a [] 1 -CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x -<2: refreshed> 3 1 1 -<3: time difference at least> 1000 -<4: next refresh in> 2 -<4.5: altered> Scheduled Finished 2052-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src -<5: no refresh> 3 -<6: refreshed> 2 -<7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<8: refreshed> 20 -<9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 -<9: refreshed> b Scheduled Finished 2054-01-01 00:00:00 -<10: waiting> a Scheduled [] 2054-01-01 00:00:00 -<10: waiting> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 -<11: chain-refreshed a> 4 -<12: chain-refreshed b> 40 -<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 -<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 -<14: waiting for next cycle> a Scheduled [] 2058-01-01 00:00:00 -<14: waiting for next cycle> b WaitingForDependencies ['default.a'] 2060-01-01 00:00:00 -<15: chain-refreshed a> 6 -<16: chain-refreshed b> 60 -<17: chain-refreshed> a Scheduled 2062-01-01 00:00:00 -<17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 -<18: removed dependency> b Scheduled [] 2062-03-03 03:03:03 2064-01-01 00:00:00 5 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<19: exception> 1 -<20: unexception> 1 -<21: rename> 1 -<22: rename> d Finished -<23: simple refresh> 1 -<24: rename during refresh> 1 -<25: rename during refresh> f Running -<27: cancelled> f Scheduled -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 -<29: randomize> 1 1 -CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src -<30: to existing table> 10 -<31: to existing table> 10 -<31: to existing table> 20 -<32: empty> i Scheduled Unknown -<32: empty> j Scheduled Finished From 861bdb51f8e098c4b06cb14988e00febbb5a0ac7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 09:42:38 +0200 Subject: [PATCH 0467/1170] Fix test --- .../test_replicated_table_attach/test.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 02ef1ad6353..499220def2c 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -27,7 +27,20 @@ def started_cluster(): cluster.shutdown() +def start_clean_clickhouse(): + # remove fault injection if present + if "fault_injection.xml" in node.exec_in_container( + ["bash", "-c", "ls /etc/clickhouse-server/config.d"] + ): + print("Removing fault injection") + node.exec_in_container( + ["bash", "-c", "rm /etc/clickhouse-server/config.d/fault_injection.xml"] + ) + node.restart_clickhouse() + + def test_startup_with_small_bg_pool(started_cluster): + start_clean_clickhouse() node.query( "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" ) @@ -45,6 +58,7 @@ def test_startup_with_small_bg_pool(started_cluster): def test_startup_with_small_bg_pool_partitioned(started_cluster): + start_clean_clickhouse() node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" ) From cd036886ec9ba030da7b6b6151bb81e5cc3f7636 Mon Sep 17 00:00:00 2001 From: heguangnan Date: Tue, 30 Jul 2024 15:53:48 +0800 Subject: [PATCH 0468/1170] add test --- ...unt_distinct_null_key_memory_leak.reference | 0 ...214_count_distinct_null_key_memory_leak.sql | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.reference create mode 100644 tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql diff --git a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.reference b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql new file mode 100644 index 00000000000..847d3742dc3 --- /dev/null +++ b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql @@ -0,0 +1,18 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS testnull; +CREATE TABLE testnull +( + `a` Nullable(String), + `b` Nullable(String), + `c` Nullable(String) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY c +SETTINGS index_granularity = 8192, allow_nullable_key=1; + +INSERT INTO testnull(b,c) SELECT toString(rand64()) AS b, toString(rand64()) AS c FROM numbers(1000000) +SELECT count(distinct b) FROM testnull GROUP BY a SETTINGS max_memory_usage = 54748364; -- {serverError MEMORY_LIMIT_EXCEEDED} + +DROP TABLE testnull; \ No newline at end of file From a70571762f7d73a7ecc94981e8086418ecfdeb3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 10:08:35 +0200 Subject: [PATCH 0469/1170] Enable text_log by default --- programs/server/config.xml | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 94825a55f67..844aff8f668 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1130,8 +1130,7 @@ 7500 - + system part_log
@@ -1143,9 +1142,9 @@ false
- system text_log
@@ -1154,9 +1153,8 @@ 8192 524288 false - + trace
- --> From c427c4e2bba852f6f8f9b9346a9a2d0a09f0e4be Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 30 Jul 2024 10:34:47 +0200 Subject: [PATCH 0470/1170] Typo --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 98526e5c1cd..a8e5fd7e6aa 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1355,7 +1355,7 @@ void DatabaseCatalog::dropTableDataTask() } catch (...) { - /// We don't re-throw expection, because we are in a background pool. + /// We don't re-throw exception, because we are in a background pool. tryLogCurrentException(log, "Cannot drop tables. Will retry later."); } } From 3e6a1b99e023eb3d592c72c17ae4913a9074b5af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:05:45 +0200 Subject: [PATCH 0471/1170] Fix file/URI parsing with archive syntax --- src/IO/Archives/ArchiveUtils.cpp | 50 +++++++++++++++++++ src/IO/Archives/ArchiveUtils.h | 14 ++++++ src/IO/Archives/createArchiveReader.cpp | 13 ++--- src/IO/Archives/createArchiveWriter.cpp | 9 ++-- src/IO/S3/URI.cpp | 39 +++++++-------- src/IO/S3/URI.h | 3 +- src/Storages/StorageFile.cpp | 8 ++- src/TableFunctions/TableFunctionFile.cpp | 9 ++-- .../03214_parsing_archive_name_file.reference | 12 +++++ .../03214_parsing_archive_name_file.sh | 21 ++++++++ .../03215_parsing_archive_name_s3.reference | 2 + .../03215_parsing_archive_name_s3.sql | 6 +++ .../data_minio/::03215_archive.csv | 1 + .../data_minio/test :: 03215_archive.csv | 1 + .../data_minio/test::03215_archive.csv | 1 + 15 files changed, 145 insertions(+), 44 deletions(-) create mode 100644 src/IO/Archives/ArchiveUtils.cpp create mode 100644 tests/queries/0_stateless/03214_parsing_archive_name_file.reference create mode 100755 tests/queries/0_stateless/03214_parsing_archive_name_file.sh create mode 100644 tests/queries/0_stateless/03215_parsing_archive_name_s3.reference create mode 100644 tests/queries/0_stateless/03215_parsing_archive_name_s3.sql create mode 100644 tests/queries/0_stateless/data_minio/::03215_archive.csv create mode 100644 tests/queries/0_stateless/data_minio/test :: 03215_archive.csv create mode 100644 tests/queries/0_stateless/data_minio/test::03215_archive.csv diff --git a/src/IO/Archives/ArchiveUtils.cpp b/src/IO/Archives/ArchiveUtils.cpp new file mode 100644 index 00000000000..50009087de3 --- /dev/null +++ b/src/IO/Archives/ArchiveUtils.cpp @@ -0,0 +1,50 @@ +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +using namespace std::literals; +constexpr std::array tar_extensions{".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; +constexpr std::array zip_extensions{".zip"sv, ".zipx"sv}; +constexpr std::array sevenz_extensiosns{".7z"sv}; + +bool hasSupportedExtension(std::string_view path, const auto & supported_extensions) +{ + for (auto supported_extension : supported_extensions) + { + if (path.ends_with(supported_extension)) + return true; + } + + return false; +} + +} + +bool hasSupportedTarExtension(std::string_view path) +{ + return hasSupportedExtension(path, tar_extensions); +} + +bool hasSupportedZipExtension(std::string_view path) +{ + return hasSupportedExtension(path, zip_extensions); +} + +bool hasSupported7zExtension(std::string_view path) +{ + return hasSupportedExtension(path, sevenz_extensiosns); +} + +bool hasSupportedArchiveExtension(std::string_view path) +{ + return hasSupportedTarExtension(path) || hasSupportedZipExtension(path) || hasSupported7zExtension(path); +} + +} diff --git a/src/IO/Archives/ArchiveUtils.h b/src/IO/Archives/ArchiveUtils.h index 1b66be005a2..cdb731d1d57 100644 --- a/src/IO/Archives/ArchiveUtils.h +++ b/src/IO/Archives/ArchiveUtils.h @@ -10,3 +10,17 @@ #include #include #endif + +#include + +namespace DB +{ + +bool hasSupportedTarExtension(std::string_view path); +bool hasSupportedZipExtension(std::string_view path); +bool hasSupported7zExtension(std::string_view path); + +bool hasSupportedArchiveExtension(std::string_view path); + + +} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 782602091ac..dfa098eede0 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -12,7 +13,6 @@ extern const int CANNOT_UNPACK_ARCHIVE; extern const int SUPPORT_IS_DISABLED; } - std::shared_ptr createArchiveReader(const String & path_to_archive) { return createArchiveReader(path_to_archive, {}, 0); @@ -24,11 +24,7 @@ std::shared_ptr createArchiveReader( [[maybe_unused]] const std::function()> & archive_read_function, [[maybe_unused]] size_t archive_size) { - using namespace std::literals; - static constexpr std::array tar_extensions{ - ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; - - if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) + if (hasSupportedZipExtension(path_to_archive)) { #if USE_MINIZIP return std::make_shared(path_to_archive, archive_read_function, archive_size); @@ -36,8 +32,7 @@ std::shared_ptr createArchiveReader( throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - else if (std::any_of( - tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) + else if (hasSupportedTarExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, archive_read_function); @@ -45,7 +40,7 @@ std::shared_ptr createArchiveReader( throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif } - else if (path_to_archive.ends_with(".7z")) + else if (hasSupported7zExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive); diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 9a169587088..53be0a85a10 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -24,10 +25,7 @@ std::shared_ptr createArchiveWriter(const String & path_to_archi std::shared_ptr createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique_ptr archive_write_buffer) { - using namespace std::literals; - static constexpr std::array tar_extensions{ - ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.bz2"sv, ".tar.lzma"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv}; - if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) + if (hasSupportedZipExtension(path_to_archive)) { #if USE_MINIZIP return std::make_shared(path_to_archive, std::move(archive_write_buffer)); @@ -35,8 +33,7 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - else if (std::any_of( - tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) + else if (hasSupportedTarExtension(path_to_archive)) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, std::move(archive_write_buffer)); diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 4bf7a3ddf86..b9c400d2b98 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -54,10 +55,7 @@ URI::URI(const std::string & uri_) static constexpr auto OSS = "OSS"; static constexpr auto EOS = "EOS"; - if (containsArchive(uri_)) - std::tie(uri_str, archive_pattern) = getPathToArchiveAndArchivePattern(uri_); - else - uri_str = uri_; + std::tie(uri_str, archive_pattern) = getURIAndArchivePattern(uri_); uri = Poco::URI(uri_str); std::unordered_map mapper; @@ -167,32 +165,29 @@ void URI::validateBucket(const String & bucket, const Poco::URI & uri) !uri.empty() ? " (" + uri.toString() + ")" : ""); } -bool URI::containsArchive(const std::string & source) +std::pair> URI::getURIAndArchivePattern(const std::string & source) { size_t pos = source.find("::"); - return (pos != std::string::npos); -} + if (pos == String::npos) + return {source, std::nullopt}; -std::pair URI::getPathToArchiveAndArchivePattern(const std::string & source) -{ - size_t pos = source.find("::"); - assert(pos != std::string::npos); + std::string_view path_to_archive_view = std::string_view{source}.substr(0, pos); + while (path_to_archive_view.ends_with(' ')) + path_to_archive_view.remove_suffix(1); - std::string path_to_archive = source.substr(0, pos); - while ((!path_to_archive.empty()) && path_to_archive.ends_with(' ')) - path_to_archive.pop_back(); + if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + return {source, std::nullopt}; - if (path_to_archive.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + auto archive_uri = path_to_archive_view; - std::string_view path_in_archive_view = std::string_view{source}.substr(pos + 2); - while (path_in_archive_view.front() == ' ') - path_in_archive_view.remove_prefix(1); + std::string_view archive_pattern_view = std::string_view{source}.substr(pos + 2); + while (archive_pattern_view.front() == ' ') + archive_pattern_view.remove_prefix(1); - if (path_in_archive_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); + if (archive_pattern_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Archive pattern is empty"); - return {path_to_archive, std::string{path_in_archive_view}}; + return std::pair{std::string{archive_uri}, std::string{archive_pattern_view}}; } } diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index 363f98c46f5..e4bb0d9eae1 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -42,8 +42,7 @@ struct URI static void validateBucket(const std::string & bucket, const Poco::URI & uri); private: - bool containsArchive(const std::string & source); - std::pair getPathToArchiveAndArchivePattern(const std::string & source); + std::pair> getURIAndArchivePattern(const std::string & source); }; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 98cd5c4dfa9..de56fcf66a0 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -2247,8 +2248,11 @@ void StorageFile::parseFileSource(String source, String & filename, String & pat while (path_to_archive_view.ends_with(' ')) path_to_archive_view.remove_suffix(1); - if (path_to_archive_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + { + filename = std::move(source); + return; + } path_to_archive = path_to_archive_view; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1b6d86f8fa5..12b88ae2b14 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -63,9 +63,12 @@ std::optional TableFunctionFile::tryGetFormatFromFirstArgument() return FormatFactory::instance().tryGetFormatFromFileName(filename); } -StoragePtr TableFunctionFile::getStorage(const String & source, - const String & format_, const ColumnsDescription & columns, - ContextPtr global_context, const std::string & table_name, +StoragePtr TableFunctionFile::getStorage( + const String & source, + const String & format_, + const ColumnsDescription & columns, + ContextPtr global_context, + const std::string & table_name, const std::string & compression_method_) const { // For `file` table function, we are going to use format settings from the diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference new file mode 100644 index 00000000000..243a7c8fd02 --- /dev/null +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference @@ -0,0 +1,12 @@ +::nonexistentfile.csv +1 +nonexistent::nonexistentfile.csv +1 +nonexistent :: nonexistentfile.csv +1 +nonexistent ::nonexistentfile.csv +1 +nonexistent.tar.gz :: nonexistentfile.csv +1 +nonexistent.zip:: nonexistentfile.csv +1 diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh new file mode 100755 index 00000000000..32bf3246c84 --- /dev/null +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function try_to_read_file() +{ + file_to_read=$1 + file_argument=$2 + + echo $file_argument + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument')" 2>&1 | rg -c "Cannot stat file.*$file_to_read" +} + +try_to_read_file "::nonexistentfile.csv" "::nonexistentfile.csv" +try_to_read_file "nonexistent::nonexistentfile.csv" "nonexistent::nonexistentfile.csv" +try_to_read_file "nonexistent :: nonexistentfile.csv" "nonexistent :: nonexistentfile.csv" +try_to_read_file "nonexistent ::nonexistentfile.csv" "nonexistent ::nonexistentfile.csv" +try_to_read_file "nonexistent.tar.gz" "nonexistent.tar.gz :: nonexistentfile.csv" +try_to_read_file "nonexistent.zip" "nonexistent.zip:: nonexistentfile.csv" diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference new file mode 100644 index 00000000000..9dd925a7480 --- /dev/null +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -0,0 +1,2 @@ +::03215_archive.csv test/::03215_archive.csv +test::03215_archive.csv test/test::03215_archive.csv diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql new file mode 100644 index 00000000000..9d01f53c838 --- /dev/null +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -0,0 +1,6 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); +SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } +SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); diff --git a/tests/queries/0_stateless/data_minio/::03215_archive.csv b/tests/queries/0_stateless/data_minio/::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/::03215_archive.csv @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/data_minio/test::03215_archive.csv b/tests/queries/0_stateless/data_minio/test::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test::03215_archive.csv @@ -0,0 +1 @@ +1 From 5381619b2ab465386f11e86242883419e48e5f6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 11:36:25 +0200 Subject: [PATCH 0472/1170] Remove bad feature. --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 722ae4f8268..a4c873ba3f9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -65,7 +65,6 @@ * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). * The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). -* Allow matching column names in a case-insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). From 4d4fc8fd6f0123613305423d861429f54222d23f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 11:57:37 +0200 Subject: [PATCH 0473/1170] Add setting to disable archive path syntax --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/IO/S3/URI.cpp | 8 ++++++-- src/IO/S3/URI.h | 2 +- src/Storages/ObjectStorage/S3/Configuration.cpp | 8 ++++---- src/Storages/StorageFile.cpp | 14 ++++++++++++-- src/Storages/StorageFile.h | 2 +- src/TableFunctions/TableFunctionFile.cpp | 5 +++-- .../03214_parsing_archive_name_file.reference | 4 ++++ .../0_stateless/03214_parsing_archive_name_file.sh | 10 ++++++++-- .../03215_parsing_archive_name_s3.reference | 1 + .../0_stateless/03215_parsing_archive_name_s3.sql | 1 + .../data_minio/test.zip::03215_archive.csv | 1 + 13 files changed, 44 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4fc2034b855..5114a8204cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -936,6 +936,7 @@ class IColumn; M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ M(Bool, parallel_replicas_prefer_local_join, 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.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ + M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..8483a267237 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -124,6 +124,7 @@ static std::initializer_list mapper; diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index e4bb0d9eae1..80e2da96cd4 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -36,7 +36,7 @@ struct URI bool is_virtual_hosted_style; URI() = default; - explicit URI(const std::string & uri_); + explicit URI(const std::string & uri_, bool allow_archive_path_syntax = false); void addRegionToURI(const std::string & region); static void validateBucket(const std::string & bucket, const Poco::URI & uri); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 094ca069e7a..7542f59dcc4 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -142,14 +142,14 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection, ContextPtr context) { - const auto settings = context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); if (!filename.empty()) - url = S3::URI(std::filesystem::path(collection.get("url")) / filename); + url = S3::URI(std::filesystem::path(collection.get("url")) / filename, settings.allow_archive_path_syntax); else - url = S3::URI(collection.get("url")); + url = S3::URI(collection.get("url"), settings.allow_archive_path_syntax); auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); @@ -330,7 +330,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ } /// This argument is always the first - url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + url = S3::URI(checkAndGetLiteralArgument(args[0], "url"), context->getSettingsRef().allow_archive_path_syntax); if (engine_args_to_idx.contains("format")) { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index de56fcf66a0..efb39f90053 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2208,7 +2208,11 @@ void registerStorageFile(StorageFactory & factory) else if (type == Field::Types::UInt64) source_fd = static_cast(literal->value.get()); else if (type == Field::Types::String) - StorageFile::parseFileSource(literal->value.get(), source_path, storage_args.path_to_archive); + StorageFile::parseFileSource( + literal->value.get(), + source_path, + storage_args.path_to_archive, + factory_args.getLocalContext()->getSettingsRef().allow_archive_path_syntax); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument must be path or file descriptor"); } @@ -2235,8 +2239,14 @@ SchemaCache & StorageFile::getSchemaCache(const ContextPtr & context) return schema_cache; } -void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive) +void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive, bool allow_archive_path_syntax) { + if (!allow_archive_path_syntax) + { + filename = std::move(source); + return; + } + size_t pos = source.find("::"); if (pos == String::npos) { diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 895a8a663b8..bb969c1877c 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -128,7 +128,7 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & context); - static void parseFileSource(String source, String & filename, String & path_to_archive); + static void parseFileSource(String source, String & filename, String & path_to_archive, bool allow_archive_path_syntax); static ArchiveInfo getArchiveInfo( const std::string & path_to_archive, diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 12b88ae2b14..af327cfe54e 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -26,7 +26,7 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr if (context->getApplicationType() != Context::ApplicationType::LOCAL) { ITableFunctionFileLike::parseFirstArguments(arg, context); - StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); + StorageFile::parseFileSource(std::move(filename), filename, path_to_archive, context->getSettingsRef().allow_archive_path_syntax); return; } @@ -42,7 +42,8 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr else if (filename == "stderr") fd = STDERR_FILENO; else - StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); + StorageFile::parseFileSource( + std::move(filename), filename, path_to_archive, context->getSettingsRef().allow_archive_path_syntax); } else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference index 243a7c8fd02..d793d26dfc3 100644 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.reference +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.reference @@ -10,3 +10,7 @@ nonexistent.tar.gz :: nonexistentfile.csv 1 nonexistent.zip:: nonexistentfile.csv 1 +nonexistent.tar.gz :: nonexistentfile.csv SETTINGS allow_archive_path_syntax=0 +1 +nonexistent.zip:: nonexistentfile.csv SETTINGS allow_archive_path_syntax=0 +1 diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh index 32bf3246c84..2f77627f6be 100755 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -8,14 +8,20 @@ function try_to_read_file() { file_to_read=$1 file_argument=$2 + settings=$3 - echo $file_argument - $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument')" 2>&1 | rg -c "Cannot stat file.*$file_to_read" + echo $file_argument $settings + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | rg -c "Cannot stat file.*$file_to_read" } +# if archive extension is not detected for part before '::', path is taken as is try_to_read_file "::nonexistentfile.csv" "::nonexistentfile.csv" try_to_read_file "nonexistent::nonexistentfile.csv" "nonexistent::nonexistentfile.csv" try_to_read_file "nonexistent :: nonexistentfile.csv" "nonexistent :: nonexistentfile.csv" try_to_read_file "nonexistent ::nonexistentfile.csv" "nonexistent ::nonexistentfile.csv" +# if archive extension is detected for part before '::', path is split into archive and filename try_to_read_file "nonexistent.tar.gz" "nonexistent.tar.gz :: nonexistentfile.csv" try_to_read_file "nonexistent.zip" "nonexistent.zip:: nonexistentfile.csv" +# disabling archive syntax will always parse path as is +try_to_read_file "nonexistent.tar.gz :: nonexistentfile.csv" "nonexistent.tar.gz :: nonexistentfile.csv" "SETTINGS allow_archive_path_syntax=0" +try_to_read_file "nonexistent.zip:: nonexistentfile.csv" "nonexistent.zip:: nonexistentfile.csv" "SETTINGS allow_archive_path_syntax=0" diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference index 9dd925a7480..b4804c82dc2 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -1,2 +1,3 @@ ::03215_archive.csv test/::03215_archive.csv test::03215_archive.csv test/test::03215_archive.csv +test.zip::03215_archive.csv test/test.zip::03215_archive.csv diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql index 9d01f53c838..3a7ed0b864c 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -4,3 +4,4 @@ SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); +SELECT _file, _path FROM s3(s3_conn, filename='test.zip::03215_archive.csv') ORDER BY (_file, _path) SETTINGS allow_archive_path_syntax=0; diff --git a/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv b/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv @@ -0,0 +1 @@ +1 From b1e80883f0324995d84250d5edf37fd8ab475987 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 11:39:22 +0200 Subject: [PATCH 0474/1170] `accept_invalid_certificate` in client config: additional testing #65238 --- .../configs/ssl_config_strict.xml | 17 +++++++++ .../test_accept_invalid_certificate/test.py | 35 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml diff --git a/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml b/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml new file mode 100644 index 00000000000..a4383a77ac4 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/configs/ssl_config_strict.xml @@ -0,0 +1,17 @@ + + + 9440 + + + + + /etc/clickhouse-server/config.d/self-cert.pem + /etc/clickhouse-server/config.d/self-key.pem + /etc/clickhouse-server/config.d/ca-cert.pem + strict + + + diff --git a/tests/integration/test_accept_invalid_certificate/test.py b/tests/integration/test_accept_invalid_certificate/test.py index 87229d75f90..f43e9e6140a 100644 --- a/tests/integration/test_accept_invalid_certificate/test.py +++ b/tests/integration/test_accept_invalid_certificate/test.py @@ -17,6 +17,19 @@ instance = cluster.add_instance( "certs/self-cert.pem", "certs/ca-cert.pem", ], + with_zookeeper=False, +) + + +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/ssl_config_strict.xml", + "certs/self-key.pem", + "certs/self-cert.pem", + "certs/ca-cert.pem", + ], + with_zookeeper=False, ) @@ -90,3 +103,25 @@ def test_connection_accept(): ) == "1\n" ) + + +def test_strict_reject(): + with pytest.raises(Exception) as err: + execute_query_native(node1, "SELECT 1", "") + assert "certificate verify failed" in str(err.value) + + +def test_strict_reject_with_config(): + with pytest.raises(Exception) as err: + execute_query_native(node1, "SELECT 1", config_accept) + assert "alert certificate required" in str(err.value) + + +def test_strict_connection_reject(): + with pytest.raises(Exception) as err: + execute_query_native( + node1, + "SELECT 1", + config_connection_accept.format(ip_address=f"{instance.ip_address}"), + ) + assert "certificate verify failed" in str(err.value) From bdf98cbcc0121ab94dd1db39fc5cf977a7ed42ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Jul 2024 10:06:01 +0000 Subject: [PATCH 0475/1170] Fix public backports --- tests/ci/cherry_pick.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 0b2aa9a2d35..b660ad2c040 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -505,7 +505,7 @@ class Backport: ReleaseBranch( ( br - if self._repo_name == "ClickHouse/Clickhouse" + if self._repo_name == "ClickHouse/ClickHouse" else f"release/{br}" ), pr, From 9a05a3ed9e7cccada42f49a0cd5c3896010f9edb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:34 +0200 Subject: [PATCH 0476/1170] Add missing documentation for `groupConcat` after series of reverts and merges https://github.com/ClickHouse/ClickHouse/pull/65384 --- .../reference/groupconcat.md | 90 +++++++++++++++++++ .../aspell-ignore/en/aspell-dict.txt | 2 + 2 files changed, 92 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/groupconcat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md new file mode 100644 index 00000000000..072252de8c9 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -0,0 +1,90 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/groupconcat +sidebar_position: 363 +sidebar_label: groupConcat +title: groupConcat +--- + +Calculates a concatenated string from a group of strings, optionally separated by a delimiter, and optionally limited by a maximum number of elements. + +**Syntax** + +``` sql +groupConcat(expression [, delimiter] [, limit]); +``` + +**Arguments** + +- `expression` — The expression or column name that outputs strings to be concatenated.. +- `delimiter` — A [string](../../../sql-reference/data-types/string.md) that will be used to separate concatenated values. This parameter is optional and defaults to an empty string if not specified. +- `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. + +:::note +If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. +::: + +**Returned value** + +- Returns a [string](../../../sql-reference/data-types/string.md) consisting of the concatenated values of the column or expression. If the group has no elements or only null elements, and the function does not specify a handling for only null values, the result is a nullable string with a null value. + +**Examples** + +Input table: + +``` text +┌─id─┬─name─┐ +│ 1 │ John│ +│ 2 │ Jane│ +│ 3 │ Bob│ +└────┴──────┘ +``` + +1. Basic usage without a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name) FROM Employees; +``` + +Result: + +``` text +JohnJaneBob +``` + +This concatenates all names into one continuous string without any separator. + + +2. Using comma as a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane, Bob +``` + +This output shows the names separated by a comma followed by a space. + + +3. Limiting the number of concatenated elements + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane +``` + +This query limits the output to the first two names, even though there are more names in the table. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b21ae0764c6..d82b70cfdb4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1733,6 +1733,7 @@ groupBitmap groupBitmapAnd groupBitmapOr groupBitmapXor +groupConcat groupUniqArray grouparray grouparrayinsertat @@ -1749,6 +1750,7 @@ groupbitmapor groupbitmapxor groupbitor groupbitxor +groupconcat groupuniqarray grpc grpcio From fb466287dac16801518547f34b42edbc16a57fae Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:13:47 +0200 Subject: [PATCH 0477/1170] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index be7cfa78492..c29b604d23d 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-distributed-cache +# Tags: long, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d7534c5b40c315cdb3d52101f626dbee7d565cdb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 Jul 2024 12:20:54 +0200 Subject: [PATCH 0478/1170] Update 03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql --- ...imize_rewrite_aggregate_function_with_if_return_type_bug.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql index b620a6434bf..565a481940a 100644 --- a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql +++ b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + -- For function count, rewrite countState to countStateIf changes the type from AggregateFunction(count, Nullable(UInt64)) to AggregateFunction(count, UInt64) -- We can cast AggregateFunction(count, UInt64) back to AggregateFunction(count, Nullable(UInt64)) with additional _CAST select hex(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; From 6ef628a7c80ebd6ec727365ee69d8141a4f11400 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 Jul 2024 10:33:22 +0000 Subject: [PATCH 0479/1170] Fixing build. --- src/Processors/QueryPlan/DistinctStep.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index a481454139d..b1c24fc01ce 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static ITransformingStep::Traits getTraits(bool pre_distinct) { const bool preserves_number_of_streams = pre_distinct; @@ -90,7 +95,8 @@ void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const Buil /// final distinct for sorted stream (sorting inside and among chunks) if (input_stream.sort_scope == DataStream::SortScope::Global) { - assert(input_stream.has_single_port); + if (pipeline.getNumStreams() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "DistinctStep with in-order expects single input"); if (distinct_sort_desc.size() < columns.size()) { From 0f8feff4d3806fa6f81d24184ab68bcd6e727551 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:34:30 +0200 Subject: [PATCH 0480/1170] Add KeeperMap retries --- src/Storages/StorageKeeperMap.cpp | 132 +++++++++++++++++------------- 1 file changed, 73 insertions(+), 59 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index ef157239e26..5534bb7f346 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -35,6 +35,7 @@ #include #include +#include "Common/ZooKeeper/ZooKeeperRetries.h" #include #include #include @@ -120,7 +121,7 @@ public: : SinkToStorage(header), storage(storage_), context(std::move(context_)) { auto primary_key = storage.getPrimaryKey(); - assert(primary_key.size() == 1); + chassert(primary_key.size() == 1); primary_key_pos = getHeader().getPositionByName(primary_key[0]); } @@ -171,76 +172,89 @@ public: template void finalize(bool strict) { - auto zookeeper = storage.getClient(); + const auto & settings = context->getSettingsRef(); - auto keys_limit = storage.keysLimit(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.insert_keeper_max_retries, + settings.insert_keeper_retry_initial_backoff_ms, + settings.insert_keeper_retry_max_backoff_ms}, + context->getProcessListElement()}; - size_t current_keys_num = 0; - size_t new_keys_num = 0; - - // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded - // (e.g if parallel insert queries are being run) - if (keys_limit != 0) + retries_ctl.retryLoop([&]() { - Coordination::Stat data_stat; - zookeeper->get(storage.dataPath(), &data_stat); - current_keys_num = data_stat.numChildren; - } + auto zookeeper = storage.getClient(); + auto keys_limit = storage.keysLimit(); - std::vector key_paths; - key_paths.reserve(new_values.size()); - for (const auto & [key, _] : new_values) - key_paths.push_back(storage.fullPathForKey(key)); + size_t current_keys_num = 0; + size_t new_keys_num = 0; - zkutil::ZooKeeper::MultiExistsResponse results; - - if constexpr (!for_update) - { - if (!strict) - results = zookeeper->exists(key_paths); - } - - Coordination::Requests requests; - requests.reserve(key_paths.size()); - for (size_t i = 0; i < key_paths.size(); ++i) - { - auto key = fs::path(key_paths[i]).filename(); - - if constexpr (for_update) + // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded + // (e.g if parallel insert queries are being run) + if (keys_limit != 0) { - int32_t version = -1; - if (strict) - version = versions.at(key); - - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], version)); + Coordination::Stat data_stat; + zookeeper->get(storage.dataPath(), &data_stat); + current_keys_num = data_stat.numChildren; } - else + + std::vector key_paths; + key_paths.reserve(new_values.size()); + for (const auto & [key, _] : new_values) + key_paths.push_back(storage.fullPathForKey(key)); + + zkutil::ZooKeeper::MultiExistsResponse results; + + if constexpr (!for_update) { - if (!strict && results[i].error == Coordination::Error::ZOK) + if (!strict) + results = zookeeper->exists(key_paths); + } + + Coordination::Requests requests; + requests.reserve(key_paths.size()); + for (size_t i = 0; i < key_paths.size(); ++i) + { + auto key = fs::path(key_paths[i]).filename(); + + if constexpr (for_update) { - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + int32_t version = -1; + if (strict) + version = versions.at(key); + + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], version)); } else { - requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); - ++new_keys_num; + if (!strict && results[i].error == Coordination::Error::ZOK) + { + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(key_paths[i], new_values[key], zkutil::CreateMode::Persistent)); + ++new_keys_num; + } } } - } - if (new_keys_num != 0) - { - auto will_be = current_keys_num + new_keys_num; - if (keys_limit != 0 && will_be > keys_limit) - throw Exception( - ErrorCodes::LIMIT_EXCEEDED, - "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", - new_keys_num, - keys_limit, - will_be); - } + if (new_keys_num != 0) + { + auto will_be = current_keys_num + new_keys_num; + if (keys_limit != 0 && will_be > keys_limit) + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", + new_keys_num, + keys_limit, + will_be); + } - zookeeper->multi(requests, /* check_session_valid */ true); + zookeeper->multi(requests, /* check_session_valid */ true); + }); } }; @@ -529,8 +543,8 @@ Pipe StorageKeeperMap::read( size_t num_keys = keys->size(); size_t num_threads = std::min(num_streams, keys->size()); - assert(num_keys <= std::numeric_limits::max()); - assert(num_threads <= std::numeric_limits::max()); + chassert(num_keys <= std::numeric_limits::max()); + chassert(num_threads <= std::numeric_limits::max()); for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) { @@ -1160,7 +1174,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca bool strict = local_context->getSettingsRef().keeper_map_strict_mode; - assert(commands.size() == 1); + chassert(commands.size() == 1); auto metadata_snapshot = getInMemoryMetadataPtr(); auto storage = getStorageID(); @@ -1236,7 +1250,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca return; } - assert(commands.front().type == MutationCommand::Type::UPDATE); + chassert(commands.front().type == MutationCommand::Type::UPDATE); if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); From 2530c5eb41a759baded5380a4e697c2e884c0abd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 12:37:35 +0200 Subject: [PATCH 0481/1170] Fix tests --- tests/queries/0_stateless/02952_archive_parsing.reference | 0 tests/queries/0_stateless/02952_archive_parsing.sql | 1 - tests/queries/0_stateless/03214_parsing_archive_name_file.sh | 2 +- 3 files changed, 1 insertion(+), 2 deletions(-) delete mode 100644 tests/queries/0_stateless/02952_archive_parsing.reference delete mode 100644 tests/queries/0_stateless/02952_archive_parsing.sql diff --git a/tests/queries/0_stateless/02952_archive_parsing.reference b/tests/queries/0_stateless/02952_archive_parsing.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02952_archive_parsing.sql b/tests/queries/0_stateless/02952_archive_parsing.sql deleted file mode 100644 index 49b0223e6ec..00000000000 --- a/tests/queries/0_stateless/02952_archive_parsing.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT * FROM file('::a'); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh index 2f77627f6be..b54cbb10aa6 100755 --- a/tests/queries/0_stateless/03214_parsing_archive_name_file.sh +++ b/tests/queries/0_stateless/03214_parsing_archive_name_file.sh @@ -11,7 +11,7 @@ function try_to_read_file() settings=$3 echo $file_argument $settings - $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | rg -c "Cannot stat file.*$file_to_read" + $CLICKHOUSE_LOCAL -q "SELECT * FROM file('$file_argument') $settings" 2>&1 | grep -c "Cannot stat file.*$file_to_read" } # if archive extension is not detected for part before '::', path is taken as is From 27a15bc5ace68acb0ba62791ca6e3d3f17ae569c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 10:39:35 +0000 Subject: [PATCH 0482/1170] Remove sh files --- .../03036_dynamic_read_subcolumns_1.sh | 19 ------------------- .../03036_dynamic_read_subcolumns_2.sh | 19 ------------------- .../03036_dynamic_read_subcolumns_3.sh | 19 ------------------- 3 files changed, 57 deletions(-) delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh delete mode 100755 tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh deleted file mode 100755 index ed548ae74e9..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_1.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "Memory" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=Memory" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh deleted file mode 100755 index 95dafcf5832..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_2.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" -test -$CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh deleted file mode 100755 index a3c2d93e568..00000000000 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_3.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -# shellcheck source=./03036_dynamic_read_subcolumns.lib -. "$CUR_DIR"/03036_dynamic_read_subcolumns.lib - -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" - -$CH_CLIENT -q "drop table if exists test;" - -echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" -test -$CH_CLIENT -q "drop table test;" From 72d6467fd2c34a82e1ef8ac73a451240843279a6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 10:43:21 +0000 Subject: [PATCH 0483/1170] Bump Azure to 1.13 --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index ea3e19a7be0..67272b7ee0a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 +Subproject commit 67272b7ee0adff6b69921b26eb071ba1a353062c From d4537d91875d1ecec832af94fde15073c45a63d7 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 30 Jul 2024 12:07:47 +0200 Subject: [PATCH 0484/1170] Add `groupConcat` to fuzzer https://github.com/ClickHouse/ClickHouse/pull/65384 --- tests/fuzz/dictionaries/functions.dict | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index ec7f8017fb2..6f2a88c22fa 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -1588,6 +1588,7 @@ "groupBitmapXorResample" "groupBitmapXorSimpleState" "groupBitmapXorState" +"groupConcat" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" From d4d3d590e38436da44b13dbf11a92cc6d00863e7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 20:06:55 +0000 Subject: [PATCH 0485/1170] Remove obsolete --multiquery parameter from tests --- .../integration/test_distributed_format/test.py | 8 ++++---- ...80_client_break_at_exception_in_batch_mode.sh | 2 +- .../00463_long_sessions_in_http_interface.sh | 10 +++++----- .../0_stateless/00474_readonly_settings.sh | 4 ++-- .../0_stateless/00612_pk_in_tuple_perf.sh | 4 ++-- .../0_stateless/00630_arbitrary_csv_delimiter.sh | 6 +++--- .../00650_csv_with_specified_quote_rule.sh | 4 ++-- ...00651_default_database_on_client_reconnect.sh | 2 +- .../00652_mutations_default_database.sh | 2 +- ...cated_mutations_default_database_zookeeper.sh | 2 +- .../00699_materialized_view_mutations.sh | 8 ++++---- .../00704_drop_truncate_memory_table.sh | 6 +++--- .../0_stateless/00705_drop_create_merge_tree.sh | 4 ++-- .../0_stateless/00763_lock_buffer_long.sh | 4 ++-- ...3_long_lock_buffer_alter_destination_table.sh | 4 ++-- .../00825_protobuf_format_array_3dim.sh | 2 +- .../00825_protobuf_format_array_of_arrays.sh | 2 +- .../00825_protobuf_format_enum_mapping.sh | 2 +- .../0_stateless/00825_protobuf_format_map.sh | 2 +- .../00825_protobuf_format_nested_in_nested.sh | 2 +- .../00825_protobuf_format_nested_optional.sh | 2 +- .../00825_protobuf_format_no_length_delimiter.sh | 6 +++--- .../0_stateless/00825_protobuf_format_persons.sh | 4 ++-- ...5_protobuf_format_skipped_column_in_nested.sh | 2 +- .../00825_protobuf_format_splitted_nested.sh | 2 +- .../0_stateless/00825_protobuf_format_squares.sh | 2 +- .../00825_protobuf_format_table_default.sh | 2 +- .../0_stateless/00900_long_parquet_load.sh | 2 +- .../0_stateless/00900_orc_arrow_parquet_maps.sh | 2 +- .../00937_format_schema_rows_template.sh | 4 ++-- .../0_stateless/00956_sensitive_data_masking.sh | 12 ++++++------ .../01019_alter_materialized_view_atomic.sh | 2 +- .../01019_alter_materialized_view_consistent.sh | 2 +- .../0_stateless/01035_lc_empty_part_bug.sh | 4 ++-- ...ystem_reload_dictionary_reloads_completely.sh | 2 +- .../01052_window_view_proc_tumble_to_now.sh | 2 +- .../01053_window_view_proc_hop_to_now.sh | 2 +- .../01054_window_view_proc_tumble_to.sh | 2 +- .../0_stateless/01055_window_view_proc_hop_to.sh | 2 +- ...057_window_view_event_tumble_to_strict_asc.sh | 2 +- .../01058_window_view_event_hop_to_strict_asc.sh | 2 +- .../01060_window_view_event_tumble_to_asc.sh | 2 +- .../01061_window_view_event_hop_to_asc.sh | 2 +- .../01063_window_view_event_tumble_to_bounded.sh | 2 +- .../01064_window_view_event_hop_to_bounded.sh | 2 +- ...w_view_event_tumble_to_strict_asc_lateness.sh | 2 +- ...7_window_view_event_tumble_to_asc_lateness.sh | 2 +- ...ndow_view_event_tumble_to_bounded_lateness.sh | 2 +- .../01071_window_view_event_tumble_asc_join.sh | 2 +- ...01072_window_view_multiple_columns_groupby.sh | 2 +- ...3_window_view_event_tumble_to_asc_populate.sh | 4 ++-- ...window_view_event_tumble_asc_join_populate.sh | 2 +- ...75_window_view_proc_tumble_to_now_populate.sh | 2 +- .../01076_window_view_alter_query_to.sh | 4 ++-- ...7_window_view_alter_query_to_modify_source.sh | 4 ++-- ...1079_window_view_inner_table_memory_tumble.sh | 2 +- .../01080_window_view_inner_table_memory_hop.sh | 2 +- .../01081_window_view_target_table_engine.sh | 2 +- .../0_stateless/01083_window_view_select.sh | 2 +- .../01084_window_view_with_table_identifier.sh | 2 +- .../0_stateless/01086_window_view_cleanup.sh | 2 +- .../0_stateless/01087_window_view_alter_query.sh | 4 ++-- .../01088_window_view_default_column.sh | 2 +- .../0_stateless/01133_begin_commit_race.sh | 6 +++--- .../01169_alter_partition_isolation_stress.sh | 4 ++-- ...01169_old_alter_partition_isolation_stress.sh | 10 +++++----- .../01171_mv_select_insert_isolation_long.sh | 14 +++++++------- .../0_stateless/01174_select_insert_isolation.sh | 8 ++++---- .../0_stateless/01198_client_quota_key.sh | 2 +- .../0_stateless/01285_engine_join_donmikel.sh | 4 ++-- .../0_stateless/01293_optimize_final_force.sh | 2 +- .../queries/0_stateless/01304_direct_io_long.sh | 4 ++-- ...orage_file_tsv_csv_with_names_write_prefix.sh | 4 ++-- .../0_stateless/01443_merge_truncate_long.sh | 2 +- .../01527_clickhouse_local_optimize.sh | 2 +- .../01543_avro_deserialization_with_lc.sh | 2 +- .../0_stateless/01544_file_engine_settings.sh | 4 ++-- .../0_stateless/01600_detach_permanently.sh | 2 +- .../01600_parts_states_metrics_long.sh | 2 +- tests/queries/0_stateless/01606_git_import.sh | 4 ++-- .../0_stateless/01607_arrays_as_nested_csv.sh | 4 ++-- .../0_stateless/01632_tinylog_read_write.sh | 4 ++-- .../01658_read_file_to_stringcolumn.sh | 4 ++-- .../01666_merge_tree_max_query_limit.sh | 2 +- .../0_stateless/01747_system_session_log_long.sh | 6 +++--- .../01801_nullable_low_cardinality_tsv.sh | 2 +- .../01834_alias_columns_laziness_filimonov.sh | 2 +- .../01923_network_receive_time_metric_insert.sh | 4 ++-- .../01939_network_receive_bytes_metrics.sh | 4 ++-- .../01946_test_wrong_host_name_access.sh | 2 +- ..._with_escape_sequence_at_the_end_of_buffer.sh | 2 +- tests/queries/0_stateless/02009_from_infile.sh | 2 +- .../0_stateless/02024_compression_in_query.sh | 6 +++--- .../02048_parallel_reading_from_infile.sh | 6 +++--- ...02104_clickhouse_local_columns_description.sh | 2 +- ...insert_deduplication_token_multiple_blocks.sh | 10 +++++----- ...eduplication_token_multiple_blocks_replica.sh | 10 +++++----- .../queries/0_stateless/02125_many_mutations.sh | 6 +++--- .../0_stateless/02125_many_mutations_2.sh | 6 +++--- .../queries/0_stateless/02135_local_create_db.sh | 2 +- .../0_stateless/02151_client_option_echo.sh | 4 ++-- .../0_stateless/02151_hash_table_sizes_stats.sh | 2 +- .../02151_hash_table_sizes_stats_distributed.sh | 2 +- .../02158_explain_ast_alter_commands.sh | 2 +- .../02206_clickhouse_local_use_database.sh | 2 +- .../02226_filesystem_cache_profile_events.sh | 14 +++++++------- .../02227_test_create_empty_sqlite_db.sh | 4 ++-- .../0_stateless/02235_remote_fs_cache_stress.sh | 6 +++--- .../02240_protobuflist_format_persons.sh | 4 ++-- .../02246_clickhouse_local_drop_database.sh | 4 ++-- .../0_stateless/02286_drop_filesystem_cache.sh | 2 +- .../02337_drop_filesystem_cache_access.sh | 8 ++++---- .../0_stateless/02364_window_view_segfault.sh | 2 +- .../0_stateless/02373_datetime64_monotonicity.sh | 2 +- .../0_stateless/02416_rename_database_rbac.sh | 4 ++-- .../02435_rollback_cancelled_queries.sh | 2 +- ...60_projections_and_aggregate_null_if_empty.sh | 2 +- .../02494_query_cache_user_isolation.sh | 8 ++++---- ...503_cache_on_write_with_small_segment_size.sh | 4 ++-- .../queries/0_stateless/02521_merge_over_gap.sh | 2 +- .../02530_dictionaries_update_field.sh | 2 +- .../0_stateless/02702_allow_skip_errors_enum.sh | 2 +- .../0_stateless/02704_keeper_map_zk_nodes.sh | 2 +- ...ckhouse_local_implicit_file_table_function.sh | 4 ++-- .../02712_bool_better_exception_message.sh | 8 ++++---- .../0_stateless/02722_database_filesystem.sh | 8 ++++---- tests/queries/0_stateless/02724_database_s3.sh | 14 +++++++------- tests/queries/0_stateless/02725_database_hdfs.sh | 10 +++++----- .../0_stateless/02725_local_query_parameters.sh | 2 +- .../02751_multiquery_with_argument.reference | 2 -- .../02751_multiquery_with_argument.sh | 16 ++++++---------- .../02815_no_throw_in_simple_queries.sh | 2 +- .../02843_insertion_table_schema_infer.sh | 2 +- .../02864_restore_table_with_broken_part.sh | 2 +- ...ultiple_batches_array_inconsistent_offsets.sh | 2 +- .../0_stateless/02875_merge_engine_set_index.sh | 2 +- .../02877_optimize_read_in_order_from_view.sh | 2 +- ...02884_create_view_with_sql_security_option.sh | 16 ++++++++-------- ...885_async_insert_access_check_for_defaults.sh | 4 ++-- ...900_clickhouse_local_drop_current_database.sh | 2 +- .../02956_clickhouse_local_system_parts.sh | 2 +- .../02973_backup_of_in_memory_compressed.sh | 10 +++++----- .../02973_parse_crlf_with_tsv_files.sh | 2 +- .../0_stateless/02995_forget_partition.sh | 4 ++-- tests/queries/0_stateless/02995_index_1.sh | 6 +++--- tests/queries/0_stateless/02995_index_10.sh | 6 +++--- tests/queries/0_stateless/02995_index_2.sh | 6 +++--- tests/queries/0_stateless/02995_index_3.sh | 6 +++--- tests/queries/0_stateless/02995_index_4.sh | 6 +++--- tests/queries/0_stateless/02995_index_5.sh | 6 +++--- tests/queries/0_stateless/02995_index_6.sh | 6 +++--- tests/queries/0_stateless/02995_index_7.sh | 6 +++--- tests/queries/0_stateless/02995_index_8.sh | 6 +++--- tests/queries/0_stateless/02995_index_9.sh | 6 +++--- .../0_stateless/02998_native_parquet_reader.sh | 2 +- .../03001_backup_matview_after_modify_query.sh | 2 +- .../03001_matview_columns_after_modify_query.sh | 2 +- .../03006_correct_revoke_for_partial_rights.sh | 2 +- .../03147_system_columns_access_checks.sh | 4 ++-- .../0_stateless/03201_local_named_collections.sh | 6 +++--- .../0_stateless/03212_thousand_exceptions.sh | 2 +- 161 files changed, 316 insertions(+), 322 deletions(-) diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 91afb8f7b34..5611f465e8b 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -55,7 +55,7 @@ def test_single_file(started_cluster, cluster): path = get_dist_path(cluster, "distr_1", 1) query = f"select * from file('{path}/1.bin', 'Distributed')" out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" @@ -65,7 +65,7 @@ def test_single_file(started_cluster, cluster): select * from t; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" @@ -106,7 +106,7 @@ def test_two_files(started_cluster, cluster): select * from t order by x; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "0\t_\n1\ta\n2\tbb\n3\tccc\n" @@ -141,7 +141,7 @@ def test_single_file_old(started_cluster, cluster): select * from t; """ out = node.exec_in_container( - ["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query] + ["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query] ) assert out == "1\ta\n2\tbb\n3\tccc\n" diff --git a/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh b/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh index 62f891db33c..0aab52d15c2 100755 --- a/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh +++ b/tests/queries/0_stateless/00380_client_break_at_exception_in_batch_mode.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query="SELECT 1; SELECT xyz; SELECT 2;" 2> /dev/null || true; +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT xyz; SELECT 2;" 2> /dev/null || true; diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh index d41d6409315..6ee1649c9ed 100755 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh @@ -25,7 +25,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_4&se ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_5&session_timeout=60" --data-binary "SELECT 1" echo "Sessions are local per user:" -${CLICKHOUSE_CLIENT} --multiquery --query "DROP USER IF EXISTS test_00463; CREATE USER test_00463; GRANT ALL ON *.* TO test_00463;" +${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS test_00463; CREATE USER test_00463; GRANT ALL ON *.* TO test_00463;" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6&session_timeout=600" --data-binary "CREATE TEMPORARY TABLE t (s String)" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "INSERT INTO t VALUES ('Hello')" @@ -37,7 +37,7 @@ ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${C ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" -${CLICKHOUSE_CLIENT} --multiquery --query "DROP USER test_00463"; +${CLICKHOUSE_CLIENT} --query "DROP USER test_00463"; echo "And cannot be accessed for a non-existent user:" ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=test_00463&session_id=${CLICKHOUSE_DATABASE}_6" --data-binary "SELECT * FROM t" | grep -c -F 'Exception' @@ -59,7 +59,7 @@ done echo "A session successfully expire after a timeout and the session's temporary table shadows the permanent table:" # An infinite loop is required to make the test reliable. We will check that the timeout corresponds to the observed time at least once -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (s String) ENGINE = Memory; INSERT INTO t VALUES ('World');" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (s String) ENGINE = Memory; INSERT INTO t VALUES ('World');" while true do ( @@ -70,7 +70,7 @@ do ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_8" --data-binary "SELECT * FROM t" ) | tr -d '\n' | grep -F 'HelloWorld' && break || sleep 1 done -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE t" +${CLICKHOUSE_CLIENT} --query "DROP TABLE t" echo "A session cannot be used by concurrent connections:" @@ -83,5 +83,5 @@ do done ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT 1" | grep -c -F 'SESSION_IS_LOCKED' -${CLICKHOUSE_CLIENT} --multiquery --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; +${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; wait diff --git a/tests/queries/0_stateless/00474_readonly_settings.sh b/tests/queries/0_stateless/00474_readonly_settings.sh index 3a857d81a74..ed3558c6d7a 100755 --- a/tests/queries/0_stateless/00474_readonly_settings.sh +++ b/tests/queries/0_stateless/00474_readonly_settings.sh @@ -8,8 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT --readonly=1 --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT --readonly=1 --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value diff --git a/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh b/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh index c8297635c43..7b2973669de 100755 --- a/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh +++ b/tests/queries/0_stateless/00612_pk_in_tuple_perf.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -$CLICKHOUSE_CLIENT --multiquery </dev/null ||: +${CLICKHOUSE_CLIENT} --ignore-error --query "DROP TABLE IF EXISTS tab_00651; CREATE TABLE tab_00651 (val UInt64) engine = Memory; SHOW CREATE TABLE tab_00651 format abcd; DESC tab_00651; DROP TABLE tab_00651;" 2>/dev/null ||: diff --git a/tests/queries/0_stateless/00652_mutations_default_database.sh b/tests/queries/0_stateless/00652_mutations_default_database.sh index eed45540f9b..577943bc3fd 100755 --- a/tests/queries/0_stateless/00652_mutations_default_database.sh +++ b/tests/queries/0_stateless/00652_mutations_default_database.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --mutations_sync=1 << EOF +${CLICKHOUSE_CLIENT} --mutations_sync=1 << EOF DROP TABLE IF EXISTS mutations; DROP TABLE IF EXISTS for_subquery; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh index 0ac5a2f748a..d4f6d3b290c 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 --multiquery << EOF +${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 << EOF DROP TABLE IF EXISTS mutations_r1; DROP TABLE IF EXISTS for_subquery; diff --git a/tests/queries/0_stateless/00699_materialized_view_mutations.sh b/tests/queries/0_stateless/00699_materialized_view_mutations.sh index a0f7db536dc..07ca9bc0f67 100755 --- a/tests/queries/0_stateless/00699_materialized_view_mutations.sh +++ b/tests/queries/0_stateless/00699_materialized_view_mutations.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS view_00699; DROP TABLE IF EXISTS null_00699; @@ -20,14 +20,14 @@ SELECT count(), min(x), max(x) FROM view_00699; ALTER TABLE null_00699 DELETE WHERE x % 2 = 0;" --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; ALTER TABLE view_00699 DELETE WHERE x % 2 = 0; " --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; @@ -35,7 +35,7 @@ ALTER TABLE null_00699 DELETE WHERE x % 2 = 1; ALTER TABLE view_00699 DELETE WHERE x % 2 = 1; " --mutations_sync=1 -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SELECT count(), min(x), max(x) FROM null_00699; SELECT count(), min(x), max(x) FROM view_00699; diff --git a/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh b/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh index e1540d1a25e..e40da11b893 100755 --- a/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh +++ b/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS memory; CREATE TABLE memory (x UInt64) ENGINE = Memory; @@ -21,13 +21,13 @@ INSERT INTO memory SELECT * FROM numbers(1000);" # But if the table will be dropped before query - just pass. # It's Ok, because otherwise the test will depend on the race condition in the test itself. -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" SET max_threads = 1; SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|^0$|Exception' & sleep 0.05; -${CLICKHOUSE_CLIENT} --multiquery --query=" +${CLICKHOUSE_CLIENT} --query=" TRUNCATE TABLE memory; DROP TABLE memory; " diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index ea8b9d02e49..fd002668696 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & -yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & +yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT & +yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT & wait ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table" diff --git a/tests/queries/0_stateless/00763_lock_buffer_long.sh b/tests/queries/0_stateless/00763_lock_buffer_long.sh index 444a66767aa..92f917aa287 100755 --- a/tests/queries/0_stateless/00763_lock_buffer_long.sh +++ b/tests/queries/0_stateless/00763_lock_buffer_long.sh @@ -16,12 +16,12 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_2 (s String) ENGINE = Bu function thread1() { - seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --fsync-metadata 0 --multiquery --ignore-error ||: + seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --fsync-metadata 0 --ignore-error ||: } function thread2() { - seq 1 500 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' + seq 1 500 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' } thread1 & diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 7e2384cfc52..79df667d45f 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), function thread1() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --ignore-error ||: } function thread2() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' } thread1 & diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh index 3cd842a10ba..468ced802cd 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh @@ -9,7 +9,7 @@ SCHEMADIR=$CURDIR/format_schemas set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery < /dev/null < /dev/null < $row_format_file -$CLICKHOUSE_CLIENT --multiline --multiquery --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ +$CLICKHOUSE_CLIENT --multiline --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ format_template_row = '$row_format_file', \ format_template_row_format = 'Question: \${question:Quoted}, Answer: \${answer:Quoted}, Number of Likes: \${likes:Raw}, Date: \${date:Raw}', \ format_template_rows_between_delimiter = ';\n'; --{clientError 474}" @@ -38,7 +38,7 @@ format_template_rows_between_delimiter = ';\n'"; # Test that if both format_template_result_format setting and format_template_resultset are provided, error is thrown resultset_output_file="$CURDIR"/"$CLICKHOUSE_TEST_UNIQUE_NAME"_template_output_format_resultset.tmp echo -ne '===== Resultset ===== \n \${data} \n ===============' > $resultset_output_file -$CLICKHOUSE_CLIENT --multiline --multiquery --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ +$CLICKHOUSE_CLIENT --multiline --query "SELECT * FROM template GROUP BY question, answer, likes, date WITH TOTALS ORDER BY date LIMIT 3 FORMAT Template SETTINGS \ format_template_resultset = '$resultset_output_file', \ format_template_resultset_format = '===== Resultset ===== \n \${data} \n ===============', \ format_template_row_format = 'Question: \${question:Quoted}, Answer: \${answer:Quoted}, Number of Likes: \${likes:Raw}, Date: \${date:Raw}', \ diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index 926557e4ba6..bd65b937648 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -17,7 +17,7 @@ echo 1 # normal execution $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM numbers(1) FORMAT Null" \ - --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 + --log_queries=1 --ignore-error >"$tmp_file" 2>&1 grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 1a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 1b' @@ -38,7 +38,7 @@ echo 3 # failure at before query start $CLICKHOUSE_CLIENT \ --query="SELECT 1 FROM system.numbers WHERE credit_card_number='find_me_TOPSECRET=TOPSECRET' FORMAT Null" \ - --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > "$tmp_file" + --log_queries=1 --ignore-error |& grep -v '^(query: ' > "$tmp_file" grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 3a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 3b' @@ -56,7 +56,7 @@ echo 4 # failure at the end of query $CLICKHOUSE_CLIENT \ --query="SELECT 'find_me_TOPSECRET=TOPSECRET', intDiv( 100, number - 10) FROM numbers(11) FORMAT Null" \ - --log_queries=1 --ignore-error --max_block_size=2 --multiquery |& grep -v '^(query: ' > "$tmp_file" + --log_queries=1 --ignore-error --max_block_size=2 |& grep -v '^(query: ' > "$tmp_file" grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 4a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 4b' @@ -67,7 +67,7 @@ rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ --function_sleep_max_microseconds_per_block 60000000 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ - --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & + --log_queries=1 --ignore-error |& grep -v '^(query: ' > $tmp_file2" & rm -f "$tmp_file" >/dev/null 2>&1 # check that executing query doesn't expose secrets in processlist @@ -133,7 +133,7 @@ insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' insert into sensitive select number as id, toDate('2019-01-01') as date, 'find_me_TOPSECRET=TOPSECRET' as value1, rand() as valuer from numbers(10); insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' as value1, rand() as valuer from numbers(10000); select * from sensitive WHERE value1 = 'find_me_TOPSECRET=TOPSECRET' FORMAT Null; -drop table sensitive;" --log_queries=1 --ignore-error --multiquery >"$tmp_file" 2>&1 +drop table sensitive;" --log_queries=1 --ignore-error >"$tmp_file" 2>&1 grep -F 'find_me_[hidden]' "$tmp_file" >/dev/null || echo 'fail 8a' grep -F 'TOPSECRET' "$tmp_file" && echo 'fail 8b' @@ -144,7 +144,7 @@ echo 9 $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ --query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_date >= yesterday() and message like '%find_me%'; - select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery + select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error echo 'finish' rm -f "$tmp_file" >/dev/null 2>&1 diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index 4bd21fcee02..eb12a76eb62 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery <&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION @@ -24,7 +24,7 @@ function begin_commit_readonly() function begin_rollback_readonly() { while true; do - $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=wait_unknown --multiquery --query " + $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=wait_unknown --query " BEGIN TRANSACTION; SET TRANSACTION SNAPSHOT 42; ROLLBACK;" @@ -34,7 +34,7 @@ function begin_rollback_readonly() function begin_insert_commit() { while true; do - $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=async --multiquery --query " + $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=async --query " BEGIN TRANSACTION; INSERT INTO mt VALUES ($RANDOM); COMMIT;" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index d4884cbf457..8873fd88f0e 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -22,7 +22,7 @@ function thread_insert() set -eu val=1 while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); @@ -210,7 +210,7 @@ function thread_select() set -eu while true; do output=$( - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; -- no duplicates SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh index 0d2016952d4..404042ab64e 100755 --- a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -19,7 +19,7 @@ function thread_insert() set -e val=1 while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); @@ -40,7 +40,7 @@ function thread_partition_src_to_dst() sum=0 for i in {1..20}; do out=$( - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($i, 3) */ ($i, 3); INSERT INTO dst SELECT * FROM src; @@ -49,7 +49,7 @@ function thread_partition_src_to_dst() SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; COMMIT;" 2>&1) ||: - echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --multiquery --query " + echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --query " begin transaction; set transaction snapshot 3; select $i, 'src', type, n, _part from src order by type, n; @@ -68,7 +68,7 @@ function thread_partition_dst_to_src() if (( i % 2 )); then action="COMMIT" fi - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " SYSTEM STOP MERGES dst; ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick SYSTEM SYNC TRANSACTION LOG; @@ -87,7 +87,7 @@ function thread_select() { set -e while true; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; -- no duplicates SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 3b9bb50517d..2fb58e4cc57 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -65,7 +65,7 @@ function insert_commit_action() local tag=$1; shift # some transactions will fail due to constraint - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($i, $tag) */ ($i, $tag); SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $tag))) /* ($i, $tag) */ FORMAT Null; @@ -83,7 +83,7 @@ function insert_rollback_action() local i=$1; shift local tag=$1; shift - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* (42, $tag) */ (42, $tag); SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$tag) != 1) FORMAT Null; @@ -112,7 +112,7 @@ function optimize_action() action="ROLLBACK" fi - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; $optimize_query; $action; @@ -126,7 +126,7 @@ function select_action() { set -e - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; @@ -140,7 +140,7 @@ function select_insert_action() { set -e - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null; INSERT INTO tmp SELECT 1, n*m FROM src; @@ -199,7 +199,7 @@ wait $PID_8 || echo "second select_insert_action has failed with status $?" 2>&1 wait_for_queries_to_finish $WAIT_FINISH -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; @@ -209,7 +209,7 @@ $CLICKHOUSE_CLIENT --multiquery --query " COMMIT; " -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 6321f6ff01b..235d98fb5de 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree OR function thread_insert_commit() { for i in {1..50}; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO mt VALUES /* ($i, $1) */ ($i, $1); INSERT INTO mt VALUES /* (-$i, $1) */ (-$i, $1); @@ -27,7 +27,7 @@ function thread_insert_commit() function thread_insert_rollback() { for _ in {1..50}; do - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; INSERT INTO mt VALUES /* (42, $1) */ (42, $1); ROLLBACK;"; @@ -38,7 +38,7 @@ function thread_select() { while true; do # The first and the last queries must get the same result - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SET throw_on_unsupported_query_inside_transaction=0; CREATE TEMPORARY TABLE tmp AS SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt FORMAT Null; @@ -58,7 +58,7 @@ kill -TERM $PID_4 wait wait_for_queries_to_finish 40 -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; diff --git a/tests/queries/0_stateless/01198_client_quota_key.sh b/tests/queries/0_stateless/01198_client_quota_key.sh index 3f5f5df5071..d08aa2e364f 100755 --- a/tests/queries/0_stateless/01198_client_quota_key.sh +++ b/tests/queries/0_stateless/01198_client_quota_key.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'" +$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'" diff --git a/tests/queries/0_stateless/01285_engine_join_donmikel.sh b/tests/queries/0_stateless/01285_engine_join_donmikel.sh index 7522ed9924b..ce273ab8e0c 100755 --- a/tests/queries/0_stateless/01285_engine_join_donmikel.sh +++ b/tests/queries/0_stateless/01285_engine_join_donmikel.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS NmSubj; DROP TABLE IF EXISTS events; @@ -60,7 +60,7 @@ FROM events as e INNER JOIN NmSubj as ns ON ns.NmId = toUInt32(e.Param1) WHERE e.EventDate = today() - 7 AND e.EventId = 'GCO' AND ns.SubjectId = 2073" -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE NmSubj; DROP TABLE events; " diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index e838af8af9b..9c135d272e4 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -11,7 +11,7 @@ TIMELIMIT=31 while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ]; do it=$((it+1)) - $CLICKHOUSE_CLIENT --multiquery --query " + $CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt8, k UInt8 DEFAULT 0) ENGINE = SummingMergeTree ORDER BY k; diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index 2e27c2f7728..1241f299d94 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS bug; CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi', merge_max_block_size = 8192; @@ -18,5 +18,5 @@ cat "$LOG" | grep Loaded rm "$LOG" -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE bug;" diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh index af8d3f4e69b..a634f689dca 100755 --- a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo 'zero rows' for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format - ${CLICKHOUSE_LOCAL} --multiquery --query=" + ${CLICKHOUSE_LOCAL} --query=" CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1) WHERE number < 0; SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; @@ -22,7 +22,7 @@ echo 'multi clickhouse-local one file' for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format for _ in {1..2}; do - ${CLICKHOUSE_LOCAL} --multiquery --query=" + ${CLICKHOUSE_LOCAL} --query=" CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1); SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.sh b/tests/queries/0_stateless/01443_merge_truncate_long.sh index 65b9bcd366e..51654b2e4e1 100755 --- a/tests/queries/0_stateless/01443_merge_truncate_long.sh +++ b/tests/queries/0_stateless/01443_merge_truncate_long.sh @@ -34,7 +34,7 @@ do SELECT count() FROM t HAVING count() > 0; SELECT ${i}; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} wait diff --git a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh index d61d34244d9..c1d5c357308 100755 --- a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh +++ b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh @@ -10,6 +10,6 @@ rm -rf "${WORKING_FOLDER_01527}" mkdir -p "${WORKING_FOLDER_01527}" # OPTIMIZE was crashing due to lack of temporary volume in local -${CLICKHOUSE_LOCAL} --multiquery --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}" +${CLICKHOUSE_LOCAL} --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}" rm -rf "${WORKING_FOLDER_01527}" diff --git a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh index a5697a62dc2..bc9efaedd5d 100755 --- a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh +++ b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --query " SET allow_suspicious_low_cardinality_types=1; CREATE TABLE IF NOT EXISTS test_01543 (value LowCardinality(String), value2 LowCardinality(UInt64)) ENGINE=Memory(); " diff --git a/tests/queries/0_stateless/01544_file_engine_settings.sh b/tests/queries/0_stateless/01544_file_engine_settings.sh index b31754f9531..eb0a8a964d0 100755 --- a/tests/queries/0_stateless/01544_file_engine_settings.sh +++ b/tests/queries/0_stateless/01544_file_engine_settings.sh @@ -10,7 +10,7 @@ rm -f -- "$the_file" # We are going to check that format settings work for File engine, # by creating a table with a non-default delimiter, and reading from it. -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; insert into t select 1 a, 1 b; " @@ -18,7 +18,7 @@ ${CLICKHOUSE_LOCAL} --multiquery --query " # See what's in the file cat "$the_file" -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|'; select * from t; " diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index 6721dbf3015..679e9a749ee 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -18,7 +18,7 @@ mkdir -p "${WORKING_FOLDER_01600}" clickhouse_local() { local query="$1" shift - ${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --multiquery --query "$query" "$@" --path="${WORKING_FOLDER_01600}" + ${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --query "$query" "$@" --path="${WORKING_FOLDER_01600}" } test_detach_attach_sequence() { 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 7215f270a4c..47b5a4dea13 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -24,7 +24,7 @@ verify() if [[ $i -eq 5000 ]] then - $CLICKHOUSE_CLIENT --multiquery " + $CLICKHOUSE_CLIENT " 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; diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index 48558d79f93..6986d6b14cf 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -19,7 +19,7 @@ done ${CLICKHOUSE_GIT_IMPORT} 2>&1 | wc -l -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " DROP TABLE IF EXISTS commits; DROP TABLE IF EXISTS file_changes; @@ -122,7 +122,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM commits" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM file_changes" ${CLICKHOUSE_CLIENT} --query "SELECT count(), round(avg(indent), 1) FROM line_changes" -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " DROP TABLE commits; DROP TABLE file_changes; DROP TABLE line_changes; diff --git a/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh b/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh index 946be7fb4af..2a1182c14c1 100755 --- a/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh +++ b/tests/queries/0_stateless/01607_arrays_as_nested_csv.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS test; CREATE TABLE test (a Array(String)) ENGINE = Memory; " @@ -22,7 +22,7 @@ ${CLICKHOUSE_CLIENT} --input_format_csv_arrays_as_nested_csv 1 --query "INSERT I """Hello"", ""world"", ""42"""" TV""" END -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM test; DROP TABLE IF EXISTS test; " diff --git a/tests/queries/0_stateless/01632_tinylog_read_write.sh b/tests/queries/0_stateless/01632_tinylog_read_write.sh index 10625ec5d27..68d28b080e9 100755 --- a/tests/queries/0_stateless/01632_tinylog_read_write.sh +++ b/tests/queries/0_stateless/01632_tinylog_read_write.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test; CREATE TABLE IF NOT EXISTS test (x UInt64, s Array(Nullable(String))) ENGINE = TinyLog;" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test; CREATE TABLE IF NOT EXISTS test (x UInt64, s Array(Nullable(String))) ENGINE = TinyLog;" function thread_select { local TIMELIMIT=$((SECONDS+$1)) @@ -47,4 +47,4 @@ thread_insert $TIMEOUT & wait echo "Done" -$CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test;" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test;" diff --git a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh index 685fe69642a..ceb6aa060ea 100755 --- a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh +++ b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh @@ -24,7 +24,7 @@ ${CLICKHOUSE_CLIENT} --query "select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$? ${CLICKHOUSE_CLIENT} --query "select file('c.txt'), * from data";echo ":"$? -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " create table filenames(name String) engine=MergeTree() order by tuple(); insert into filenames values ('a.txt'), ('b.txt'), ('c.txt'); select file(name) from filenames format TSV; @@ -56,7 +56,7 @@ echo $c_count # Valid cases: # The default dir is the CWD path in LOCAL mode -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " drop table if exists data; create table data (A String, B String) engine=MergeTree() order by A; select file('a.txt'), file('b.txt'); diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index e04c9515009..ec318db98bf 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -8,7 +8,7 @@ function wait_for_query_to_start() { while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT sum(read_rows) FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done } -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " drop table if exists simple; create table simple (i int, j int) engine = MergeTree order by i diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index 022bf488886..07055f96782 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -82,7 +82,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" + ! ${CLICKHOUSE_CLIENT} --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" } function createUser() @@ -303,7 +303,7 @@ function runEndpointTests() if [[ -n "${setup_queries}" ]] then # echo "Executing setup queries: ${setup_queries}" - echo "${setup_queries}" | executeQuery --multiquery + echo "${setup_queries}" | executeQuery fi testTCP "${auth_type}" "${username}" "${password}" @@ -357,7 +357,7 @@ testAsUserIdentifiedBy "plaintext_password" testAsUserIdentifiedBy "sha256_password" testAsUserIdentifiedBy "double_sha1_password" -executeQuery --multiquery <= 1000000 ? 1 : time FROM system.query_log WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" diff --git a/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh b/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh index 03babad40f3..b2335a0365b 100755 --- a/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh +++ b/tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" seq 1 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" -${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; WITH ProfileEvents['NetworkReceiveBytes'] AS bytes SELECT bytes >= 8000 AND bytes < 9000 ? 1 : bytes FROM system.query_log WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" diff --git a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh index a00f307673e..ed2828c3f54 100755 --- a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh +++ b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery --query " +${CLICKHOUSE_CLIENT} --query " DROP USER IF EXISTS dns_fail_1, dns_fail_2; CREATE USER dns_fail_1 HOST NAME 'non.existing.host.name', '${MYHOSTNAME}'; CREATE USER dns_fail_2 HOST NAME '${MYHOSTNAME}', 'non.existing.host.name';" diff --git a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh index 0aedef028a2..b3748581f4f 100755 --- a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh +++ b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) mkdir -p ${USER_FILES_PATH}/ cp $CUR_DIR/data_zstd/test_01946.zstd ${USER_FILES_PATH}/ -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +${CLICKHOUSE_CLIENT} --multiline --query " set min_chunk_bytes_for_parallel_parsing=10485760; set max_read_buffer_size = 65536; set input_format_parallel_parsing = 0; diff --git a/tests/queries/0_stateless/02009_from_infile.sh b/tests/queries/0_stateless/02009_from_infile.sh index 6a31aa4ac55..578ac14f558 100755 --- a/tests/queries/0_stateless/02009_from_infile.sh +++ b/tests/queries/0_stateless/02009_from_infile.sh @@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_infile;" # if it not fails, select will print information -${CLICKHOUSE_LOCAL} --multiquery --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;" +${CLICKHOUSE_LOCAL} --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=DROP+TABLE" -d 'IF EXISTS test_infile_url' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=CREATE" -d 'TABLE test_infile_url (x String) ENGINE = Memory' diff --git a/tests/queries/0_stateless/02024_compression_in_query.sh b/tests/queries/0_stateless/02024_compression_in_query.sh index caa74523bd7..2936568c991 100755 --- a/tests/queries/0_stateless/02024_compression_in_query.sh +++ b/tests/queries/0_stateless/02024_compression_in_query.sh @@ -55,8 +55,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_compression_keyword;" [ -e "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz ] && rm "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz # create files using compression method and without it to check that both queries work correct -${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;" -${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;" # check content of files cp ${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_to_decomp.gz @@ -68,7 +68,7 @@ gunzip ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp.gz cat ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp # create table to check inserts -${CLICKHOUSE_LOCAL} --multiquery --query " +${CLICKHOUSE_LOCAL} --query " DROP TABLE IF EXISTS test_compression_keyword; CREATE TABLE test_compression_keyword (text String) Engine=Memory; INSERT INTO TABLE test_compression_keyword FROM INFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated; diff --git a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh index f055ea304b2..efc19cad054 100755 --- a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh +++ b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh @@ -17,7 +17,7 @@ echo -e "103" > "${CLICKHOUSE_TMP}"/test_infile_parallel_3 gzip "${CLICKHOUSE_TMP}"/test_infile_parallel -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -q "27" && echo "Correct" || echo 'Fail' -${CLICKHOUSE_LOCAL} --multiquery <&1 | grep 'AlterCommand' + $CLICKHOUSE_CLIENT --readonly 1 2>&1 | grep 'AlterCommand' diff --git a/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh b/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh index 59ede739e4a..3b71c8754c9 100755 --- a/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh +++ b/tests/queries/0_stateless/02206_clickhouse_local_use_database.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --echo --multiline --multiquery -q """ +$CLICKHOUSE_LOCAL --echo --multiline -q """ SHOW TABLES; CREATE DATABASE test1; CREATE TABLE test1.table1 (a Int32) ENGINE=Memory; diff --git a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh index d0e61541b15..18ae2d7b4b3 100755 --- a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh +++ b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh @@ -10,7 +10,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do echo "Using storage policy: $STORAGE_POLICY" - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SET max_memory_usage='20G'; SET enable_filesystem_cache_on_write_operations = 0; @@ -25,7 +25,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1" 2>&1) - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -40,14 +40,14 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do LIMIT 1; """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ set remote_filesystem_read_method = 'read'; set local_filesystem_read_method = 'pread'; """ query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1" 2>&1) - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -63,13 +63,13 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ set remote_filesystem_read_method='threadpool'; """ query_id=$($CLICKHOUSE_CLIENT --query "select queryID() from ($query) limit 1") - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SYSTEM FLUSH LOGS; SELECT ProfileEvents['CachedReadBufferReadFromCacheHits'] > 0 as remote_fs_cache_hit, ProfileEvents['CachedReadBufferReadFromCacheMisses'] > 0 as remote_fs_cache_miss, @@ -84,7 +84,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 'azure_cache'; do LIMIT 1; """ - $CLICKHOUSE_CLIENT --multiquery --multiline --query """ + $CLICKHOUSE_CLIENT --multiline --query """ SELECT * FROM test_02226 WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; SET enable_filesystem_cache_on_write_operations = 1; diff --git a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh index 344452767cc..a3fe5f19de0 100755 --- a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh +++ b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh @@ -17,7 +17,7 @@ export CURR_DATABASE="test_01889_sqllite_${CLICKHOUSE_DATABASE}" DB_PATH=${USER_FILES_PATH}/${CURR_DATABASE}_db1 -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" DROP DATABASE IF EXISTS ${CURR_DATABASE}; CREATE DATABASE ${CURR_DATABASE} ENGINE = SQLite('${DB_PATH}'); SHOW TABLES FROM ${CURR_DATABASE}; @@ -25,6 +25,6 @@ SHOW TABLES FROM ${CURR_DATABASE}; sqlite3 "${DB_PATH}" 'CREATE TABLE table1 (col1 text, col2 smallint);' -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" SHOW TABLES FROM ${CURR_DATABASE}; """ diff --git a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh index ffc38c0c1bd..aa5db33417c 100755 --- a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh +++ b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_suspicious_low_cardinality_types=1 --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --allow_suspicious_low_cardinality_types=1 --multiline --query=""" DROP TABLE IF EXISTS t_01411; DROP TABLE IF EXISTS t_01411_num; @@ -35,7 +35,7 @@ insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, n function go() { -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" select sum(toUInt64(str)), sum(toUInt64(pat)) from lc_dict_reading where val < 8129 or val > 8192 * 4; @@ -67,7 +67,7 @@ for _ in `seq 1 32`; do go | grep -q "Exception" && echo 'FAIL' || echo 'OK' ||: wait -${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" +${CLICKHOUSE_CLIENT} --multiline --query=""" DROP TABLE IF EXISTS t_01411; DROP TABLE IF EXISTS t_01411_num; """ diff --git a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh index 637e01b9e63..e5e717d00a8 100755 --- a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh +++ b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh @@ -15,7 +15,7 @@ SCHEMADIR=$CURDIR/format_schemas set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <&1 | rg -Fc "'w' character" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'w' character" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": wtf}'); END -$CLICKHOUSE_LOCAL --multiquery <&1 | rg -Fc "expected 'false'" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "expected 'false'" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": ftw}'); END -$CLICKHOUSE_LOCAL --multiquery <&1 | rg -Fc "'{' character" +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'{' character" SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": {}}'); END diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 2d0ff256c95..fa23d847d90 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -30,7 +30,7 @@ cp ${user_files_tmp_dir}/tmp_numbers_1.csv ${user_files_tmp_dir}/tmp/tmp_numbers ################# echo "Test 1: create filesystem database and check implicit calls" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; CREATE DATABASE test1 ENGINE = Filesystem; """ @@ -57,20 +57,20 @@ ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --multiline --multiquery --query """ +${CLICKHOUSE_CLIENT} --multiline --query """ USE test1; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; """ 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: path should be inside user_files -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 80b47282146..cc7f012c8cf 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ################# echo "Test 1: select from s3" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; CREATE DATABASE test1 ENGINE = S3; USE test1; @@ -17,7 +17,7 @@ SELECT * FROM \"http://localhost:11111/test/a.tsv\" ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1 # check credentials with absolute path -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = S3('', 'test', 'testtest'); USE test2; @@ -25,7 +25,7 @@ SELECT * FROM \"http://localhost:11111/test/b.tsv\" """ # check credentials with relative path -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = S3('http://localhost:11111/test', 'test', 'testtest'); USE test4; @@ -33,7 +33,7 @@ SELECT * FROM \"b.tsv\" """ # Check named collection loading -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test5; CREATE DATABASE test5 ENGINE = S3(s3_conn_db); SELECT * FROM test5.\`b.tsv\` @@ -41,20 +41,20 @@ SELECT * FROM test5.\`b.tsv\` ################# echo "Test 2: check exceptions" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "S3_ERROR" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ USE test3; SELECT * FROM \"abacaba\" """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Cleanup -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test1; DROP DATABASE IF EXISTS test2; DROP DATABASE IF EXISTS test3; diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 1eb22976b84..7fd35c72ef1 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -25,7 +25,7 @@ fi echo "Test 1: select from hdfs database" # Database without specific host -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_1; CREATE DATABASE test_hdfs_1 ENGINE = HDFS; USE test_hdfs_1; @@ -34,7 +34,7 @@ SELECT * FROM \"hdfs://localhost:12222/test_02725_1.tsv\" ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test_hdfs_1 # Database with host -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_2; CREATE DATABASE test_hdfs_2 ENGINE = HDFS('hdfs://localhost:12222'); USE test_hdfs_2; @@ -45,12 +45,12 @@ ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test_hdfs_2 ################# echo "Test 2: check exceptions" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_3; CREATE DATABASE test_hdfs_3 ENGINE = HDFS('abacaba'); """ 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_4; CREATE DATABASE test_hdfs_4 ENGINE = HDFS; USE test_hdfs_4; @@ -64,7 +64,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222 # Cleanup -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ DROP DATABASE IF EXISTS test_hdfs_1; DROP DATABASE IF EXISTS test_hdfs_2; DROP DATABASE IF EXISTS test_hdfs_3; diff --git a/tests/queries/0_stateless/02725_local_query_parameters.sh b/tests/queries/0_stateless/02725_local_query_parameters.sh index 92d7f645454..151a854d5b9 100755 --- a/tests/queries/0_stateless/02725_local_query_parameters.sh +++ b/tests/queries/0_stateless/02725_local_query_parameters.sh @@ -5,6 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --multiquery -q " +$CLICKHOUSE_LOCAL -q " SET param_x=1; SELECT {x:UInt64}, {x:String};" diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 2e55712e49c..f02e9bab2cd 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -5,8 +5,6 @@ Syntax error Empty query Empty query -BAD_ARGUMENTS -BAD_ARGUMENTS 301 302 304 diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index 7b959a3c3dc..4021194656b 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -4,18 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --multiquery "SELECT 100" -$CLICKHOUSE_LOCAL --multiquery "SELECT 101;" -$CLICKHOUSE_LOCAL --multiquery "SELECT 102;SELECT 103;" +$CLICKHOUSE_LOCAL "SELECT 100" +$CLICKHOUSE_LOCAL "SELECT 101;" +$CLICKHOUSE_LOCAL "SELECT 102;SELECT 103;" # Invalid SQL. -$CLICKHOUSE_LOCAL --multiquery "SELECT 200; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_LOCAL --multiquery "; SELECT 201;" 2>&1 | grep -o 'Empty query' -$CLICKHOUSE_LOCAL --multiquery "; S; SELECT 202" 2>&1 | grep -o 'Empty query' - -# Simultaneously passing --queries-file + --query (multiquery) is prohibited. -$CLICKHOUSE_LOCAL --queries-file "queries.csv" --multiquery "SELECT 250;" 2>&1 | grep -o 'BAD_ARGUMENTS' -$CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL "SELECT 200; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_LOCAL "; SELECT 201;" 2>&1 | grep -o 'Empty query' +$CLICKHOUSE_LOCAL "; S; SELECT 202" 2>&1 | grep -o 'Empty query' # Error expectation cases. # -n is also interpreted as a query diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh index 68c55f9b66a..18ffc9dfec3 100755 --- a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --query "SHOW TABLES" || echo "Failed" $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables WHERE database = currentDatabase() FORMAT Null" || echo "Failed" # Multi queries are ok: -$CLICKHOUSE_LOCAL --multiquery "SELECT 1; SELECT 2;" || echo "Failed" +$CLICKHOUSE_LOCAL "SELECT 1; SELECT 2;" || echo "Failed" # It can run in interactive mode: function run() diff --git a/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh b/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh index d806b678456..9207e48092f 100755 --- a/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh +++ b/tests/queries/0_stateless/02843_insertion_table_schema_infer.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_DIR=$CUR_DIR/data_tsv -$CLICKHOUSE_LOCAL --multiquery \ +$CLICKHOUSE_LOCAL \ "CREATE VIEW users AS SELECT * FROM file('$DATA_DIR/mock_data.tsv', TSVWithNamesAndTypes); CREATE TABLE users_output (name String, tag UInt64)ENGINE = Memory; INSERT INTO users_output WITH (SELECT groupUniqArrayArray(mapKeys(Tags)) FROM users) AS unique_tags SELECT UserName AS name, length(unique_tags) AS tag FROM users; diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index 08313e2fd3b..229f832ba14 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -39,7 +39,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" # Then try to restore with the setting `restore_broken_parts_as_detached` set to true. $CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}') SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}' -$CLICKHOUSE_CLIENT --multiquery <&1 | grep -c "INVOKER") >= 1 )) && echo "OK" || echo "UNEXPECTED" (( $(${CLICKHOUSE_CLIENT} --query "SHOW TABLE $db.test_view_2" 2>&1 | grep -c "DEFINER = $user1") >= 1 )) && echo "OK" || echo "UNEXPECTED" -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" (( $(${CLICKHOUSE_CLIENT} --query "INSERT INTO $db.test_table VALUES ('foo'), ('bar');" 2>&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" -${CLICKHOUSE_CLIENT} --multiquery <\n" $CLICKHOUSE_CLIENT --query "SELECT * FROM file(${UNIX_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32');" -$CLICKHOUSE_CLIENT --multiquery --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32'); --{serverError 117}" +$CLICKHOUSE_CLIENT --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32'); --{serverError 117}" echo -e "\n<-- Read DOS endings with setting input_format_tsv_crlf_end_of_line=1 -->\n" $CLICKHOUSE_CLIENT --query "SELECT * FROM file(${DOS_ENDINGS}, 'TabSeparated', 'SearchTerm String, Date Date, Hits UInt32') SETTINGS input_format_tsv_crlf_end_of_line = 1;" diff --git a/tests/queries/0_stateless/02995_forget_partition.sh b/tests/queries/0_stateless/02995_forget_partition.sh index c22f5829130..6fa0b96e90d 100755 --- a/tests/queries/0_stateless/02995_forget_partition.sh +++ b/tests/queries/0_stateless/02995_forget_partition.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ drop table if exists forget_partition; create table forget_partition @@ -31,7 +31,7 @@ alter table forget_partition drop partition '20240102'; # DROP PARTITION do not wait for a part to be removed from memory due to possible concurrent SELECTs, so we have to do wait manually here while [[ $(${CLICKHOUSE_CLIENT} -q "select count() from system.parts where database=currentDatabase() and table='forget_partition' and partition IN ('20240101', '20240102')") != 0 ]]; do sleep 1; done -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +${CLICKHOUSE_CLIENT} --multiline -q """ set allow_unrestricted_reads_from_keeper=1; select '---before---'; diff --git a/tests/queries/0_stateless/02995_index_1.sh b/tests/queries/0_stateless/02995_index_1.sh index a5f1b30c2e8..128697fd0fe 100755 --- a/tests/queries/0_stateless/02995_index_1.sh +++ b/tests/queries/0_stateless/02995_index_1.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_10.sh b/tests/queries/0_stateless/02995_index_10.sh index d72c7c72705..c15ba00fd05 100755 --- a/tests/queries/0_stateless/02995_index_10.sh +++ b/tests/queries/0_stateless/02995_index_10.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_2.sh b/tests/queries/0_stateless/02995_index_2.sh index e7451c7ee4b..a32f5c511f8 100755 --- a/tests/queries/0_stateless/02995_index_2.sh +++ b/tests/queries/0_stateless/02995_index_2.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh index 506429e2696..9cc937391fc 100755 --- a/tests/queries/0_stateless/02995_index_3.sh +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_4.sh b/tests/queries/0_stateless/02995_index_4.sh index 1a0458728f9..e450997e48b 100755 --- a/tests/queries/0_stateless/02995_index_4.sh +++ b/tests/queries/0_stateless/02995_index_4.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_5.sh b/tests/queries/0_stateless/02995_index_5.sh index 60c12a8146d..80f75a532e3 100755 --- a/tests/queries/0_stateless/02995_index_5.sh +++ b/tests/queries/0_stateless/02995_index_5.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_6.sh b/tests/queries/0_stateless/02995_index_6.sh index 4936f73f36b..e90387c7c0c 100755 --- a/tests/queries/0_stateless/02995_index_6.sh +++ b/tests/queries/0_stateless/02995_index_6.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_7.sh b/tests/queries/0_stateless/02995_index_7.sh index 26be310abce..a5fdd98b2f8 100755 --- a/tests/queries/0_stateless/02995_index_7.sh +++ b/tests/queries/0_stateless/02995_index_7.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_8.sh b/tests/queries/0_stateless/02995_index_8.sh index 8c2620b59fd..adb835aedca 100755 --- a/tests/queries/0_stateless/02995_index_8.sh +++ b/tests/queries/0_stateless/02995_index_8.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_9.sh b/tests/queries/0_stateless/02995_index_9.sh index 76160c62aaa..4b78777cd2a 100755 --- a/tests/queries/0_stateless/02995_index_9.sh +++ b/tests/queries/0_stateless/02995_index_9.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} --multiquery +done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} --multiquery "DROP TABLE test" +${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02998_native_parquet_reader.sh b/tests/queries/0_stateless/02998_native_parquet_reader.sh index d6369c4921b..95b659815ed 100755 --- a/tests/queries/0_stateless/02998_native_parquet_reader.sh +++ b/tests/queries/0_stateless/02998_native_parquet_reader.sh @@ -208,4 +208,4 @@ CH_SCHEMA="\ QUERY="SELECT * from file('$PAR_PATH', 'Parquet', '$CH_SCHEMA')" # there may be more than on group in parquet files, unstable results may generated by multithreads -$CLICKHOUSE_LOCAL --multiquery --max_threads 1 --max_parsing_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY" +$CLICKHOUSE_LOCAL --max_threads 1 --max_parsing_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY" diff --git a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh index f857358a5ea..8c6aa70f14c 100755 --- a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh +++ b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh @@ -9,7 +9,7 @@ db="$CLICKHOUSE_DATABASE" db_2="${db}_2" backup_name="${db}_backup" -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS mv; CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; diff --git a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh index 2ec5832fac6..96cbd391a44 100755 --- a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh +++ b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS mv; CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; diff --git a/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh b/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh index 8c79dfdbafc..312fb03668c 100755 --- a/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh +++ b/tests/queries/0_stateless/03006_correct_revoke_for_partial_rights.sh @@ -8,7 +8,7 @@ db=${CLICKHOUSE_DATABASE} user1="user1_03006_${db}_$RANDOM" user2="user2_03006_${db}_$RANDOM" -${CLICKHOUSE_CLIENT} --multiquery <&1 | grep --text -F -v "ASan doesn't fully support makecontext/swapcontext functions" -${CLICKHOUSE_CLIENT} --multiquery " +${CLICKHOUSE_CLIENT} " DROP TABLE test; " diff --git a/tests/queries/0_stateless/03212_thousand_exceptions.sh b/tests/queries/0_stateless/03212_thousand_exceptions.sh index 0a6abf35c10..1237cbf537f 100755 --- a/tests/queries/0_stateless/03212_thousand_exceptions.sh +++ b/tests/queries/0_stateless/03212_thousand_exceptions.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # This should not be too slow, even under sanitizers. -yes "SELECT throwIf(1); SELECT '.' FORMAT Values;" | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery --ignore-error 2>/dev/null +yes "SELECT throwIf(1); SELECT '.' FORMAT Values;" | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error 2>/dev/null From 6372fdee6d344bd87d58ce89fa069b55750c9aba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 13:46:05 +0200 Subject: [PATCH 0486/1170] Update tests --- tests/queries/0_stateless/02995_index_3.sh | 2 +- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh index 506429e2696..219ae81154f 100755 --- a/tests/queries/0_stateless/02995_index_3.sh +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..a927e4f1e1f 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-distributed-cache DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From eb129b539fce2a407182d892ce3bd00f782a5833 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 13:46:27 +0200 Subject: [PATCH 0487/1170] Add tests --- src/Storages/StorageKeeperMap.cpp | 135 +++++++++++++----- src/Storages/StorageKeeperMap.h | 3 +- .../test_keeper_map_retries/__init__.py | 0 .../configs/enable_keeper_map.xml | 3 + .../configs/fault_injection.xml | 6 + .../test_keeper_map_retries/test.py | 78 ++++++++++ .../02911_backup_restore_keeper_map.sh | 15 +- 7 files changed, 194 insertions(+), 46 deletions(-) create mode 100644 tests/integration/test_keeper_map_retries/__init__.py create mode 100644 tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml create mode 100644 tests/integration/test_keeper_map_retries/configs/fault_injection.xml create mode 100644 tests/integration/test_keeper_map_retries/test.py diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 5534bb7f346..09c21ae28f5 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -183,7 +183,7 @@ public: settings.insert_keeper_retry_max_backoff_ms}, context->getProcessListElement()}; - retries_ctl.retryLoop([&]() + zk_retry.retryLoop([&]() { auto zookeeper = storage.getClient(); auto keys_limit = storage.keysLimit(); @@ -205,12 +205,12 @@ public: for (const auto & [key, _] : new_values) key_paths.push_back(storage.fullPathForKey(key)); - zkutil::ZooKeeper::MultiExistsResponse results; + zkutil::ZooKeeper::MultiTryGetResponse results; if constexpr (!for_update) { if (!strict) - results = zookeeper->exists(key_paths); + results = zookeeper->tryGet(key_paths); } Coordination::Requests requests; @@ -231,7 +231,8 @@ public: { if (!strict && results[i].error == Coordination::Error::ZOK) { - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + if (results[i].data != new_values[key]) + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); } else { @@ -241,6 +242,9 @@ public: } } + if (requests.empty()) + return; + if (new_keys_num != 0) { auto will_be = current_keys_num + new_keys_num; @@ -259,7 +263,7 @@ public: }; template -class StorageKeeperMapSource : public ISource +class StorageKeeperMapSource : public ISource, WithContext { const StorageKeeperMap & storage; size_t max_block_size; @@ -290,8 +294,15 @@ public: KeyContainerPtr container_, KeyContainerIter begin_, KeyContainerIter end_, - bool with_version_column_) - : ISource(getHeader(header, with_version_column_)), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + bool with_version_column_, + ContextPtr context_) + : ISource(getHeader(header, with_version_column_)) + , WithContext(std::move(context_)) + , storage(storage_) + , max_block_size(max_block_size_) + , container(std::move(container_)) + , it(begin_) + , end(end_) , with_version_column(with_version_column_) { } @@ -316,12 +327,12 @@ public: for (auto & raw_key : raw_keys) raw_key = base64Encode(raw_key, /* url_encoding */ true); - return storage.getBySerializedKeys(raw_keys, nullptr, with_version_column); + return storage.getBySerializedKeys(raw_keys, nullptr, with_version_column, getContext()); } else { size_t elem_num = std::min(max_block_size, static_cast(end - it)); - auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr, with_version_column); + auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr, with_version_column, getContext()); it += elem_num; return chunk; } @@ -553,14 +564,31 @@ Pipe StorageKeeperMap::read( using KeyContainer = typename KeyContainerPtr::element_type; pipes.emplace_back(std::make_shared>( - *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column)); + *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column, context_)); } return Pipe::unitePipes(std::move(pipes)); }; - auto client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(zk_data_path))); + { + const auto & settings = context_->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + context_->getProcessListElement()}; + + std::vector children; + zk_retry.retryLoop([&] + { + auto client = getClient(); + children = client->getChildren(zk_data_path); + }); + return process_keys(std::make_shared>(std::move(children))); + } return process_keys(std::move(filtered_keys)); } @@ -571,11 +599,24 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); } -void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { checkTable(); - auto client = getClient(); - client->tryRemoveChildrenRecursive(zk_data_path, true); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + zk_retry.retryLoop([&] + { + auto client = getClient(); + client->tryRemoveChildrenRecursive(zk_data_path, true); + }); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) @@ -1064,10 +1105,11 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD if (raw_keys.size() != keys[0].column->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); - return getBySerializedKeys(raw_keys, &null_map, /* version_column */ false); + return getBySerializedKeys(raw_keys, &null_map, /* version_column */ false, getContext()); } -Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map, bool with_version) const +Chunk StorageKeeperMap::getBySerializedKeys( + const std::span keys, PaddedPODArray * null_map, bool with_version, const ContextPtr & local_context) const { Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); MutableColumns columns = sample_block.cloneEmptyColumns(); @@ -1084,17 +1126,27 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k null_map->resize_fill(keys.size(), 1); } - auto client = getClient(); - Strings full_key_paths; full_key_paths.reserve(keys.size()); for (const auto & key : keys) - { full_key_paths.emplace_back(fullPathForKey(key)); - } - auto values = client->tryGet(full_key_paths); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + zkutil::ZooKeeper::MultiTryGetResponse values; + zk_retry.retryLoop([&]{ + auto client = getClient(); + values = client->tryGet(full_key_paths); + }); for (size_t i = 0; i < keys.size(); ++i) { @@ -1182,16 +1234,16 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca if (commands.front().type == MutationCommand::Type::DELETE) { - MutationsInterpreter::Settings settings(true); - settings.return_all_columns = true; - settings.return_mutated_rows = true; + MutationsInterpreter::Settings mutation_settings(true); + mutation_settings.return_all_columns = true; + mutation_settings.return_mutated_rows = true; auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, local_context, - settings); + mutation_settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -1200,8 +1252,6 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca auto primary_key_pos = header.getPositionByName(primary_key); auto version_position = header.getPositionByName(std::string{version_column_name}); - auto client = getClient(); - Block block; while (executor.pull(block)) { @@ -1229,7 +1279,23 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca } Coordination::Responses responses; - auto status = client->tryMulti(delete_requests, responses, /* check_session_valid */ true); + + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; + + Coordination::Error status; + zk_retry.retryLoop([&] + { + auto client = getClient(); + status = client->tryMulti(delete_requests, responses, /* check_session_valid */ true); + }); if (status == Coordination::Error::ZOK) return; @@ -1241,9 +1307,14 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca for (const auto & delete_request : delete_requests) { - auto code = client->tryRemove(delete_request->getPath()); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException::fromPath(code, delete_request->getPath()); + zk_retry.retryLoop([&] + { + auto client = getClient(); + status = client->tryRemove(delete_request->getPath()); + }); + + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNONODE) + throw zkutil::KeeperException::fromPath(status, delete_request->getPath()); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index d4556792c48..cfbb35ab2fe 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -54,7 +54,8 @@ public: Names getPrimaryKey() const override { return {primary_key}; } Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const override; - Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map, bool with_version) const; + Chunk getBySerializedKeys( + std::span keys, PaddedPODArray * null_map, bool with_version, const ContextPtr & local_context) const; Block getSampleBlock(const Names &) const override; diff --git a/tests/integration/test_keeper_map_retries/__init__.py b/tests/integration/test_keeper_map_retries/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml b/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml new file mode 100644 index 00000000000..145945c7c7c --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -0,0 +1,6 @@ + + + 0.05 + 0.05 + + diff --git a/tests/integration/test_keeper_map_retries/test.py b/tests/integration/test_keeper_map_retries/test.py new file mode 100644 index 00000000000..352119147cd --- /dev/null +++ b/tests/integration/test_keeper_map_retries/test.py @@ -0,0 +1,78 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +import os + +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper_map.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +def start_clean_clickhouse(): + # remove fault injection if present + if "fault_injection.xml" in node.exec_in_container( + ["bash", "-c", "ls /etc/clickhouse-server/config.d"] + ): + print("Removing fault injection") + node.exec_in_container( + ["bash", "-c", "rm /etc/clickhouse-server/config.d/fault_injection.xml"] + ) + node.restart_clickhouse() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def repeat_query(query, repeat): + for _ in range(repeat): + node.query( + query, + settings={ + "keeper_max_retries": 20, + "keeper_retry_max_backoff_ms": 10000, + }, + ) + + +def test_queries(started_cluster): + start_clean_clickhouse() + + node.query("DROP TABLE IF EXISTS keeper_map_retries SYNC") + node.query( + "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a" + ) + + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CONFIG_DIR, "fault_injection.xml"), + "/etc/clickhouse-server/config.d/fault_injection.xml", + ) + node.start_clickhouse() + + repeat_count = 10 + + repeat_query( + "INSERT INTO keeper_map_retries SELECT number, number FROM numbers(500)", + repeat_count, + ) + repeat_query("SELECT * FROM keeper_map_retries", repeat_count) + repeat_query( + "ALTER TABLE keeper_map_retries UPDATE b = 3 WHERE a > 2", repeat_count + ) + repeat_query("ALTER TABLE keeper_map_retries DELETE WHERE a > 2", repeat_count) + repeat_query("TRUNCATE keeper_map_retries", repeat_count) diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index ee070b40f6f..c04667505c3 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -13,20 +13,9 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; " -# KeeperMap table engine doesn't have internal retries for interaction with Keeper. Do it on our own, otherwise tests with overloaded server can be flaky. -while true -do - $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000; - " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue - break -done +$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" -while true -do - $CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000; - " 2>&1 | grep -q "KEEPER_EXCEPTION" && sleep 1 && continue - break -done +$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" backup_path="$database_name" for i in $(seq 1 3); do From a1ececb24c0a6f21d3985f5a9f8a726befef78c3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 11:51:48 +0000 Subject: [PATCH 0488/1170] Fix use-of-unitialized-value --- src/Columns/ColumnAggregateFunction.cpp | 33 ++++++++++++++++++++++--- 1 file changed, 30 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 9934970c868..4bc48c62eb4 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -332,9 +332,36 @@ void ColumnAggregateFunction::expand(const Filter & mask, bool inverted) { ensureOwnership(); Arena & arena = createOrGetArena(); - char * default_ptr = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); - func->create(default_ptr); - expandDataByMask(data, mask, inverted, default_ptr); + + if (mask.size() < data.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mask size should be no less than data size."); + + ssize_t from = data.size() - 1; + ssize_t index = mask.size() - 1; + data.resize(mask.size()); + while (index >= 0) + { + if (!!mask[index] ^ inverted) + { + if (from < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too many bytes in mask"); + + /// Copy only if it makes sense. + if (index != from) + data[index] = data[from]; + --from; + } + else + { + data[index] = arena.alignedAlloc(func->sizeOfData(), func->alignOfData()); + func->create(data[index]); + } + + --index; + } + + if (from != -1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough bytes in mask"); } ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const From fd075470d6e141d5aa4d01ccefcc3d3ee04130c7 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 11:54:00 +0000 Subject: [PATCH 0489/1170] Add docs --- tests/integration/README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/integration/README.md b/tests/integration/README.md index cde4cb05aec..d5137a9c148 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -130,6 +130,14 @@ docker build -t clickhouse/integration-test . ``` The helper container used by the `runner` script is in `docker/test/integration/runner/Dockerfile`. +It can be rebuild with + +``` +cd docker/test/integration/runner +docker build -t clickhouse/integration-test-runner . +``` + +Also you need to add option --network=host if you rebuild image for a local integration testsing. ### Adding new tests From fd26672864a7e1557908b878d7daa018de20c61a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 13:54:30 +0200 Subject: [PATCH 0490/1170] Revert some change --- src/Storages/StorageKeeperMap.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 09c21ae28f5..1559b442e43 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -35,7 +35,6 @@ #include #include -#include "Common/ZooKeeper/ZooKeeperRetries.h" #include #include #include @@ -44,6 +43,7 @@ #include #include #include +#include #include #include @@ -205,12 +205,12 @@ public: for (const auto & [key, _] : new_values) key_paths.push_back(storage.fullPathForKey(key)); - zkutil::ZooKeeper::MultiTryGetResponse results; + zkutil::ZooKeeper::MultiExistsResponse results; if constexpr (!for_update) { if (!strict) - results = zookeeper->tryGet(key_paths); + results = zookeeper->exists(key_paths); } Coordination::Requests requests; @@ -231,8 +231,7 @@ public: { if (!strict && results[i].error == Coordination::Error::ZOK) { - if (results[i].data != new_values[key]) - requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); + requests.push_back(zkutil::makeSetRequest(key_paths[i], new_values[key], -1)); } else { @@ -242,9 +241,6 @@ public: } } - if (requests.empty()) - return; - if (new_keys_num != 0) { auto will_be = current_keys_num + new_keys_num; From 0124d211ec81a3779fe0e99c868fe85be8856629 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 14:00:52 +0200 Subject: [PATCH 0491/1170] Better --- tests/integration/test_replicated_table_attach/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 499220def2c..de60b7ec291 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -41,6 +41,7 @@ def start_clean_clickhouse(): def test_startup_with_small_bg_pool(started_cluster): start_clean_clickhouse() + node.query("DROP TABLE IF EXISTS replicated_table SYNC") node.query( "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" ) @@ -54,11 +55,10 @@ def test_startup_with_small_bg_pool(started_cluster): node.restart_clickhouse(stop_start_wait_sec=10) assert_values() - node.query("DROP TABLE replicated_table SYNC") - def test_startup_with_small_bg_pool_partitioned(started_cluster): start_clean_clickhouse() + node.query("DROP TABLE IF EXISTS replicated_table_partitioned SYNC") node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" ) @@ -81,5 +81,3 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): # check that we activate it in the end node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") - - node.query("DROP TABLE replicated_table_partitioned SYNC") From de99ee1b05e68b964535664d4197afd0944d0261 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 30 Jul 2024 12:07:31 +0000 Subject: [PATCH 0492/1170] Change docs --- tests/integration/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index d5137a9c148..c1eb511fa44 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -137,7 +137,7 @@ cd docker/test/integration/runner docker build -t clickhouse/integration-test-runner . ``` -Also you need to add option --network=host if you rebuild image for a local integration testsing. +If your docker configuration doesn't allow access to public internet with docker build command you may also need to add option --network=host if you rebuild image for a local integration testsing. ### Adding new tests From c3e8825c8f14bb82f60c41754a021813d3dbc8aa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 14:23:32 +0200 Subject: [PATCH 0493/1170] Use correct order of fields in StorageURLSource --- src/Storages/StorageURL.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 63d01a02417..6c95cad474c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -228,12 +228,12 @@ private: bool need_only_count; size_t total_rows_in_file = 0; + Poco::Net::HTTPBasicCredentials credentials; + std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; - - Poco::Net::HTTPBasicCredentials credentials; }; class StorageURLSink : public SinkToStorage From 8b52d7b711d54f1d4bb5b2f39bf4aea3966f64dc Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 13:35:19 +0100 Subject: [PATCH 0494/1170] fxs --- src/IO/S3/Client.cpp | 19 +++++++++---------- src/IO/S3/Client.h | 2 +- .../test_checking_s3_blobs_paranoid/test.py | 12 ++++++++++++ tests/integration/test_storage_delta/test.py | 12 ++++++++++++ 4 files changed, 34 insertions(+), 11 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7196dfa9bdc..8f037ea71be 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -385,10 +385,9 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.overrideURI(std::move(*bucket_uri)); - if (isClientForDisk() && error.GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - - return enrichErrorMessage( + /// The next call is NOT a recurcive call + /// This is a virtuall call Aws::S3::S3Client::HeadObject(const Model::HeadObjectRequest&) + return processRequestResult( HeadObject(static_cast(request))); } @@ -409,11 +408,8 @@ Model::ListObjectsOutcome Client::ListObjects(ListObjectsRequest & request) cons Model::GetObjectOutcome Client::GetObject(GetObjectRequest & request) const { - auto resp = doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); }); - if (!resp.IsSuccess() && isClientForDisk() && resp.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); - - return enrichErrorMessage(std::move(resp)); + return processRequestResult( + doRequest(request, [this](const Model::GetObjectRequest & req) { return GetObject(req); })); } Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(AbortMultipartUploadRequest & request) const @@ -699,11 +695,14 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request } template -RequestResult Client::enrichErrorMessage(RequestResult && outcome) const +RequestResult Client::processRequestResult(RequestResult && outcome) const { if (outcome.IsSuccess() || !isClientForDisk()) return std::forward(outcome); + if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + String enriched_message = fmt::format( "{} {}", outcome.GetError().GetMessage(), diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 11cace4e1fd..e54953419e1 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -274,7 +274,7 @@ private: void insertRegionOverride(const std::string & bucket, const std::string & region) const; template - RequestResult enrichErrorMessage(RequestResult && outcome) const; + RequestResult processRequestResult(RequestResult && outcome) const; String initial_endpoint; std::shared_ptr credentials_provider; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 1ed70e20b79..dde636b5d29 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -689,3 +689,15 @@ def test_no_key_found_disk(cluster, broken_s3): "DB::Exception: The specified key does not exist. This error happened for S3 disk." in error ) + + s3_disk_no_key_errors_metric_value = int( + node.query( + """ + SELECT value + FROM system.metrics + WHERE metric = 'S3DiskNoKeyErrors' + """ + ).strip() + ) + + assert s3_disk_no_key_errors_metric_value > 0 diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d3dd7cfe52a..67cc7cdd6da 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -452,6 +452,18 @@ def test_restart_broken(started_cluster): f"SELECT count() FROM {TABLE_NAME}" ) + s3_disk_no_key_errors_metric_value = int( + instance.query( + """ + SELECT value + FROM system.metrics + WHERE metric = 'S3DiskNoKeyErrors' + """ + ).strip() + ) + + assert s3_disk_no_key_errors_metric_value == 0 + minio_client.make_bucket(bucket) upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") From d69f6cccde7633214eba48c08d7647e4ea9a40da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 15:08:26 +0200 Subject: [PATCH 0495/1170] Fix --- src/Common/ThreadPoolTaskTracker.cpp | 5 ++++- src/IO/WriteBufferFromS3.cpp | 9 ++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/ThreadPoolTaskTracker.cpp b/src/Common/ThreadPoolTaskTracker.cpp index 61d34801f7a..1697a13f780 100644 --- a/src/Common/ThreadPoolTaskTracker.cpp +++ b/src/Common/ThreadPoolTaskTracker.cpp @@ -19,6 +19,10 @@ TaskTracker::TaskTracker(ThreadPoolCallbackRunnerUnsafe scheduler_, size_t TaskTracker::~TaskTracker() { + /// Tasks should be waited outside of dtor. + /// Important for WriteBufferFromS3/AzureBlobStorage, where TaskTracker is currently used. + chassert(finished_futures.empty() && futures.empty()); + safeWaitAll(); } @@ -170,4 +174,3 @@ bool TaskTracker::isAsync() const } } - diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 3682e49b018..e702b4d35ad 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -277,12 +277,10 @@ WriteBufferFromS3::~WriteBufferFromS3() "The file might not be written to S3. " "{}.", getVerboseLogDetails()); - return; } - - /// That destructor could be call with finalized=false in case of exceptions - if (!finalized && !canceled) + else if (!finalized) { + /// That destructor could be call with finalized=false in case of exceptions LOG_INFO( log, "WriteBufferFromS3 is not finalized in destructor. " @@ -291,9 +289,10 @@ WriteBufferFromS3::~WriteBufferFromS3() getVerboseLogDetails()); } + /// Wait for all tasks, because they contain reference to this write buffer. task_tracker->safeWaitAll(); - if (!multipart_upload_id.empty() && !multipart_upload_finished) + if (!canceled && !multipart_upload_id.empty() && !multipart_upload_finished) { LOG_WARNING(log, "WriteBufferFromS3 was neither finished nor aborted, try to abort upload in destructor. {}.", getVerboseLogDetails()); tryToAbortMultipartUpload(); From 20faed85ca30c6352fd091e8d4d763fb98fe1311 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 15:18:37 +0200 Subject: [PATCH 0496/1170] Remove useless file --- tests/queries/0_stateless/data_minio/test :: 03215_archive.csv | 1 - 1 file changed, 1 deletion(-) delete mode 100644 tests/queries/0_stateless/data_minio/test :: 03215_archive.csv diff --git a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv b/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/data_minio/test :: 03215_archive.csv +++ /dev/null @@ -1 +0,0 @@ -1 From 21aa514c80cb463f079f9877ae97048a8b13dfbe Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 Jul 2024 15:03:09 +0100 Subject: [PATCH 0497/1170] don't run removed tests --- tests/performance/scripts/entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/scripts/entrypoint.sh b/tests/performance/scripts/entrypoint.sh index 0c3bfa550f4..db7d96ad150 100755 --- a/tests/performance/scripts/entrypoint.sh +++ b/tests/performance/scripts/entrypoint.sh @@ -118,7 +118,7 @@ then # far in the future and have unrelated test changes. base=$(git -C right/ch merge-base pr origin/master) git -C right/ch diff --name-only "$base" pr -- . | tee all-changed-files.txt - git -C right/ch diff --name-only "$base" pr -- tests/performance/*.xml | tee changed-test-definitions.txt + git -C right/ch diff --name-only --diff-filter=d "$base" pr -- tests/performance/*.xml | tee changed-test-definitions.txt git -C right/ch diff --name-only "$base" pr -- :!tests/performance/*.xml :!docker/test/performance-comparison | tee other-changed-files.txt fi From 7d5c30e76cf0fd17515803fec96899f4aad1294e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 16:26:19 +0200 Subject: [PATCH 0498/1170] No retries when partitioned --- tests/integration/test_keeper_map/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 31316af7b1e..7aee5df5746 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -46,7 +46,11 @@ def assert_keeper_exception_after_partition(query): with PartitionManager() as pm: pm.drop_instance_zk_connections(node) try: - error = node.query_and_get_error_with_retry(query, sleep_time=1) + error = node.query_and_get_error_with_retry( + query, + sleep_time=1, + settings={"insert_keeper_max_retries": 1, "keeper_max_retries": 1}, + ) assert "Coordination::Exception" in error except: print_iptables_rules() @@ -84,7 +88,9 @@ def test_keeper_map_without_zk(started_cluster): node.restart_clickhouse(60) try: error = node.query_and_get_error_with_retry( - "SELECT * FROM test_keeper_map_without_zk", sleep_time=1 + "SELECT * FROM test_keeper_map_without_zk", + sleep_time=1, + settings={"keeper_max_retries": 1}, ) assert "Failed to activate table because of connection issues" in error except: From aec431f68bf16b45a6b36deb61c146e08cb4f644 Mon Sep 17 00:00:00 2001 From: heguangnan Date: Tue, 30 Jul 2024 22:54:05 +0800 Subject: [PATCH 0499/1170] fix test --- .../0_stateless/03214_count_distinct_null_key_memory_leak.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql index 847d3742dc3..d8428ec6b4a 100644 --- a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql +++ b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql @@ -12,7 +12,7 @@ PARTITION BY tuple() ORDER BY c SETTINGS index_granularity = 8192, allow_nullable_key=1; -INSERT INTO testnull(b,c) SELECT toString(rand64()) AS b, toString(rand64()) AS c FROM numbers(1000000) +INSERT INTO testnull(b,c) SELECT toString(rand64()) AS b, toString(rand64()) AS c FROM numbers(1000000); SELECT count(distinct b) FROM testnull GROUP BY a SETTINGS max_memory_usage = 54748364; -- {serverError MEMORY_LIMIT_EXCEEDED} DROP TABLE testnull; \ No newline at end of file From 83c6d97cd2aae0a3f79c2776ffc3a5691f8fd4bb Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Tue, 30 Jul 2024 17:34:38 +0200 Subject: [PATCH 0500/1170] squash! added somme tests in relation with https://github.com/ClickHouse/ClickHouse/pull/54881 with new behaviour when enable_named_columns_in_function_tuple=1 (default value) --- .../0_stateless/00309_formats.reference | Bin 20353 -> 18666 bytes tests/queries/0_stateless/00309_formats.sql | 5 +++++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index cab311692be229716b58af39079275d3942b01cc..f3ea45520bb50fb936caf6724e9fedf3cdd00b75 100644 GIT binary patch delta 3052 zcmeIy$4}c(00wZIgtksdZ-w^I3JD}eL2?j6{R4XG1$t-?y;Q0c4wb5?2YL>Bj|8*# z-Xp>6y?59U_TGDMNaN%a?SDX3#j^bK`<|bk?ZfAlN^UDiNE{JVHgaD5e8+19DxExLIy^` zXcz-yVH}Ky2`~{R!DN^MQ}xOSA`L-0%z&9N3ueO{m<#h@J}iKRum~2z5?IQq+n-P( zP{A_D!g5#vD`6F^hBdGj*1>w%02^VGPHumqnY-wY76U{pY=iBv19rkL*bRGNFYJT; zPz?v*AoeGQ5Ddc+I10z$IGlhQI0>iVG@OC6a1PGH1;h6f#3F(vxC~d|DqMr>a070_ zEw~ML;4a*Q`|v>j|8;{y1V``~p1@Oh2G8LIyo6Wq8s5NLcn9yHg%&qjk~HD*O`kk{ zMhS#wBF4@26;n)ok$HM~`}q3#2LuKMhlGZON5~bCOjLACY+QUoVp4KSYFc_mW>$7i zZeD&tVNr2OsZv$OmRD3(RoB$k)i*RYHMg|3wRd!Ob@%l4^{WR4hlWQ+$Hpf#lT*_( zvvczci%ZKZt842Un_JsEyL2uP zs~KC(*lPbltFhlLRQ$(oSyIc$t;|{F>ph;XVgHfpC$+5H%8ajnp%+Y<&DY2OORU_k z(V7&oZmv$Gj7LgIiK)l0pFiA9KYx&VyyL(B=HYJsi3@C{mioxSQew%acGgGsx>HI? tDE$HfH|yd|adXt)D_oo(mI%Z&HeM!>@7a6Ht;zJ#Jk0d+*gt^JbdYKDu4%*X9d$a^ZOYUl#FBHF2yr}q<;@67HiYtm^#Z|>iir*-% zDXuF{6fY}&tN5Mb_lj2(e^C5U@h8Qria#snd+xpO{s$g>=;22meeCfko}78A|MWA@J~z8>|MLgJ!58LIidkNiRoxsq zJUB8u`r@(UFP(V#6D$6wrgio0oJnBji=r;A-&v0K-- z%ZqMZ-|En<>suYVb$zQtx30$|*-G(uTzhnP-g&=^d;Asr39~@AuK#;~vDmn-A16x* zR`tnxBCGmdyM|AAwVs$&ebTP#ldV?ulqNAv(}O7;BueJfKp}=K;v>=KamanSU6UWrDN?_JXVk8gMI`72?`P<#P@i7&<`pS zWF+WF5R#xIK}v#_1ThI}667T4Nf4BvC_z$!rUX$5suE-+=t>Zlpe#XJg0=*43F;E$ zCFn~Kn4mC0VuHp5kqIgjWG3iL5SpMgL28251hENf6XYi7O%R-*I6-oP<^<6RsuN@< z=uQxxpgci(g7yUQ3F;H%C+JUb0Ko$U7Z7|va00;#1UC@;KyU=X69iWfd_iyq!5aj3 z5d48~#s?7ye?<5r!XFX-i10^*KO+1Q;g1M^M8t7~KO+1Q;g1M^MEE1Z9})hD9M2rk z64B_#G0j2EQO#k^am|4Ze?<5r!XFX-i0}tWF!-P!{1M@g2!BNQBf=jM{)q5Lgg+wu z5#f&re?<5r!XFX-i10^*KO+1Q;g1M^#G){NzzgHs7A`ETZCepq_933?i~U2rWWJXqI8KuOcyVRaKQz-H_k$cyu3VDiwye}g+f_bcJ1@D;TzuN5 zD@s+avnf(-W3}#V+EUgwouv-u47MnGiFGeI6M7J0X>pSftOBKaoh3fnm@GvczL`Ta z^L_N{6Fw0i&Ve~T1wJJ{6+SgSO&=q%!s~pgh07?nk!+)EGuh^{Eo58Dw&Hr*^k(>Z zybQ>^Hqtrc{32`%3@;;YAm1kG-$5QXD0j)XNqnD@?1}G`_*RMUmH1|f@0J>DqOcTU z*dnqtku4%yM7D@*5!oWLMaC8xTV$doTV!mJu|>uf8Czs*k+DV27CBqwY>|unY>~4? z&K5aasMw-ni;68Owy4;mVvC9`Dz>QDqGpSlEo!!?*`j8Pnk{O!sM(@si<&JOwrJR* zVT*<>8n$TIqG5}MEgH6H*a8(t*|H6)jlxDpGfP)fUuE8&1TrL@Sn5)r6Ya*MU7eL0wb!Wn76AR5ztX z#+6_}jZ<1=T!|M{I=Mw62Gvg4mvJR%Q1g@)8CT*4l}~AraV2z6|Kt`49@Ic(U&fXA zK_ygLWLya$)I+&Nf(SKH*_Ux8j!+qu78zGU3H4EFk#QxOP$lISi6>M_WnadXkV3sw zT4Y>_DpXCSg~!FbC|e#E)l=EBEh?z8Wm{BHWy|BDk}6xVZKpC4TZt>wRQasL73!+8 zFSZg_sIE$j*tQdl#8s5ILYJI}Nejl>nIuG~}N3YAyc7h8!d)L*4V z#+A526;@hgT!|~xV!1`)3iVjom;05tLRD5;5#VP`KA{>G~DrX+|J4NE4^tt78W7My1Ewbk!9(?&_}d~o*M=)&sS=v zm5YR#+gx;@nz8VMIrYYX7g?<00RC^)91khQIPJmG@OrgrFt^k;LYv_Lo5d@`!DxB3 zGDLsD_K3_;G)I{^%FR(>j!JV>nWNeqHPfRx<;2)BT_0Y1Xf&A9h!Y`)0|nvDH@x8n zHr{gEJg2=CS+6-2Y@=}a^V*~wFA#5d<+f6RtRKlbl5c8*WMo}Q)|X_RN!FWW-AUK{ zDc6tAH={hZdv7)Rc=}3mQ`+9<(5@$S>={hVObTHRbg2bmQ`U{6_yqMxorr-!m=tXtHQD> zEvwS9DlMzhvMMdB(y}TotJ1P6EvwS9sw}I@vZ^er%Cf2~tID#fEUU`0sw}I@vZ^hs z+On!GtJ<=vEvwqHsx7P9vZ^hs+OldatH!cwEUU({YAmb9vT7`=#}TLyo>A3TXJ|u*PXZY@b0cX?~)z)dEQU-ex~gv+M8t>k(P&*cKVkL~(9{aiZ0vklwdB?P?7?an*> zTu#9I-mbsX&!q*_8f<@;81VeWop<`V+<^DVU4N&aOAmPG-1T?*xeS5#(rtg2Bk;7w zop<`VG=X>8U4N&a%M^I;-Sv0+xnzOoK(@b27kFphop<`VjDh#+U4N&aOB#6h-u8EC z1J9b=d8ePt9C&Zv^>_NY_NY{DCT;>+ke)`2)`|ZGV?P@Y@r2-syKefBb6wVB0s=_`E`L+6m)Ezc_5xV{`a(e6A7cfwpXiGp`$XFl3xq_$>F>d{|a@uQ~ Lvprlx%YOd>&)TR8 diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index b0939c00a10..0366cdeea5c 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -9,3 +9,8 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSON; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT JSONCompact; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT XML; + +SET enable_named_columns_in_function_tuple = 1; + +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; From a70cdb8bba5503f3723a2e29957617ea06106c4d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 30 Jul 2024 15:37:59 +0000 Subject: [PATCH 0501/1170] Add comment regarding default value for force_connected --- src/Client/ConnectionEstablisher.h | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 304ec4d34b4..ff071e59aea 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -24,8 +24,12 @@ public: const QualifiedTableName * table_to_check = nullptr); /// Establish connection and save it in result, write possible exception message in fail_message. - /// The connection is returned from the pool, it can be stale. Use force_connected flag - /// to ensure that connection is working one + /// The connection is returned from connection pool and it can be stale. Use force_connected flag to ensure that connection is working one. + /// NOTE: force_connected is false by default due to the following consideration ... + /// When true, it implies sending a Ping packet to another peer and, if it fails - reestablishing the connection. + /// Ping-Pong round trip can be unnecessary in case of connection is still alive. + /// So, the optimistic approach is used by default. In this case, stale connections can be handled by retrying, + /// - see ConnectionPoolWithFailover, as example void run(TryResult & result, std::string & fail_message, bool force_connected = false); /// Set async callback that will be called when reading from socket blocks. From aa26291ff25b16539efa3d50a540fc32a05b702d Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 18:15:24 +0100 Subject: [PATCH 0502/1170] fxs --- src/Databases/DatabaseReplicated.cpp | 57 +++++++++++++++---- src/Databases/DatabaseReplicatedWorker.cpp | 21 +++++-- src/Databases/DatabaseReplicatedWorker.h | 4 +- src/Storages/System/StorageSystemClusters.cpp | 6 +- .../test_recovery_time_metric/test.py | 36 ++++++++++-- 5 files changed, 98 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b11b9382732..06cea65d62e 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -340,31 +341,63 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const { - ReplicasInfo res; + Strings paths_get, paths_exists; + + paths_get.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); - auto zookeeper = getZooKeeper(); 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) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); + paths_exists.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); + } + } + try { - UInt32 max_log_ptr = parse(zookeeper->get(zookeeper_path + "/max_log_ptr")); + auto current_zookeeper = getZooKeeper(); + auto get_res = current_zookeeper->get(paths_get); + auto exist_res = current_zookeeper->exists(paths_exists); + chassert(get_res.size() == exist_res.size() + 1); + auto max_log_ptr_zk = get_res[0]; + if (max_log_ptr_zk.error != Coordination::Error::ZOK) + throw Coordination::Exception(max_log_ptr_zk.error); + + UInt32 max_log_ptr = parse(max_log_ptr_zk.data); + + ReplicasInfo replicas_info; + replicas_info.resize(exist_res.size()); + + 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]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - UInt32 log_ptr = parse(zookeeper->get(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr")); - bool is_active = zookeeper->exists(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - res.push_back(ReplicaInfo{ - .is_active = is_active, - .replication_lag = max_log_ptr - log_ptr, + auto replica_active = exist_res[global_replica_index]; + auto replica_log_ptr = get_res[global_replica_index + 1]; + + if (replica_active.error != Coordination::Error::ZOK && replica_active.error != Coordination::Error::ZNONODE) + throw Coordination::Exception(replica_active.error); + + if (replica_log_ptr.error != Coordination::Error::ZOK) + throw Coordination::Exception(replica_log_ptr.error); + + replicas_info[global_replica_index] = ReplicaInfo{ + .is_active = replica_active.error == Coordination::Error::ZOK, + .replication_lag = max_log_ptr - parse(replica_log_ptr.data), .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, - }); + }; + + ++global_replica_index; } } - return res; + + return replicas_info; } catch (...) { tryLogCurrentException(log); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index a9a74c5f56a..4e7408aa96e 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,8 +32,11 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { - initialization_duration_timer.restart(); - initializing.store(true, std::memory_order_release); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.emplace(); + initialization_duration_timer->start(); + } while (!stop_flag) { @@ -72,7 +75,10 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - initializing.store(false, std::memory_order_relaxed); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } return true; } catch (...) @@ -82,7 +88,11 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - initializing.store(false, std::memory_order_relaxed); + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } + return false; } @@ -466,7 +476,8 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const { - return initializing.load(std::memory_order_acquire) ? initialization_duration_timer.elapsedMilliseconds() : 0; + 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 3e5887be825..2309c831839 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -59,8 +59,8 @@ private: /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; - Stopwatch initialization_duration_timer; - std::atomic initializing = false; + 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 0da4bd70cbd..4b9802c9089 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -71,7 +71,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); - size_t replica_idx = 0; + size_t global_replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { const auto & shard_info = shards_info[shard_index]; @@ -108,7 +108,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam } else { - const auto & replica_info = replicas_info[replica_idx++]; + const auto & replica_info = replicas_info[global_replica_idx]; res_columns[i++]->insert(replica_info.is_active); res_columns[i++]->insert(replica_info.replication_lag); if (replica_info.recovery_time != 0) @@ -116,6 +116,8 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam else res_columns[i++]->insertDefault(); } + + ++global_replica_idx; } } } diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 4dad844b950..8f369d7759c 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -5,7 +5,6 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.xml"], - with_zookeeper=True, stay_alive=True, ) @@ -21,9 +20,36 @@ def start_cluster(): def test_recovery_time_metric(start_cluster): node.query( - "CREATE DATABASE rdb ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1');" + """ + CREATE DATABASE rdb + ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') + """ ) - node.query("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.query( + """ + 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() - assert node.query("SELECT any(recovery_time) FROM system.clusters;") != "0\n" + + ret = int( + node.query( + """ + SELECT recovery_time + FROM system.clusters + WHERE cluster = 'rdb' + """ + ).strip() + ) + assert ret > 0 From 7e51e9962c34320a9c60ba6abcf8b38cf517e86c Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Tue, 30 Jul 2024 17:20:07 +0000 Subject: [PATCH 0503/1170] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22ff9b7194f..a8110500f13 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -341,15 +341,19 @@ void MergeTreeDeduplicationLog::shutdown() stopped = true; if (current_writer) { + /// If an error has occurred during finalize, we'd like to have the exception set for reset. + /// Otherwise, we'll be in a situation when a finalization didn't happen, and we didn't get + /// any error, causing logical error (see ~MemoryBuffer()). try { current_writer->finalize(); + current_writer.reset(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + current_writer.reset(); } - current_writer.reset(); } } From 51af0d305c9959fb4870bd8a57035d48207648b2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 30 Jul 2024 20:00:23 +0200 Subject: [PATCH 0504/1170] Reduce number of tested combinations --- tests/queries/0_stateless/02473_multistep_prewhere.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index a942568233c..11095202039 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -193,7 +193,7 @@ def main(): url = os.environ["CLICKHOUSE_URL"] + "&max_threads=1" default_index_granularity = 10 - total_rows = 8 * default_index_granularity + total_rows = 7 * default_index_granularity step = default_index_granularity session = requests.Session() for index_granularity in [ From bd83ba88b03b7de15fbc7530d6c827156674f3b5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Jul 2024 18:48:34 +0000 Subject: [PATCH 0505/1170] 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 dacf044c3dee65d799242b7f4846f7d6d8b2bd34 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 30 Jul 2024 19:20:52 +0000 Subject: [PATCH 0506/1170] Update version_date.tsv and changelogs after v24.7.1.2915-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.7.1.2915-stable.md | 524 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 530 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.7.1.2915-stable.md diff --git a/SECURITY.md b/SECURITY.md index 53328b6e16b..8930dc96f8a 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,9 +14,10 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.7 | ✔️ | | 24.6 | ✔️ | | 24.5 | ✔️ | -| 24.4 | ✔️ | +| 24.4 | ❌ | | 24.3 | ✔️ | | 24.2 | ❌ | | 24.1 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index c59ef1b919a..e99c86267f9 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.6.2.17" +ARG VERSION="24.7.1.2915" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 240df79aeb1..fb562b911a3 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.6.2.17" +ARG VERSION="24.7.1.2915" 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 ac64655991a..51f4e6a0f40 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.6.2.17" +ARG VERSION="24.7.1.2915" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.7.1.2915-stable.md b/docs/changelogs/v24.7.1.2915-stable.md new file mode 100644 index 00000000000..abffbe58bfc --- /dev/null +++ b/docs/changelogs/v24.7.1.2915-stable.md @@ -0,0 +1,524 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.1.2915-stable (a37d2d43da7) FIXME as compared to v24.7.1.1-new (aa023477a92) + +#### Backward Incompatible Change +* Change binary serialization of Variant data type: add `compact` mode to avoid writing the same discriminator multiple times for granules with single variant or with only NULL values. Add MergeTree setting `use_compact_variant_discriminators_serialization` that is enabled by default. Note that Variant type is still experimental and backward-incompatible change in serialization is ok. [#62774](https://github.com/ClickHouse/ClickHouse/pull/62774) ([Kruglov Pavel](https://github.com/Avogar)). +* Forbid `CREATE MATERIALIZED VIEW ... ENGINE Replicated*MergeTree POPULATE AS SELECT ...` with Replicated databases. [#63963](https://github.com/ClickHouse/ClickHouse/pull/63963) ([vdimir](https://github.com/vdimir)). +* `clickhouse-keeper-client` will only accept paths in string literals, such as `ls '/hello/world'`, not bare strings such as `ls /hello/world`. [#65494](https://github.com/ClickHouse/ClickHouse/pull/65494) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Metric `KeeperOutstandingRequets` was renamed to `KeeperOutstandingRequests`. This fixes a typo reported in [#66179](https://github.com/ClickHouse/ClickHouse/issues/66179). [#66206](https://github.com/ClickHouse/ClickHouse/pull/66206) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Extend function `tuple` to construct named tuples in query. Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). +* `ASOF JOIN` support for `full_sorting_join` algorithm Close [#54493](https://github.com/ClickHouse/ClickHouse/issues/54493). [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). +* A new table function, `fuzzQuery,` was added. This function allows you to modify a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1');`. [#62103](https://github.com/ClickHouse/ClickHouse/pull/62103) ([pufit](https://github.com/pufit)). +* Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). +* Support JWT authentication in `clickhouse-client`. [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). +* Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). +* Support accept_invalid_certificate in client's config in order to allow for client to connect over secure TCP to a server running with self-signed certificate - can be used as a shorthand for corresponding `openSSL` client settings `verificationMode=none` + `invalidCertificateHandler.name=AcceptCertificateHandler`. [#65238](https://github.com/ClickHouse/ClickHouse/pull/65238) ([peacewalker122](https://github.com/peacewalker122)). +* Add system.error_log which contains history of error values from table system.errors, periodically flushed to disk. [#65381](https://github.com/ClickHouse/ClickHouse/pull/65381) ([Pablo Marcos](https://github.com/pamarcos)). +* Add aggregate function `groupConcat`. About the same as `arrayStringConcat( groupArray(column), ',')` Can receive 2 parameters: a string delimiter and the number of elements to be processed. [#65451](https://github.com/ClickHouse/ClickHouse/pull/65451) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Add AzureQueue storage. [#65458](https://github.com/ClickHouse/ClickHouse/pull/65458) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a new setting to disable/enable writing page index into parquet files. [#65475](https://github.com/ClickHouse/ClickHouse/pull/65475) ([lgbo](https://github.com/lgbo-ustc)). +* Allow system administrators to configure `logger.console_log_level`. [#65559](https://github.com/ClickHouse/ClickHouse/pull/65559) ([Azat Khuzhin](https://github.com/azat)). +* Automatically append a wildcard `*` to the end of a directory path with table function `file`. [#66019](https://github.com/ClickHouse/ClickHouse/pull/66019) ([Zhidong (David) Guo](https://github.com/Gun9niR)). +* Add `--memory-usage` option to client in non interactive mode. [#66393](https://github.com/ClickHouse/ClickHouse/pull/66393) ([vdimir](https://github.com/vdimir)). + +#### Performance Improvement +* Enable `optimize_functions_to_subcolumns` by default. [#58661](https://github.com/ClickHouse/ClickHouse/pull/58661) ([Anton Popov](https://github.com/CurtizJ)). +* Replace int to string algorithm with a faster one (from a modified amdn/itoa to a modified jeaiii/itoa). [#61661](https://github.com/ClickHouse/ClickHouse/pull/61661) ([Raúl Marín](https://github.com/Algunenano)). +* Sizes of hash tables created by join (`parallel_hash` algorithm) is collected and cached now. This information will be used to preallocate space in hash tables for subsequent query executions and save time on hash table resizes. [#64553](https://github.com/ClickHouse/ClickHouse/pull/64553) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized queries with `ORDER BY` primary key and `WHERE` that have a condition with high selectivity by using of buffering. It is controlled by setting `read_in_order_use_buffering` (enabled by default) and can increase memory usage of query. [#64607](https://github.com/ClickHouse/ClickHouse/pull/64607) ([Anton Popov](https://github.com/CurtizJ)). +* Improve performance of loading `plain_rewritable` metadata. [#65634](https://github.com/ClickHouse/ClickHouse/pull/65634) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attaching tables on read-only disks will use fewer resources by not loading outdated parts. [#65635](https://github.com/ClickHouse/ClickHouse/pull/65635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support minmax hyperrectangle for Set indices. [#65676](https://github.com/ClickHouse/ClickHouse/pull/65676) ([AntiTopQuark](https://github.com/AntiTopQuark)). +* Unload primary index of outdated parts to reduce total memory usage. [#65852](https://github.com/ClickHouse/ClickHouse/pull/65852) ([Anton Popov](https://github.com/CurtizJ)). +* Functions `replaceRegexpAll` and `replaceRegexpOne` are now significantly faster if the pattern is trivial, i.e. contains no metacharacters, pattern classes, flags, grouping characters etc. (Thanks to Taiyang Li). [#66185](https://github.com/ClickHouse/ClickHouse/pull/66185) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Improvement +* Support rocksdb as backend storage of keeper. [#56626](https://github.com/ClickHouse/ClickHouse/pull/56626) ([Han Fei](https://github.com/hanfei1991)). +* The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). +* The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). +* This PR changes how deduplication for MV works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). +* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). +* In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Make an interactive client for clickhouse-disks, add local disk from the local directory. Fixes [#56791](https://github.com/ClickHouse/ClickHouse/issues/56791). [#64446](https://github.com/ClickHouse/ClickHouse/pull/64446) ([Daniil Ivanik](https://github.com/divanik)). +* Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). +* `StorageS3Queue` related fixes and improvements. Deduce a default value of `s3queue_processing_threads_num` according to the number of physical cpu cores on the server (instead of the previous default value as 1). Set default value of `s3queue_loading_retries` to 10. Fix possible vague "Uncaught exception" in exception column of `system.s3queue`. Do not increment retry count on `MEMORY_LIMIT_EXCEEDED` exception. Move files commit to a stage after insertion into table fully finished to avoid files being commited while not inserted. Add settings `s3queue_max_processed_files_before_commit`, `s3queue_max_processed_rows_before_commit`, `s3queue_max_processed_bytes_before_commit`, `s3queue_max_processing_time_sec_before_commit`, to better control commit and flush time. [#65046](https://github.com/ClickHouse/ClickHouse/pull/65046) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed broken multiple columns aggregation on s390x. [#65062](https://github.com/ClickHouse/ClickHouse/pull/65062) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Support aliases in parametrized view function (only new analyzer). [#65190](https://github.com/ClickHouse/ClickHouse/pull/65190) ([Kseniia Sumarokova](https://github.com/kssenii)). +* S3. reduce retires time for queries, increase retries count for backups. 8.5 minutes and 100 retires for queries, 1.2 hours and 1000 retries for backup restore. [#65232](https://github.com/ClickHouse/ClickHouse/pull/65232) ([Sema Checherinda](https://github.com/CheSema)). +* Updated to mask account key in logs in azureBlobStorage. [#65273](https://github.com/ClickHouse/ClickHouse/pull/65273) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Partition pruning for `IN` predicates when filter expression is a part of `PARTITION BY` expression. [#65335](https://github.com/ClickHouse/ClickHouse/pull/65335) ([Eduard Karacharov](https://github.com/korowa)). +* Add system tables with main information about all detached tables. [#65400](https://github.com/ClickHouse/ClickHouse/pull/65400) ([Konstantin Morozov](https://github.com/k-morozov)). +* Add support for `cluster_for_parallel_replicas` when using custom key parallel replicas. It allows you to use parallel replicas with custom key with MergeTree tables. [#65453](https://github.com/ClickHouse/ClickHouse/pull/65453) ([Antonio Andelic](https://github.com/antonio2368)). +* Support query plan LIMIT optimization. Support LIMIT pushdown for PostgreSQL storage and table function. [#65454](https://github.com/ClickHouse/ClickHouse/pull/65454) ([Maksim Kita](https://github.com/kitaisreal)). +* Arraymin/max can be applicable to all data types that are comparable. [#65455](https://github.com/ClickHouse/ClickHouse/pull/65455) ([pn](https://github.com/chloro-pn)). +* Improved memory accounting for cgroups v2 to exclude the amount occupied by the page cache. [#65470](https://github.com/ClickHouse/ClickHouse/pull/65470) ([Nikita Taranov](https://github.com/nickitat)). +* Do not create format settings for each rows when serializing chunks to insert to EmbeddedRocksDB table. [#65474](https://github.com/ClickHouse/ClickHouse/pull/65474) ([Duc Canh Le](https://github.com/canhld94)). +* Fixed out-of-range exception in parsing Dwarf5 on s390x. [#65501](https://github.com/ClickHouse/ClickHouse/pull/65501) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Reduce `clickhouse-local` prompt to just `:)`. `getFQDNOrHostName()` takes too long on macOS, and we don't want a hostname in the prompt for `clickhouse-local` anyway. [#65510](https://github.com/ClickHouse/ClickHouse/pull/65510) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Avoid printing a message from jemalloc about per-CPU arenas on low-end virtual machines. [#65532](https://github.com/ClickHouse/ClickHouse/pull/65532) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable filesystem cache background download by default. It will be enabled back when we fix the issue with possible "Memory limit exceeded" because memory deallocation is done outside of query context (while buffer is allocated inside of query context) if we use background download threads. Plus we need to add a separate setting to define max size to download for background workers (currently it is limited by max_file_segment_size, which might be too big). [#65534](https://github.com/ClickHouse/ClickHouse/pull/65534) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new option to config `` which allow to specify how often clickhouse will reload config. [#65545](https://github.com/ClickHouse/ClickHouse/pull/65545) ([alesapin](https://github.com/alesapin)). +* Implement binary encoding for ClickHouse data types and add its specification in docs. Use it in Dynamic binary serialization, allow to use it in RowBinaryWithNamesAndTypes and Native formats under settings. [#65546](https://github.com/ClickHouse/ClickHouse/pull/65546) ([Kruglov Pavel](https://github.com/Avogar)). +* Improved ZooKeeper load balancing. The current session doesn't expire until the optimal nodes become available despite `fallback_session_lifetime`. Added support for AZ-aware balancing. [#65570](https://github.com/ClickHouse/ClickHouse/pull/65570) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Server settings `compiled_expression_cache_size` and `compiled_expression_cache_elements_size` are now shown in `system.server_settings`. [#65584](https://github.com/ClickHouse/ClickHouse/pull/65584) ([Robert Schulze](https://github.com/rschu1ze)). +* When lightweight delete happens on a table with projection(s), users have choices either throw an exception (by default) or drop the projection lightweight delete would happen. [#65594](https://github.com/ClickHouse/ClickHouse/pull/65594) ([jsc0218](https://github.com/jsc0218)). +* Add support for user identification based on x509 SubjectAltName extension. [#65626](https://github.com/ClickHouse/ClickHouse/pull/65626) ([Anton Kozlov](https://github.com/tonickkozlov)). +* `clickhouse-local` will respect the `max_server_memory_usage` and `max_server_memory_usage_to_ram_ratio` from the configuration file. It will also set the max memory usage to 90% of the system memory by default, like `clickhouse-server` does. This closes [#65695](https://github.com/ClickHouse/ClickHouse/issues/65695). [#65697](https://github.com/ClickHouse/ClickHouse/pull/65697) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a script to backup your files to ClickHouse. This is strange, but works. [#65699](https://github.com/ClickHouse/ClickHouse/pull/65699) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* PostgreSQL source support cancel. [#65722](https://github.com/ClickHouse/ClickHouse/pull/65722) ([Maksim Kita](https://github.com/kitaisreal)). +* Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow to use `concat` function with empty arguments ``` sql :) select concat();. [#65887](https://github.com/ClickHouse/ClickHouse/pull/65887) ([李扬](https://github.com/taiyang-li)). +* Allow controlling named collections in clickhouse-local. [#65973](https://github.com/ClickHouse/ClickHouse/pull/65973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve Azure profile events. [#65999](https://github.com/ClickHouse/ClickHouse/pull/65999) ([alesapin](https://github.com/alesapin)). +* `Query was cancelled` might have been printed twice in clickhouse-client. This behaviour is fixed. [#66005](https://github.com/ClickHouse/ClickHouse/pull/66005) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support ORC file read by writer time zone. [#66025](https://github.com/ClickHouse/ClickHouse/pull/66025) ([kevinyhzou](https://github.com/KevinyhZou)). +* Refactor JSONExtract functions, support more types including experimental Dynamic type. [#66046](https://github.com/ClickHouse/ClickHouse/pull/66046) ([Kruglov Pavel](https://github.com/Avogar)). +* DatabaseCatalog drops tables faster by using up to database_catalog_drop_table_concurrency threads. [#66065](https://github.com/ClickHouse/ClickHouse/pull/66065) ([Sema Checherinda](https://github.com/CheSema)). +* This PR changes how deduplication for MV works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#66144](https://github.com/ClickHouse/ClickHouse/pull/66144) ([Sema Checherinda](https://github.com/CheSema)). +* Support null map subcolumn for Variant and Dynamic subcolumns. [#66178](https://github.com/ClickHouse/ClickHouse/pull/66178) ([Kruglov Pavel](https://github.com/Avogar)). +* Add settings to control connection to the PostgreSQL. * Setting `postgresql_connection_attempt_timeout` specifies the value passed to `connect_timeout` parameter of connection URL. * Setting `postgresql_connection_pool_retries` specifies the number of retries to establish a connection to the PostgreSQL end-point. [#66232](https://github.com/ClickHouse/ClickHouse/pull/66232) ([Dmitry Novik](https://github.com/novikd)). +* Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us. [#66239](https://github.com/ClickHouse/ClickHouse/pull/66239) ([Azat Khuzhin](https://github.com/azat)). +* Improve FilesystemCache ProfileEvents. [#66249](https://github.com/ClickHouse/ClickHouse/pull/66249) ([zhukai](https://github.com/nauu)). +* Add settings to ignore ON CLUSTER clause in queries for named collection management with replicated storage. [#66288](https://github.com/ClickHouse/ClickHouse/pull/66288) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Upgraded `pocketfft` dependency to the recent commit https://github.com/mreineck/pocketfft/commit/f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3. [#66291](https://github.com/ClickHouse/ClickHouse/pull/66291) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Upgraded `azure-sdk-for-cpp` to the recent commit https://github.com/ClickHouse/azure-sdk-for-cpp/commit/ea3e19a7be08519134c643177d56c7484dfec884. [#66292](https://github.com/ClickHouse/ClickHouse/pull/66292) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Function `generateSnowflakeID` now allows to specify a machine ID as a parameter to prevent collisions in large clusters. [#66374](https://github.com/ClickHouse/ClickHouse/pull/66374) ([ZAWA_ll](https://github.com/Zawa-ll)). +* Disable suspending on Ctrl+Z in interactive mode. This is a common trap and is not expected behavior for almost all users. I imagine only a few extreme power users could appreciate suspending terminal applications to the background, but I don't know any. [#66511](https://github.com/ClickHouse/ClickHouse/pull/66511) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add option for validating the Primary key type in Dictionaries. Without this option for simple layouts any column type will be implicitly converted to UInt64. ### Documentation entry for user-facing changes. [#66595](https://github.com/ClickHouse/ClickHouse/pull/66595) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix SHOW MERGES remaining time. [#66735](https://github.com/ClickHouse/ClickHouse/pull/66735) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Closes [#65355](https://github.com/ClickHouse/ClickHouse/issues/65355). Also fix some bugs in dependencies creation. [#65405](https://github.com/ClickHouse/ClickHouse/pull/65405) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fixed crash while using MaterializedMySQL with TABLE OVERRIDE that maps MySQL NULL field into ClickHouse not NULL field. [#54649](https://github.com/ClickHouse/ClickHouse/pull/54649) ([Filipp Ozinov](https://github.com/bakwc)). +* Fix logical error when PREWHERE expression read no columns and table has no adaptive index granularity (very old table). Fix [#56640](https://github.com/ClickHouse/ClickHouse/issues/56640). [#59173](https://github.com/ClickHouse/ClickHouse/pull/59173) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug with cancelation buffer when canceling a query. [#64478](https://github.com/ClickHouse/ClickHouse/pull/64478) ([Sema Checherinda](https://github.com/CheSema)). +* Fix filling parts columns from metadata (when columns.txt does not exists). [#64757](https://github.com/ClickHouse/ClickHouse/pull/64757) ([Azat Khuzhin](https://github.com/azat)). +* Fix AST formatting of 'create table b empty as a'. [#64951](https://github.com/ClickHouse/ClickHouse/pull/64951) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Fix unexpected projection name when query with CTE. [#65267](https://github.com/ClickHouse/ClickHouse/pull/65267) ([wudidapaopao](https://github.com/wudidapaopao)). +* Require `dictGet` privilege when accessing dictionaries via direct query or the `Dictionary` table engine. [#65359](https://github.com/ClickHouse/ClickHouse/pull/65359) ([Joe Lynch](https://github.com/joelynch)). +* Fix user-specific S3 auth with incremental backups. [#65481](https://github.com/ClickHouse/ClickHouse/pull/65481) ([Antonio Andelic](https://github.com/antonio2368)). +* Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix getting exception `Index out of bound for blob metadata` in case all files from list batch were filtered out. [#65523](https://github.com/ClickHouse/ClickHouse/pull/65523) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix NOT_FOUND_COLUMN_IN_BLOCK for deduplicate merge of projection. [#65573](https://github.com/ClickHouse/ClickHouse/pull/65573) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed a bug that compatibility level '23.4' was not properly applied. [#65737](https://github.com/ClickHouse/ClickHouse/pull/65737) ([cw5121](https://github.com/cw5121)). +* Fix odbc table with nullable fields. [#65738](https://github.com/ClickHouse/ClickHouse/pull/65738) ([Rodolphe Dugé de Bernonville](https://github.com/RodolpheDuge)). +* Fix data race in `TCPHandler`, which could happen on fatal error. [#65744](https://github.com/ClickHouse/ClickHouse/pull/65744) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Fix a bug leads to EmbeddedRocksDB with TTL write corrupted SST files. [#65816](https://github.com/ClickHouse/ClickHouse/pull/65816) ([Duc Canh Le](https://github.com/canhld94)). +* Functions `bitTest`, `bitTestAll`, and `bitTestAny` now return an error if the specified bit index is out-of-bounds (issue [#65517](https://github.com/ClickHouse/ClickHouse/issues/65517)). [#65818](https://github.com/ClickHouse/ClickHouse/pull/65818) ([Pablo Marcos](https://github.com/pamarcos)). +* Setting `join_any_take_last_row` is supported in any query with hash join. [#65820](https://github.com/ClickHouse/ClickHouse/pull/65820) ([vdimir](https://github.com/vdimir)). +* Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Functions `bitShiftLeft` and `bitShitfRight` return an error for out of bounds shift positions (issue [#65516](https://github.com/ClickHouse/ClickHouse/issues/65516)). [#65838](https://github.com/ClickHouse/ClickHouse/pull/65838) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix growing memory usage in S3Queue. [#65839](https://github.com/ClickHouse/ClickHouse/pull/65839) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix tie handling in `arrayAUC` to match sklearn. [#65840](https://github.com/ClickHouse/ClickHouse/pull/65840) ([gabrielmcg44](https://github.com/gabrielmcg44)). +* Fix possible issues with MySQL server protocol TLS connections. [#65917](https://github.com/ClickHouse/ClickHouse/pull/65917) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible issues with MySQL client protocol TLS connections. [#65938](https://github.com/ClickHouse/ClickHouse/pull/65938) ([Azat Khuzhin](https://github.com/azat)). +* Fix handling of `SSL_ERROR_WANT_READ`/`SSL_ERROR_WANT_WRITE` with zero timeout. [#65941](https://github.com/ClickHouse/ClickHouse/pull/65941) ([Azat Khuzhin](https://github.com/azat)). +* Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Column _size in s3 engine and s3 table function denotes the size of a file inside the archive, not a size of the archive itself. [#65993](https://github.com/ClickHouse/ClickHouse/pull/65993) ([Daniil Ivanik](https://github.com/divanik)). +* Fix resolving dynamic subcolumns in analyzer, avoid reading the whole column on dynamic subcolumn reading. [#66004](https://github.com/ClickHouse/ClickHouse/pull/66004) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix config merging for from_env with replace overrides. [#66034](https://github.com/ClickHouse/ClickHouse/pull/66034) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible hanging in `GRPCServer` during shutdown. This PR fixes [#65622](https://github.com/ClickHouse/ClickHouse/issues/65622). [#66061](https://github.com/ClickHouse/ClickHouse/pull/66061) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix reading dynamic subcolumns from altered Memory table. Previously if `max_types` peremeter of a Dynamic type was changed in Memory table via alter, further subcolumns reading can return wrong result. [#66066](https://github.com/ClickHouse/ClickHouse/pull/66066) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixed several cases in function `has` with non-constant `LowCardinality` arguments. [#66088](https://github.com/ClickHouse/ClickHouse/pull/66088) ([Anton Popov](https://github.com/CurtizJ)). +* Fix for `groupArrayIntersect`. It had incorrect behavior in the `merge()` function. Also, fixed behavior in `deserialise()` for numeric and general data. [#66103](https://github.com/ClickHouse/ClickHouse/pull/66103) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the issue when the server failed to parse Avro files with negative block size arrays encoded, which is now allowed by the Avro specification. [#66130](https://github.com/ClickHouse/ClickHouse/pull/66130) ([Serge Klochkov](https://github.com/slvrtrn)). +* Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Fix rare case with missing data in the result of distributed query, close [#61432](https://github.com/ClickHouse/ClickHouse/issues/61432). [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Fix order of parsing metadata fields in StorageDeltaLake. [#66211](https://github.com/ClickHouse/ClickHouse/pull/66211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error reporting while copying to S3 or AzureBlobStorage. [#66295](https://github.com/ClickHouse/ClickHouse/pull/66295) ([Vitaly Baranov](https://github.com/vitlibar)). +* Prevent watchdog from keeping descriptors of unlinked(rotated) log files. [#66334](https://github.com/ClickHouse/ClickHouse/pull/66334) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix the bug that logicalexpressionoptimizerpass lost logical type of constant. closes [#64487](https://github.com/ClickHouse/ClickHouse/issues/64487). [#66344](https://github.com/ClickHouse/ClickHouse/pull/66344) ([pn](https://github.com/chloro-pn)). +* Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too aggressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. [#66402](https://github.com/ClickHouse/ClickHouse/pull/66402) ([vdimir](https://github.com/vdimir)). +* Added missing column materialization for cross join. [#66413](https://github.com/ClickHouse/ClickHouse/pull/66413) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid possible logical error during import from Npy format in case of bad array nesting level, fix testing of other kinds of errors. [#66461](https://github.com/ClickHouse/ClickHouse/pull/66461) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix wrong count() result when there is non-deterministic function in predicate. [#66510](https://github.com/ClickHouse/ClickHouse/pull/66510) ([Duc Canh Le](https://github.com/canhld94)). +* Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading of uninitialized memory when hashing empty tuples. This closes [#66559](https://github.com/ClickHouse/ClickHouse/issues/66559). [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix logical error in `PrometheusRequestHandler`. [#66621](https://github.com/ClickHouse/ClickHouse/pull/66621) ([Vitaly Baranov](https://github.com/vitlibar)). +* `column_length` is not updated in `ColumnTuple::insertManyFrom`. [#66626](https://github.com/ClickHouse/ClickHouse/pull/66626) ([lgbo](https://github.com/lgbo-ustc)). +* Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix rare case of stuck merge after drop column. [#66707](https://github.com/ClickHouse/ClickHouse/pull/66707) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assertion `isUniqTypes` when insert select from remote sources. [#66722](https://github.com/ClickHouse/ClickHouse/pull/66722) ([Sema Checherinda](https://github.com/CheSema)). +* Backported in [#67026](https://github.com/ClickHouse/ClickHouse/issues/67026): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67443](https://github.com/ClickHouse/ClickHouse/issues/67443): Forbid create as select even when database_replicated_allow_heavy_create is set. It was unconditionally forbidden in 23.12 and accidentally allowed under the setting in unreleased 24.7. [#66980](https://github.com/ClickHouse/ClickHouse/pull/66980) ([vdimir](https://github.com/vdimir)). +* Backported in [#67201](https://github.com/ClickHouse/ClickHouse/issues/67201): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67383](https://github.com/ClickHouse/ClickHouse/issues/67383): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67246](https://github.com/ClickHouse/ClickHouse/issues/67246): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Build/Testing/Packaging Improvement +* Instantiate template methods ahead in different .cpp files, avoid too large translation units during compiling. [#64818](https://github.com/ClickHouse/ClickHouse/pull/64818) ([lgbo](https://github.com/lgbo-ustc)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs. [#65186](https://github.com/ClickHouse/ClickHouse/pull/65186) ([Nikita Fomichev](https://github.com/fm4v)). +* Add tests for `base64URLEncode` and `base64URLDecode`. Add analyzer tests. [#65979](https://github.com/ClickHouse/ClickHouse/pull/65979) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix problem when github terminate instances by timeout and artifacts are not collected and full test report is not generated. [#66036](https://github.com/ClickHouse/ClickHouse/pull/66036) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix test [test_grpc_protocol/test.py::test_progress](https://s3.amazonaws.com/clickhouse-test-reports/57695/188f8a3df74caf830ad1ced3c4cf6dfb0aa90093/integration_tests__asan__old_analyzer__[4_6].html). [#66063](https://github.com/ClickHouse/ClickHouse/pull/66063) ([Vitaly Baranov](https://github.com/vitlibar)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs. [#66305](https://github.com/ClickHouse/ClickHouse/pull/66305) ([Nikita Fomichev](https://github.com/fm4v)). +* Stateless tests: Improve tests speed and decrease number of parallel jobs 3. [#66363](https://github.com/ClickHouse/ClickHouse/pull/66363) ([Nikita Fomichev](https://github.com/fm4v)). +* Tests: fix tests hang up in cases when gdb catches error. [#66411](https://github.com/ClickHouse/ClickHouse/pull/66411) ([Nikita Fomichev](https://github.com/fm4v)). +* ... since [Release v24.6.1.4423-stable](https://github.com/ClickHouse/ClickHouse/releases/tag/v24.6.1.4423-stable) when build in ppc64le with dynamic openssl build (`cmake -DENABLE_OPENSSL_DYNAMIC=1 -DCMAKE_TOOLCHAIN_FILE= cmake/linux/toolchain-ppc64le.cmake `) got error: ` ld.lld: error: duplicate symbol: OPENSSL_cleanse`. [#66733](https://github.com/ClickHouse/ClickHouse/pull/66733) ([Yong Wang](https://github.com/kashwy)). + +#### NO CL CATEGORY + +* Backported in [#67084](https://github.com/ClickHouse/ClickHouse/issues/67084):. [#67040](https://github.com/ClickHouse/ClickHouse/pull/67040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67452](https://github.com/ClickHouse/ClickHouse/issues/67452):. [#67392](https://github.com/ClickHouse/ClickHouse/pull/67392) ([alesapin](https://github.com/alesapin)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Revert "Small fix for 02340_parts_refcnt_mergetree""'. [#65155](https://github.com/ClickHouse/ClickHouse/pull/65155) ([Nikita Taranov](https://github.com/nickitat)). +* NO CL ENTRY: 'Revert "Use 1MB HTTP buffers to avoid frequent send syscalls"'. [#65498](https://github.com/ClickHouse/ClickHouse/pull/65498) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Resubmit http_external_tables_memory_tracking test"'. [#65500](https://github.com/ClickHouse/ClickHouse/pull/65500) ([Nikita Taranov](https://github.com/nickitat)). +* NO CL ENTRY: 'Revert "Add an assertion in ReplicatedMergeTreeQueue"'. [#65686](https://github.com/ClickHouse/ClickHouse/pull/65686) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "insertion deduplication on retries for materialised views"'. [#66134](https://github.com/ClickHouse/ClickHouse/pull/66134) ([Sema Checherinda](https://github.com/CheSema)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Return and fix 01600_parts_states_metrics_long test. [#58748](https://github.com/ClickHouse/ClickHouse/pull/58748) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add azure_cache as storage policy to tests. [#59943](https://github.com/ClickHouse/ClickHouse/pull/59943) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Minor: replaced expression with LEGACY_MAX_LEVEL. [#61268](https://github.com/ClickHouse/ClickHouse/pull/61268) ([Vasily Nemkov](https://github.com/Enmk)). +* Make write to temporary data in cache do all checks and assertions as during write to ordinary cache. [#63348](https://github.com/ClickHouse/ClickHouse/pull/63348) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Refactoring near azure blob storage. [#63636](https://github.com/ClickHouse/ClickHouse/pull/63636) ([Anton Popov](https://github.com/CurtizJ)). +* Everything should work with Analyzer. [#63643](https://github.com/ClickHouse/ClickHouse/pull/63643) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* add some log for non using detached tables. [#64992](https://github.com/ClickHouse/ClickHouse/pull/64992) ([Konstantin Morozov](https://github.com/k-morozov)). +* Remove dag flags. [#65234](https://github.com/ClickHouse/ClickHouse/pull/65234) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix flaky autocompletion test. [#65246](https://github.com/ClickHouse/ClickHouse/pull/65246) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Disable userspace page cache by default. [#65305](https://github.com/ClickHouse/ClickHouse/pull/65305) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v24.4.3.25-stable. [#65308](https://github.com/ClickHouse/ClickHouse/pull/65308) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Do not raise a NOT_IMPLEMENTED error when getting s3 metrics with a multiple disk configuration. [#65403](https://github.com/ClickHouse/ClickHouse/pull/65403) ([Elena Torró](https://github.com/elenatorro)). +* Dodging reading from wrong table with parallel replicas. [#65417](https://github.com/ClickHouse/ClickHouse/pull/65417) ([Nikita Taranov](https://github.com/nickitat)). +* Fix: return error if can't connect to any replicas chosen for query execution. [#65467](https://github.com/ClickHouse/ClickHouse/pull/65467) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `AzureObjectStorage::exists` method. [#65471](https://github.com/ClickHouse/ClickHouse/pull/65471) ([Anton Popov](https://github.com/CurtizJ)). +* Update version after release. [#65483](https://github.com/ClickHouse/ClickHouse/pull/65483) ([Raúl Marín](https://github.com/Algunenano)). +* Generate 24.6 changelog. [#65485](https://github.com/ClickHouse/ClickHouse/pull/65485) ([Raúl Marín](https://github.com/Algunenano)). +* Fix of `PlanSquashingTransform`: pipeline stuck. [#65487](https://github.com/ClickHouse/ClickHouse/pull/65487) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix bad test `02922_deduplication_with_zero_copy`. [#65492](https://github.com/ClickHouse/ClickHouse/pull/65492) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable clang-format in special areas. [#65495](https://github.com/ClickHouse/ClickHouse/pull/65495) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `test_keeper_snapshots`. [#65497](https://github.com/ClickHouse/ClickHouse/pull/65497) ([Antonio Andelic](https://github.com/antonio2368)). +* Update to libunwind 8.1.7. [#65509](https://github.com/ClickHouse/ClickHouse/pull/65509) ([Michael Kolupaev](https://github.com/al13n321)). +* Setting `uniform_snowflake_conversion_functions` (not in any release yet) was replaced by setting `allow_deprecated_snowflake_conversion_functions`. The latter controls if the legacy snowflake conversion functions are available (by default, they are not). [#65522](https://github.com/ClickHouse/ClickHouse/pull/65522) ([Robert Schulze](https://github.com/rschu1ze)). +* Try CI without RerunCheck, jobs can be easily rerun manually though extra amount of work in CI will follow on workflow restart. [#65524](https://github.com/ClickHouse/ClickHouse/pull/65524) ([Max K.](https://github.com/maxknv)). +* Bump re2 to latest HEAD. [#65526](https://github.com/ClickHouse/ClickHouse/pull/65526) ([Robert Schulze](https://github.com/rschu1ze)). +* OpenSSL: Replace temporary fix for unsynchronized access by official fix. [#65529](https://github.com/ClickHouse/ClickHouse/pull/65529) ([Robert Schulze](https://github.com/rschu1ze)). +* Update README.md. [#65531](https://github.com/ClickHouse/ClickHouse/pull/65531) ([Tyler Hannan](https://github.com/tylerhannan)). +* CI: some time there are timeouts on DROP TABLES for random tests. [#65535](https://github.com/ClickHouse/ClickHouse/pull/65535) ([Sema Checherinda](https://github.com/CheSema)). +* Synchronize `MARK_CACHE_SIZE` value in default settings and config. [#65547](https://github.com/ClickHouse/ClickHouse/pull/65547) ([Denny Crane](https://github.com/den-crane)). +* CI: Skip removed test files in stateless flaky check job. [#65553](https://github.com/ClickHouse/ClickHouse/pull/65553) ([Max K.](https://github.com/maxknv)). +* Renames Build report jobs. [#65554](https://github.com/ClickHouse/ClickHouse/pull/65554) ([Max K.](https://github.com/maxknv)). +* Parse user from URL for dashboard.html (useful for sharing). [#65556](https://github.com/ClickHouse/ClickHouse/pull/65556) ([Azat Khuzhin](https://github.com/azat)). +* Remove tech debt. [#65561](https://github.com/ClickHouse/ClickHouse/pull/65561) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Maybe fix test `00763_lock_buffer_long.sh`. [#65562](https://github.com/ClickHouse/ClickHouse/pull/65562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix clickhouse-keeper with not system-wide directories and provide override for local development. [#65563](https://github.com/ClickHouse/ClickHouse/pull/65563) ([Azat Khuzhin](https://github.com/azat)). +* Re-configure yamllint to allow document-start. [#65565](https://github.com/ClickHouse/ClickHouse/pull/65565) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky test `01254_dict_load_after_detach_attach.sql`. [#65571](https://github.com/ClickHouse/ClickHouse/pull/65571) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve flaky test to provide more diagnostics. [#65586](https://github.com/ClickHouse/ClickHouse/pull/65586) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_parallel_replicas_distributed_skip_shards flakiness. [#65588](https://github.com/ClickHouse/ClickHouse/pull/65588) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix an error in the test about metadata_type. [#65592](https://github.com/ClickHouse/ClickHouse/pull/65592) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix slow test. [#65593](https://github.com/ClickHouse/ClickHouse/pull/65593) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fix flaky 02864_statistics_uniq. [#65599](https://github.com/ClickHouse/ClickHouse/pull/65599) ([Han Fei](https://github.com/hanfei1991)). +* Fix 03172_error_log_table_not_empty. [#65604](https://github.com/ClickHouse/ClickHouse/pull/65604) ([Pablo Marcos](https://github.com/pamarcos)). +* Enable realtime digest for Jepsen tests. [#65608](https://github.com/ClickHouse/ClickHouse/pull/65608) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: Return Job Rerun check. [#65613](https://github.com/ClickHouse/ClickHouse/pull/65613) ([Max K.](https://github.com/maxknv)). +* Update CHANGELOG.md. [#65624](https://github.com/ClickHouse/ClickHouse/pull/65624) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Module is required for authenticating in GH (in cloud). [#65628](https://github.com/ClickHouse/ClickHouse/pull/65628) ([Max K.](https://github.com/maxknv)). +* Update IObjectStorage.h. [#65631](https://github.com/ClickHouse/ClickHouse/pull/65631) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix 02834_apache_arrow_abort flakiness with MSAN. [#65640](https://github.com/ClickHouse/ClickHouse/pull/65640) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix overflow in StorageWindowView. [#65641](https://github.com/ClickHouse/ClickHouse/pull/65641) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix inconsistent AST formatting when a keyword is used as type name. [#65648](https://github.com/ClickHouse/ClickHouse/pull/65648) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Single point of setting mergeable check status. [#65658](https://github.com/ClickHouse/ClickHouse/pull/65658) ([Max K.](https://github.com/maxknv)). +* Miscellaneous and insignificant changes around Client/ClientBase. [#65669](https://github.com/ClickHouse/ClickHouse/pull/65669) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add Replicated database names to ZooKeeper for introspection. [#65675](https://github.com/ClickHouse/ClickHouse/pull/65675) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Introduce type contract checks in `IColumn`. [#65687](https://github.com/ClickHouse/ClickHouse/pull/65687) ([Nikita Taranov](https://github.com/nickitat)). +* Print slightly more information in 02982_aggregation_states_destruction. [#65688](https://github.com/ClickHouse/ClickHouse/pull/65688) ([Michael Kolupaev](https://github.com/al13n321)). +* Disable stacktrace collection in GWPAsan by default. [#65701](https://github.com/ClickHouse/ClickHouse/pull/65701) ([Antonio Andelic](https://github.com/antonio2368)). +* Build jemalloc with profiler. [#65702](https://github.com/ClickHouse/ClickHouse/pull/65702) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix duplicate symbol linkage error. [#65705](https://github.com/ClickHouse/ClickHouse/pull/65705) ([Nikita Taranov](https://github.com/nickitat)). +* Fix server restarts in performance tests. [#65717](https://github.com/ClickHouse/ClickHouse/pull/65717) ([Antonio Andelic](https://github.com/antonio2368)). +* Update 03002_part_log_rmt_fetch_mutate_error.sql. [#65720](https://github.com/ClickHouse/ClickHouse/pull/65720) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky `02265_column_ttl`. Closes [#65719](https://github.com/ClickHouse/ClickHouse/issues/65719). [#65742](https://github.com/ClickHouse/ClickHouse/pull/65742) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* See [#65745](https://github.com/ClickHouse/ClickHouse/issues/65745). It doesn't solve the issue, but helps a bit. [#65746](https://github.com/ClickHouse/ClickHouse/pull/65746) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update CHANGELOG.md. [#65752](https://github.com/ClickHouse/ClickHouse/pull/65752) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* document declarative ssh-keys authentication. [#65756](https://github.com/ClickHouse/ClickHouse/pull/65756) ([Tobias Florek](https://github.com/ibotty)). +* `base64En/Decode64Url` --> `base64En/Decode64URL`. [#65760](https://github.com/ClickHouse/ClickHouse/pull/65760) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix for issue [#65757](https://github.com/ClickHouse/ClickHouse/issues/65757). [#65763](https://github.com/ClickHouse/ClickHouse/pull/65763) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix flaky `test_replicated_database::test_alter_attach`. [#65766](https://github.com/ClickHouse/ClickHouse/pull/65766) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix: progress bar for read in order queries. [#65769](https://github.com/ClickHouse/ClickHouse/pull/65769) ([Igor Nikonov](https://github.com/devcrafter)). +* CI: Fix for Builds report job in backports and releases. [#65774](https://github.com/ClickHouse/ClickHouse/pull/65774) ([Max K.](https://github.com/maxknv)). +* CI: New create release workflow. [#65775](https://github.com/ClickHouse/ClickHouse/pull/65775) ([Max K.](https://github.com/maxknv)). +* fixed misspelled word. [#65778](https://github.com/ClickHouse/ClickHouse/pull/65778) ([Linh Giang](https://github.com/linhgiang24)). +* Refactor statistics interface. [#65792](https://github.com/ClickHouse/ClickHouse/pull/65792) ([Robert Schulze](https://github.com/rschu1ze)). +* Try to make `test_ldap_external_user_directory` less flaky. [#65794](https://github.com/ClickHouse/ClickHouse/pull/65794) ([Andrey Zvonov](https://github.com/zvonand)). +* AMI image with gh and jwt. [#65795](https://github.com/ClickHouse/ClickHouse/pull/65795) ([Max K.](https://github.com/maxknv)). +* Forbid join algorithm randomisation for 03094_one_thousand_joins. [#65798](https://github.com/ClickHouse/ClickHouse/pull/65798) ([Nikita Taranov](https://github.com/nickitat)). +* Fix 02931_rewrite_sum_column_and_constant flakiness. [#65800](https://github.com/ClickHouse/ClickHouse/pull/65800) ([Michael Kolupaev](https://github.com/al13n321)). +* Update StorageMaterializedView.cpp. [#65801](https://github.com/ClickHouse/ClickHouse/pull/65801) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix slow `getFQDNOrHostNameImpl` on macOS. [#65803](https://github.com/ClickHouse/ClickHouse/pull/65803) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* No jemalloc profiler for non-Linux. [#65834](https://github.com/ClickHouse/ClickHouse/pull/65834) ([Antonio Andelic](https://github.com/antonio2368)). +* Add missing workload identity changes. [#65848](https://github.com/ClickHouse/ClickHouse/pull/65848) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix rocksdb. [#65858](https://github.com/ClickHouse/ClickHouse/pull/65858) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update the list of easy tasks. [#65865](https://github.com/ClickHouse/ClickHouse/pull/65865) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#65866](https://github.com/ClickHouse/ClickHouse/pull/65866) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#43003](https://github.com/ClickHouse/ClickHouse/issues/43003). [#65870](https://github.com/ClickHouse/ClickHouse/pull/65870) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Uninteresting changes. [#65871](https://github.com/ClickHouse/ClickHouse/pull/65871) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Max sessions for user tests improvements. [#65888](https://github.com/ClickHouse/ClickHouse/pull/65888) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Update version_date.tsv and changelogs after v24.6.1.4423-stable. [#65909](https://github.com/ClickHouse/ClickHouse/pull/65909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Remove standalone Keeper build. [#65910](https://github.com/ClickHouse/ClickHouse/pull/65910) ([Antonio Andelic](https://github.com/antonio2368)). +* Add extra profiling helpers for Keeper. [#65918](https://github.com/ClickHouse/ClickHouse/pull/65918) ([Antonio Andelic](https://github.com/antonio2368)). +* PostgreSQL source cancel query comments. [#65919](https://github.com/ClickHouse/ClickHouse/pull/65919) ([Maksim Kita](https://github.com/kitaisreal)). +* Remove mysqlxx::Pool::Entry assignment operator. [#65920](https://github.com/ClickHouse/ClickHouse/pull/65920) ([Azat Khuzhin](https://github.com/azat)). +* No random settings for a test with `Object(JSON)`. [#65921](https://github.com/ClickHouse/ClickHouse/pull/65921) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Follow up to [#65046](https://github.com/ClickHouse/ClickHouse/issues/65046). [#65928](https://github.com/ClickHouse/ClickHouse/pull/65928) ([Kseniia Sumarokova](https://github.com/kssenii)). +* add restriction for storage join. [#65936](https://github.com/ClickHouse/ClickHouse/pull/65936) ([Han Fei](https://github.com/hanfei1991)). +* Update version_date.tsv and changelogs after v24.5.4.49-stable. [#65937](https://github.com/ClickHouse/ClickHouse/pull/65937) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Add table name to MergeTreeSource spans. [#65940](https://github.com/ClickHouse/ClickHouse/pull/65940) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SettingsChangesHistory 24.7. [#65945](https://github.com/ClickHouse/ClickHouse/pull/65945) ([Raúl Marín](https://github.com/Algunenano)). +* Fix logical error "Expected ReadBufferFromFile, but got DB::EmptyReadBuffer". [#65949](https://github.com/ClickHouse/ClickHouse/pull/65949) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use -Og instead of -O0 for debug builds. [#65953](https://github.com/ClickHouse/ClickHouse/pull/65953) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix data race for Keeper snapshot queue. [#65970](https://github.com/ClickHouse/ClickHouse/pull/65970) ([Antonio Andelic](https://github.com/antonio2368)). +* Minor changes in CHANGELOG. [#65971](https://github.com/ClickHouse/ClickHouse/pull/65971) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove unnatural punctuation from Parquet. [#65972](https://github.com/ClickHouse/ClickHouse/pull/65972) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try fix "Check timeout expired" without any server logs in report in stateless tests. [#65977](https://github.com/ClickHouse/ClickHouse/pull/65977) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix support of non-const scale arguments in rounding functions. [#65983](https://github.com/ClickHouse/ClickHouse/pull/65983) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* More aesthetic error messages. [#65985](https://github.com/ClickHouse/ClickHouse/pull/65985) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix race in s3queue. [#65986](https://github.com/ClickHouse/ClickHouse/pull/65986) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Now it's possible to specify `s3-storage`, `azure-object-storage` and in general `object-storage`. [#65988](https://github.com/ClickHouse/ClickHouse/pull/65988) ([alesapin](https://github.com/alesapin)). +* Fix flaky test_storage_s3_queue tests. [#66009](https://github.com/ClickHouse/ClickHouse/pull/66009) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Relax the check in 02982_aggregation_states_destruction. [#66011](https://github.com/ClickHouse/ClickHouse/pull/66011) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `01158_zookeeper_log_long`. [#66012](https://github.com/ClickHouse/ClickHouse/pull/66012) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove scary jemalloc log. [#66028](https://github.com/ClickHouse/ClickHouse/pull/66028) ([Antonio Andelic](https://github.com/antonio2368)). +* Move experimental settings to the experimental block. [#66030](https://github.com/ClickHouse/ClickHouse/pull/66030) ([Raúl Marín](https://github.com/Algunenano)). +* Fix lock-order-inversion in DatabaseCatalog. [#66038](https://github.com/ClickHouse/ClickHouse/pull/66038) ([Nikolay Degterinsky](https://github.com/evillique)). +* Try disabling jemalloc background threads. [#66041](https://github.com/ClickHouse/ClickHouse/pull/66041) ([Antonio Andelic](https://github.com/antonio2368)). +* Try to avoid conflicts in `SettingsChangesHistory.cpp`. [#66042](https://github.com/ClickHouse/ClickHouse/pull/66042) ([Anton Popov](https://github.com/CurtizJ)). +* Add profile events for regex cache. [#66050](https://github.com/ClickHouse/ClickHouse/pull/66050) ([Antonio Andelic](https://github.com/antonio2368)). +* Bump vectorscan to 5.4.10.1. [#66056](https://github.com/ClickHouse/ClickHouse/pull/66056) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove obsolete comment. [#66059](https://github.com/ClickHouse/ClickHouse/pull/66059) ([Robert Schulze](https://github.com/rschu1ze)). +* Maybe fix tsan assert in `test_mysql_killed_while_insert_8_0`. [#66064](https://github.com/ClickHouse/ClickHouse/pull/66064) ([Robert Schulze](https://github.com/rschu1ze)). +* Move some of `HTTPHandler` stuff to separate source files in order to reuse it in `PrometheusRequestHandler`. This PR is required for https://github.com/ClickHouse/ClickHouse/pull/64183. [#66067](https://github.com/ClickHouse/ClickHouse/pull/66067) ([Vitaly Baranov](https://github.com/vitlibar)). +* Bump rocksdb to v6.23.3. [#66068](https://github.com/ClickHouse/ClickHouse/pull/66068) ([Robert Schulze](https://github.com/rschu1ze)). +* Add protobufs for `Prometheus` `remote-write` / `remote-read` protocols to our repository. Fix cmake script for compiling protobufs. [#66069](https://github.com/ClickHouse/ClickHouse/pull/66069) ([Vitaly Baranov](https://github.com/vitlibar)). +* Use pinned versions of all python packages in CI docker images. Also makes clang-18.1.8 work with sanitizers and surprisingly fixes [#66049](https://github.com/ClickHouse/ClickHouse/issues/66049). [#66070](https://github.com/ClickHouse/ClickHouse/pull/66070) ([alesapin](https://github.com/alesapin)). +* Clean-up custom LLVM 15 patches. [#66072](https://github.com/ClickHouse/ClickHouse/pull/66072) ([Robert Schulze](https://github.com/rschu1ze)). +* Minor JWT client fixes. [#66073](https://github.com/ClickHouse/ClickHouse/pull/66073) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Bump vectorscan to 5.4.11. [#66082](https://github.com/ClickHouse/ClickHouse/pull/66082) ([Robert Schulze](https://github.com/rschu1ze)). +* Print stacktrace in case of abort after logical error. [#66091](https://github.com/ClickHouse/ClickHouse/pull/66091) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix flaky 03172_error_log_table_not_empty. [#66093](https://github.com/ClickHouse/ClickHouse/pull/66093) ([Sema Checherinda](https://github.com/CheSema)). +* Bump s2geometry to latest master. [#66094](https://github.com/ClickHouse/ClickHouse/pull/66094) ([Robert Schulze](https://github.com/rschu1ze)). +* update keeper bench example config file. [#66095](https://github.com/ClickHouse/ClickHouse/pull/66095) ([Han Fei](https://github.com/hanfei1991)). +* Avoid using source directory for generated files. [#66097](https://github.com/ClickHouse/ClickHouse/pull/66097) ([Azat Khuzhin](https://github.com/azat)). +* More precise warning message about sanitizers. [#66098](https://github.com/ClickHouse/ClickHouse/pull/66098) ([Anton Popov](https://github.com/CurtizJ)). +* Slightly better calculation of primary index. [#66099](https://github.com/ClickHouse/ClickHouse/pull/66099) ([Anton Popov](https://github.com/CurtizJ)). +* Bump Azure to 1.12. [#66100](https://github.com/ClickHouse/ClickHouse/pull/66100) ([Robert Schulze](https://github.com/rschu1ze)). +* Add a test for [#58998](https://github.com/ClickHouse/ClickHouse/issues/58998). [#66101](https://github.com/ClickHouse/ClickHouse/pull/66101) ([Anton Popov](https://github.com/CurtizJ)). +* CI: Fix sync pr merge. [#66105](https://github.com/ClickHouse/ClickHouse/pull/66105) ([Max K.](https://github.com/maxknv)). +* Remove flaky case from 02956_rocksdb_bulk_sink. [#66107](https://github.com/ClickHouse/ClickHouse/pull/66107) ([vdimir](https://github.com/vdimir)). +* Fix bugfix checker. [#66120](https://github.com/ClickHouse/ClickHouse/pull/66120) ([Raúl Marín](https://github.com/Algunenano)). +* Correctly print long processing requests in Keeper. [#66124](https://github.com/ClickHouse/ClickHouse/pull/66124) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v24.6.2.17-stable. [#66127](https://github.com/ClickHouse/ClickHouse/pull/66127) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Bump s2geometry again. [#66136](https://github.com/ClickHouse/ClickHouse/pull/66136) ([Robert Schulze](https://github.com/rschu1ze)). +* Switch submodule `contrib/orc` to a proper commit in the [main](https://github.com/ClickHouse/orc/tree/main) branch. Previously a commit from a removed branch was used ([see](https://github.com/ClickHouse/orc/pull/13)). [#66137](https://github.com/ClickHouse/ClickHouse/pull/66137) ([Vitaly Baranov](https://github.com/vitlibar)). +* Finalize MergedBlockOutputStream in dtor. [#66138](https://github.com/ClickHouse/ClickHouse/pull/66138) ([Nikita Taranov](https://github.com/nickitat)). +* Proper destruction order of AsyncLoader::Pool fields. [#66145](https://github.com/ClickHouse/ClickHouse/pull/66145) ([Sergei Trifonov](https://github.com/serxa)). +* Playing minesweeper with build system. [#66147](https://github.com/ClickHouse/ClickHouse/pull/66147) ([Nikita Taranov](https://github.com/nickitat)). +* Fix clang-tidy error in BufferWithOwnMemory.h. [#66161](https://github.com/ClickHouse/ClickHouse/pull/66161) ([Nikita Taranov](https://github.com/nickitat)). +* Use peak_threads_usage instead of arrayUniq(thread_ids) in tests. [#66162](https://github.com/ClickHouse/ClickHouse/pull/66162) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash when adding empty tuple to query cache. [#66168](https://github.com/ClickHouse/ClickHouse/pull/66168) ([Michael Kolupaev](https://github.com/al13n321)). +* tests: fix 01563_distributed_query_finish flakiness (due to system.*_log_sender). [#66171](https://github.com/ClickHouse/ClickHouse/pull/66171) ([Azat Khuzhin](https://github.com/azat)). +* Refactor `OptimizeIfWithConstantConditionVisitor` using `InDepthNodeVisitor`. [#66184](https://github.com/ClickHouse/ClickHouse/pull/66184) ([zhongyuankai](https://github.com/zhongyuankai)). +* Update README.md. [#66186](https://github.com/ClickHouse/ClickHouse/pull/66186) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix 01246_buffer_flush flakiness. [#66188](https://github.com/ClickHouse/ClickHouse/pull/66188) ([Azat Khuzhin](https://github.com/azat)). +* Avoid using harmful function `rand()` in grpc. [#66191](https://github.com/ClickHouse/ClickHouse/pull/66191) ([Vitaly Baranov](https://github.com/vitlibar)). +* Bump RocksDB. [#66216](https://github.com/ClickHouse/ClickHouse/pull/66216) ([Robert Schulze](https://github.com/rschu1ze)). +* Update README.md. [#66217](https://github.com/ClickHouse/ClickHouse/pull/66217) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fixes peak_threads_usage metric when materialised views are involved. [#66230](https://github.com/ClickHouse/ClickHouse/pull/66230) ([Sema Checherinda](https://github.com/CheSema)). +* Remove test as requested in https://github.com/ClickHouse/ClickHouse/pull/65277#issuecomment-2211361465. [#66233](https://github.com/ClickHouse/ClickHouse/pull/66233) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix test `00504_mergetree_arrays_rw.sql`. [#66248](https://github.com/ClickHouse/ClickHouse/pull/66248) ([Anton Popov](https://github.com/CurtizJ)). +* CI: Do not finalize CI running status unless all success. [#66276](https://github.com/ClickHouse/ClickHouse/pull/66276) ([Max K.](https://github.com/maxknv)). +* Collect core dumps in more tests. [#66281](https://github.com/ClickHouse/ClickHouse/pull/66281) ([Antonio Andelic](https://github.com/antonio2368)). +* Add a stateless test for gRPC protocol. [#66284](https://github.com/ClickHouse/ClickHouse/pull/66284) ([Vitaly Baranov](https://github.com/vitlibar)). +* Log message: Failed to connect to replica ... [#66289](https://github.com/ClickHouse/ClickHouse/pull/66289) ([Igor Nikonov](https://github.com/devcrafter)). +* Update run.sh. [#66290](https://github.com/ClickHouse/ClickHouse/pull/66290) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Some changes in the codebase as a preparation for LLVM 18. [#66293](https://github.com/ClickHouse/ClickHouse/pull/66293) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* There's some problem with iptables in parallel tests. [#66304](https://github.com/ClickHouse/ClickHouse/pull/66304) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Build failure if ENABLE_AWS_S3 is OFF fixed. [#66335](https://github.com/ClickHouse/ClickHouse/pull/66335) ([Ilya Golshtein](https://github.com/ilejn)). +* Enable checks in assert_cast under sanitizers. [#66336](https://github.com/ClickHouse/ClickHouse/pull/66336) ([Nikita Taranov](https://github.com/nickitat)). +* Create release workflow. [#66339](https://github.com/ClickHouse/ClickHouse/pull/66339) ([Max K.](https://github.com/maxknv)). +* Fix invalid XML. [#66342](https://github.com/ClickHouse/ClickHouse/pull/66342) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix possible deadlock for jemalloc with enabled profiler. [#66346](https://github.com/ClickHouse/ClickHouse/pull/66346) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_parallel_replicas_custom_key. [#66349](https://github.com/ClickHouse/ClickHouse/pull/66349) ([Antonio Andelic](https://github.com/antonio2368)). +* Collect logs from `minio` in stateless and statefull tests. [#66353](https://github.com/ClickHouse/ClickHouse/pull/66353) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix "Sending a batch of X files to Y (0.00 rows, 0.00 B bytes)." in case of batch restoring. [#66375](https://github.com/ClickHouse/ClickHouse/pull/66375) ([Azat Khuzhin](https://github.com/azat)). +* Fix 03030_system_flush_distributed_settings flakiness. [#66376](https://github.com/ClickHouse/ClickHouse/pull/66376) ([Azat Khuzhin](https://github.com/azat)). +* PR cleanup: remove redundant code. [#66380](https://github.com/ClickHouse/ClickHouse/pull/66380) ([Igor Nikonov](https://github.com/devcrafter)). +* New slack bot to post messages about CI events - Post message if OOM. [#66392](https://github.com/ClickHouse/ClickHouse/pull/66392) ([Max K.](https://github.com/maxknv)). +* Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_rabbitmq/test.py. [#66396](https://github.com/ClickHouse/ClickHouse/pull/66396) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add query elapsed time for non-default format in play UI. [#66398](https://github.com/ClickHouse/ClickHouse/pull/66398) ([Azat Khuzhin](https://github.com/azat)). +* Untangle setting headers. [#66404](https://github.com/ClickHouse/ClickHouse/pull/66404) ([Raúl Marín](https://github.com/Algunenano)). +* Remove noisy message. [#66406](https://github.com/ClickHouse/ClickHouse/pull/66406) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If job detected as in-progress in PR workflow run - just remove it from todo list, as it not affected by the change. [#66407](https://github.com/ClickHouse/ClickHouse/pull/66407) ([Max K.](https://github.com/maxknv)). +* CI: CIBuddy to post from master only. [#66417](https://github.com/ClickHouse/ClickHouse/pull/66417) ([Max K.](https://github.com/maxknv)). +* Add a test for [#66333](https://github.com/ClickHouse/ClickHouse/issues/66333). [#66432](https://github.com/ClickHouse/ClickHouse/pull/66432) ([max-vostrikov](https://github.com/max-vostrikov)). +* Limit number of linker jobs on arm to avoid OOM during build. [#66435](https://github.com/ClickHouse/ClickHouse/pull/66435) ([Nikita Taranov](https://github.com/nickitat)). +* [RFC] Fix jemalloc assertion due to non-monotonic CLOCK_MONOTONIC_COARSE. [#66439](https://github.com/ClickHouse/ClickHouse/pull/66439) ([Azat Khuzhin](https://github.com/azat)). +* CI: Do not block CI on few number of test failures. [#66440](https://github.com/ClickHouse/ClickHouse/pull/66440) ([Max K.](https://github.com/maxknv)). +* Stateless tests: fix flaky tests 01037_polygon_dicts*. [#66445](https://github.com/ClickHouse/ClickHouse/pull/66445) ([Nikita Fomichev](https://github.com/fm4v)). +* Related to https://github.com/ClickHouse/ClickHouse/pull/62067 https://s3.amazonaws.com/clickhouse-test-reports/66410/5557dce188cabc7477bb4e874d47e3b80278ee66/stateless_tests__release_.html ``` 2024-07-12 16:04:29 +Queries for alter_table did not finish automatically after 250+ seconds 2024-07-12 16:04:29 +==================== QUERIES ==================== 2024-07-12 16:04:29 +Row 1: 2024-07-12 16:04:29 +────── 2024-07-12 16:04:29 +is_initial_query: 1 2024-07-12 16:04:29 +user: default 2024-07-12 16:04:29 +query_id: b43ffd7d-aee6-4161-aa82-bf9fff9d78c0 2024-07-12 16:04:29 +address: ::1 2024-07-12 16:04:29 +port: 58360 2024-07-12 16:04:29 +initial_user: default 2024-07-12 16:04:29 +initial_query_id: b43ffd7d-aee6-4161-aa82-bf9fff9d78c0 ... 2024-07-12 16:04:29 +query: OPTIMIZE TABLE alter_table0 FINAL ```. [#66460](https://github.com/ClickHouse/ClickHouse/pull/66460) ([Alexander Tokmakov](https://github.com/tavplubix)). +* OOM error was not visible since process is killed and status is not set Change sets ERROR status if job was killed. [#66463](https://github.com/ClickHouse/ClickHouse/pull/66463) ([Max K.](https://github.com/maxknv)). +* Add AST fuzzers jobs for CI caching so that they can be skipped in PRs not related to build or tests. [#66468](https://github.com/ClickHouse/ClickHouse/pull/66468) ([Max K.](https://github.com/maxknv)). +* If job with the same digest has been seen in master's CI it should be skipped in PR run. [#66471](https://github.com/ClickHouse/ClickHouse/pull/66471) ([Max K.](https://github.com/maxknv)). +* CI: Check job's exit status and report if killed. [#66477](https://github.com/ClickHouse/ClickHouse/pull/66477) ([Max K.](https://github.com/maxknv)). +* This closes [#37557](https://github.com/ClickHouse/ClickHouse/issues/37557). [#66482](https://github.com/ClickHouse/ClickHouse/pull/66482) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Add retry for GH set_status_comment call. [#66488](https://github.com/ClickHouse/ClickHouse/pull/66488) ([Max K.](https://github.com/maxknv)). +* OpenSSL: Minor follow-up to [#66064](https://github.com/ClickHouse/ClickHouse/issues/66064). [#66489](https://github.com/ClickHouse/ClickHouse/pull/66489) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Fix for job filtering in PRs. [#66490](https://github.com/ClickHouse/ClickHouse/pull/66490) ([Max K.](https://github.com/maxknv)). +* CI: Create release workflow updates. [#66498](https://github.com/ClickHouse/ClickHouse/pull/66498) ([Max K.](https://github.com/maxknv)). +* Add one more revision to ignore. [#66499](https://github.com/ClickHouse/ClickHouse/pull/66499) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unit tests vomit a ton of garbage, see https://s3.amazonaws.com/clickhouse-test-reports/66457/0c82dc91f07b29ba503d7579c7d3ebecba532b73/unit_tests__tsan_/run.log - remove it. [#66501](https://github.com/ClickHouse/ClickHouse/pull/66501) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix double whitespace in EXPLAIN AST CREATE. [#66505](https://github.com/ClickHouse/ClickHouse/pull/66505) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02530_dictionaries_update_field`. [#66507](https://github.com/ClickHouse/ClickHouse/pull/66507) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Https://play.clickhouse.com/play?user=play#u0vmrunuignozwnrx3n0yxj0x3rpbwusignozwnrx25hbwusihrlc3rfbmftzswgcmvwb3j0x3vybapguk9nignozwnrcwpxsevsrsbjagvja19zdgfydf90aw1lid49ig5vdygpic0gsu5urvjwquwgmjqwiehpvvikicagieforcbwdwxsx3jlcxvlc3rfbnvtymvyid0gmaogicagqu5eihrlc3rfc3rhdhvzice9icdts0lquevejwogicagqu5eihrlc3rfc3rhdhvziexjs0ugj0yljwogicagqu5eignozwnrx3n0yxr1cyahpsanc3vjy2vzcyckicagieforcbwb3npdglvbih0zxn0x25hbwusicdhcgfjagvfyxjyb3cnksa+idakt1jervigqlkgy2hly2tfc3rhcnrfdgltzq==. [#66508](https://github.com/ClickHouse/ClickHouse/pull/66508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MSan report in GRPC. [#66509](https://github.com/ClickHouse/ClickHouse/pull/66509) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Fix for skipping Builds_2 in PRs' CI. [#66512](https://github.com/ClickHouse/ClickHouse/pull/66512) ([Max K.](https://github.com/maxknv)). +* CI: Do not block Tests_3 unless MAX_FAILED_TESTS exceeded. [#66513](https://github.com/ClickHouse/ClickHouse/pull/66513) ([Max K.](https://github.com/maxknv)). +* Fix `02918_parallel_replicas_custom_key_unavailable_replica`. [#66516](https://github.com/ClickHouse/ClickHouse/pull/66516) ([Antonio Andelic](https://github.com/antonio2368)). +* Stateless tests: improvements related to OOM of test runs. [#66520](https://github.com/ClickHouse/ClickHouse/pull/66520) ([Nikita Fomichev](https://github.com/fm4v)). +* Tests: rename bad log names. [#66522](https://github.com/ClickHouse/ClickHouse/pull/66522) ([Nikita Fomichev](https://github.com/fm4v)). +* Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Multiple fixes for handling killed jobs. [#66524](https://github.com/ClickHouse/ClickHouse/pull/66524) ([Max K.](https://github.com/maxknv)). +* Allow GWP Asan allocations only when initialization is finished. [#66526](https://github.com/ClickHouse/ClickHouse/pull/66526) ([Alexey Katsman](https://github.com/alexkats)). +* Update 02443_detach_attach_partition.sh. [#66529](https://github.com/ClickHouse/ClickHouse/pull/66529) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Follow up [#66435](https://github.com/ClickHouse/ClickHouse/issues/66435). [#66530](https://github.com/ClickHouse/ClickHouse/pull/66530) ([Nikita Taranov](https://github.com/nickitat)). +* fix log in keeper tcp handler. [#66531](https://github.com/ClickHouse/ClickHouse/pull/66531) ([Han Fei](https://github.com/hanfei1991)). +* CI: Report job start and finish to CI DB. [#66533](https://github.com/ClickHouse/ClickHouse/pull/66533) ([Max K.](https://github.com/maxknv)). +* Update 01396_inactive_replica_cleanup_nodes_zookeeper.sh. [#66535](https://github.com/ClickHouse/ClickHouse/pull/66535) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add dedicated runner to libfuzzer, update docker. [#66551](https://github.com/ClickHouse/ClickHouse/pull/66551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* fix tidy build. [#66552](https://github.com/ClickHouse/ClickHouse/pull/66552) ([Sema Checherinda](https://github.com/CheSema)). +* No-op env change. [#66553](https://github.com/ClickHouse/ClickHouse/pull/66553) ([Raúl Marín](https://github.com/Algunenano)). +* Fix typo in new_delete.cpp. [#66554](https://github.com/ClickHouse/ClickHouse/pull/66554) ([alesapin](https://github.com/alesapin)). +* Fix something in Fast Test. [#66558](https://github.com/ClickHouse/ClickHouse/pull/66558) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* update trusted contributors. [#66561](https://github.com/ClickHouse/ClickHouse/pull/66561) ([Xu Jia](https://github.com/XuJia0210)). +* Delete bad test `02805_distributed_queries_timeouts`. [#66563](https://github.com/ClickHouse/ClickHouse/pull/66563) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* More clarity in the test `03001_consider_lwd_when_merge`. [#66564](https://github.com/ClickHouse/ClickHouse/pull/66564) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Stateless tests: temporary disable sequential tests in parallel. [#66585](https://github.com/ClickHouse/ClickHouse/pull/66585) ([Nikita Fomichev](https://github.com/fm4v)). +* Move view targets to separate AST class `ASTViewTargets` in order to allow extending it to support more kinds of view targets. [#66590](https://github.com/ClickHouse/ClickHouse/pull/66590) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix bsdtar for @nikitamikhaylov. [#66592](https://github.com/ClickHouse/ClickHouse/pull/66592) ([alesapin](https://github.com/alesapin)). +* CI: POC for Auto Releases. [#66593](https://github.com/ClickHouse/ClickHouse/pull/66593) ([Max K.](https://github.com/maxknv)). +* Fix clang tidy after [#66402](https://github.com/ClickHouse/ClickHouse/issues/66402). [#66597](https://github.com/ClickHouse/ClickHouse/pull/66597) ([vdimir](https://github.com/vdimir)). +* Adjust the runtime of some slow performance test. [#66619](https://github.com/ClickHouse/ClickHouse/pull/66619) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Scale down AutoScaling Groups from runners. [#66622](https://github.com/ClickHouse/ClickHouse/pull/66622) ([Max K.](https://github.com/maxknv)). +* Allow to run clang-tidy with clang-19. [#66625](https://github.com/ClickHouse/ClickHouse/pull/66625) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix terrible test @arthurpassos. [#66632](https://github.com/ClickHouse/ClickHouse/pull/66632) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad log message. [#66633](https://github.com/ClickHouse/ClickHouse/pull/66633) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Negative sign in prewhere optimization. [#66637](https://github.com/ClickHouse/ClickHouse/pull/66637) ([cangyin](https://github.com/cangyin)). +* Closes [#66639](https://github.com/ClickHouse/ClickHouse/issues/66639#event-13533944949). [#66640](https://github.com/ClickHouse/ClickHouse/pull/66640) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid generating named tuple for special keywords (null, true, false). [#66641](https://github.com/ClickHouse/ClickHouse/pull/66641) ([Amos Bird](https://github.com/amosbird)). +* rearrange heavy tests 03008_deduplication. [#66642](https://github.com/ClickHouse/ClickHouse/pull/66642) ([Sema Checherinda](https://github.com/CheSema)). +* Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* CI: Remove aws lambda packages from oss. [#66651](https://github.com/ClickHouse/ClickHouse/pull/66651) ([Max K.](https://github.com/maxknv)). +* Introduce separate DEBUG_OR_SANITIZER_BUILD macro. [#66652](https://github.com/ClickHouse/ClickHouse/pull/66652) ([Nikita Taranov](https://github.com/nickitat)). +* Increase backoff because with slow builds sometimes 100ms is not enough to recover. [#66653](https://github.com/ClickHouse/ClickHouse/pull/66653) ([alesapin](https://github.com/alesapin)). +* Fix wrong queries hung error because of 02044_url_glob_parallel_connection_refused. [#66657](https://github.com/ClickHouse/ClickHouse/pull/66657) ([Nikita Taranov](https://github.com/nickitat)). +* add log for splitBlockIntoParts. [#66658](https://github.com/ClickHouse/ClickHouse/pull/66658) ([Han Fei](https://github.com/hanfei1991)). +* Minor: Make `CaseSensitiveness` an enum class. [#66673](https://github.com/ClickHouse/ClickHouse/pull/66673) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bad test `02210_processors_profile_log`. [#66684](https://github.com/ClickHouse/ClickHouse/pull/66684) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix something around clang-tidy. [#66694](https://github.com/ClickHouse/ClickHouse/pull/66694) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ci: dump dmesg in case of OOM. [#66705](https://github.com/ClickHouse/ClickHouse/pull/66705) ([Azat Khuzhin](https://github.com/azat)). +* fix clang tidy. [#66706](https://github.com/ClickHouse/ClickHouse/pull/66706) ([Han Fei](https://github.com/hanfei1991)). +* Https://s3.amazonaws.com/clickhouse-test-reports/61109/5cf2b53f146c1a4f24d8212f9f810d587c46bfc0/stateless_tests__release_.html. [#66724](https://github.com/ClickHouse/ClickHouse/pull/66724) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* CI: Fix issue with a skipped Build report. [#66726](https://github.com/ClickHouse/ClickHouse/pull/66726) ([Max K.](https://github.com/maxknv)). +* relax condition in test, remove unused counters. [#66730](https://github.com/ClickHouse/ClickHouse/pull/66730) ([Sema Checherinda](https://github.com/CheSema)). +* Remove bad test `host_resolver_fail_count`. [#66731](https://github.com/ClickHouse/ClickHouse/pull/66731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `03036_join_filter_push_down_equivalent_sets`. [#66736](https://github.com/ClickHouse/ClickHouse/pull/66736) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad tests `long_select_and_alter`. [#66737](https://github.com/ClickHouse/ClickHouse/pull/66737) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test test_storage_mysql/test.py::test_joins. [#66743](https://github.com/ClickHouse/ClickHouse/pull/66743) ([vdimir](https://github.com/vdimir)). +* Disallow build exclusion only by CI settings (ci_include_, ci_exclude_) to avoid running builds in auto sync prs. [#66744](https://github.com/ClickHouse/ClickHouse/pull/66744) ([Max K.](https://github.com/maxknv)). +* Use non-existent address to check connection error at table creation. [#66760](https://github.com/ClickHouse/ClickHouse/pull/66760) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67063](https://github.com/ClickHouse/ClickHouse/issues/67063): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Fix flakiness of async insert tests due to adaptive timeout. [#66771](https://github.com/ClickHouse/ClickHouse/pull/66771) ([Raúl Marín](https://github.com/Algunenano)). +* Attempt to fix flakiness of 01194_http_query_id. [#66774](https://github.com/ClickHouse/ClickHouse/pull/66774) ([Raúl Marín](https://github.com/Algunenano)). +* Turn off randomization of harmful setting. [#66776](https://github.com/ClickHouse/ClickHouse/pull/66776) ([alesapin](https://github.com/alesapin)). +* The number of batches was reduced in https://github.com/ClickHouse/ClickHouse/pull/65186, but then the parallel execution was disabled in https://github.com/ClickHouse/ClickHouse/pull/66585. So now tasks fail with timeout sometimes: https://s3.amazonaws.com/clickhouse-test-reports/66724/36275fdacc34206931f69087fe77539e25bbbedd/stateless_tests__tsan__s3_storage__[2_3].html. [#66783](https://github.com/ClickHouse/ClickHouse/pull/66783) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove obsolete code from CMakeLists. [#66786](https://github.com/ClickHouse/ClickHouse/pull/66786) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Stateless tests: split parallel tests more evenly. [#66787](https://github.com/ClickHouse/ClickHouse/pull/66787) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix test `02724_limit_num_mutations.sh`. [#66788](https://github.com/ClickHouse/ClickHouse/pull/66788) ([Anton Popov](https://github.com/CurtizJ)). +* Better diagnostics in `test_disk_configuration`. [#66802](https://github.com/ClickHouse/ClickHouse/pull/66802) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `02950_part_log_bytes_uncompressed`. [#66803](https://github.com/ClickHouse/ClickHouse/pull/66803) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better diagnostics for test trace_events_stress. [#66804](https://github.com/ClickHouse/ClickHouse/pull/66804) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make test `00997_set_index_array` lighter. [#66817](https://github.com/ClickHouse/ClickHouse/pull/66817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#66818](https://github.com/ClickHouse/ClickHouse/pull/66818) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix inconsistent formatting of lambda functions inside composite types. [#66819](https://github.com/ClickHouse/ClickHouse/pull/66819) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Never await in CI on BuildReport - just redo (await can be longer) - Remove BuildReport if no build jobs in workflow (for instance: Docs change) - Do not fail CheckReadyForMerge job if the only non-green status is Cloud Sync. [#66822](https://github.com/ClickHouse/ClickHouse/pull/66822) ([Max K.](https://github.com/maxknv)). +* Remove bad tests @azat. [#66823](https://github.com/ClickHouse/ClickHouse/pull/66823) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: New Release workflow updates and fixes. [#66830](https://github.com/ClickHouse/ClickHouse/pull/66830) ([Max K.](https://github.com/maxknv)). +* Fix signed integer overflow in function `age`. [#66831](https://github.com/ClickHouse/ClickHouse/pull/66831) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix RocksDB bs. [#66838](https://github.com/ClickHouse/ClickHouse/pull/66838) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Split a test for index. [#66839](https://github.com/ClickHouse/ClickHouse/pull/66839) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix inconsistent formatting of `NOT ((SELECT ...))`. [#66840](https://github.com/ClickHouse/ClickHouse/pull/66840) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make test `01592_long_window_functions1` lighter. [#66841](https://github.com/ClickHouse/ClickHouse/pull/66841) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* What if I will change the test for SSL authentication?. [#66844](https://github.com/ClickHouse/ClickHouse/pull/66844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Functions `[a-g]*`: Iterate over `input_rows_count` where appropriate. [#66846](https://github.com/ClickHouse/ClickHouse/pull/66846) ([Robert Schulze](https://github.com/rschu1ze)). +* Revert libunwind patch. [#66850](https://github.com/ClickHouse/ClickHouse/pull/66850) ([Antonio Andelic](https://github.com/antonio2368)). +* Split test 03038_nested_dynamic_merges to avoid timeouts. [#66863](https://github.com/ClickHouse/ClickHouse/pull/66863) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: Print instance info in runner's init script. [#66868](https://github.com/ClickHouse/ClickHouse/pull/66868) ([Max K.](https://github.com/maxknv)). +* Backported in [#67257](https://github.com/ClickHouse/ClickHouse/issues/67257): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* CI: CI Buddy to notify about fatal workflow failures. [#66890](https://github.com/ClickHouse/ClickHouse/pull/66890) ([Max K.](https://github.com/maxknv)). +* CI: Add ec2 instance lifecycle metadata to CIDB. [#66918](https://github.com/ClickHouse/ClickHouse/pull/66918) ([Max K.](https://github.com/maxknv)). +* CI: Remove ci runners scripts from oss. [#66920](https://github.com/ClickHouse/ClickHouse/pull/66920) ([Max K.](https://github.com/maxknv)). +* Backported in [#67209](https://github.com/ClickHouse/ClickHouse/issues/67209): Decrease rate limit in `01923_network_receive_time_metric_insert`. [#66924](https://github.com/ClickHouse/ClickHouse/pull/66924) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67227](https://github.com/ClickHouse/ClickHouse/issues/67227): Grouparrayintersect: fix serialization bug. [#66928](https://github.com/ClickHouse/ClickHouse/pull/66928) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67207](https://github.com/ClickHouse/ClickHouse/issues/67207): Un-flake test_runtime_configurable_cache_size. [#66934](https://github.com/ClickHouse/ClickHouse/pull/66934) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#66975](https://github.com/ClickHouse/ClickHouse/issues/66975): CI: Fixes docker server build for release branches. [#66955](https://github.com/ClickHouse/ClickHouse/pull/66955) ([Max K.](https://github.com/maxknv)). +* Backported in [#67213](https://github.com/ClickHouse/ClickHouse/issues/67213): [CI Fest] Split dynamic tests and rewrite them from sh to sql to avoid timeouts. [#66981](https://github.com/ClickHouse/ClickHouse/pull/66981) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67033](https://github.com/ClickHouse/ClickHouse/issues/67033): [CI Fest] Fix use-of-uninitialized-value in JSONExtract* numeric functions. [#66984](https://github.com/ClickHouse/ClickHouse/pull/66984) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67051](https://github.com/ClickHouse/ClickHouse/issues/67051): CI: Fix for workflow results parsing. [#67000](https://github.com/ClickHouse/ClickHouse/pull/67000) ([Max K.](https://github.com/maxknv)). +* Backported in [#67116](https://github.com/ClickHouse/ClickHouse/issues/67116): Disable setting `optimize_functions_to_subcolumns`. [#67046](https://github.com/ClickHouse/ClickHouse/pull/67046) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#67205](https://github.com/ClickHouse/ClickHouse/issues/67205): Increase max allocation size for sanitizers. [#67049](https://github.com/ClickHouse/ClickHouse/pull/67049) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67124](https://github.com/ClickHouse/ClickHouse/issues/67124): Very sad failure: ``` 2024.07.24 13:28:45.517777 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} executeQuery: (from 172.16.11.1:55890) OPTIMIZE TABLE replicated_mt FINAL (stage: Complete) 2024.07.24 13:28:45.525945 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (ReplicatedMergeTreeQueue): Waiting for 4 entries to be processed: queue-0000000004, queue-0000000002, queue-0000000001, queue-0000000000 2024.07.24 13:29:15.528024 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e) (MergerMutator): Selected 3 parts from all_0_0_0 to all_2_2_0 2024.07.24 13:29:15.530736 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Created log entry /clickhouse/tables/replicated_mt/log/log-0000000004 for merge all_0_2_1 2024.07.24 13:29:15.530873 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to process log entry 2024.07.24 13:29:15.530919 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to pull log-0000000004 to queue 2024.07.24 13:29:15.534286 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Looking for node corresponding to log-0000000004 in node1 queue 2024.07.24 13:29:15.534793 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for queue-0000000005 to disappear from node1 queue 2024.07.24 13:29:15.585533 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} TCPHandler: Processed in 30.067804125 sec. ```. [#67067](https://github.com/ClickHouse/ClickHouse/pull/67067) ([alesapin](https://github.com/alesapin)). +* Backported in [#67203](https://github.com/ClickHouse/ClickHouse/issues/67203): Fix flaky `test_seekable_formats_url` and `test_seekable_formats` S3 storage tests. [#67070](https://github.com/ClickHouse/ClickHouse/pull/67070) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67222](https://github.com/ClickHouse/ClickHouse/issues/67222): Fix 2680 flasky. [#67078](https://github.com/ClickHouse/ClickHouse/pull/67078) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#67190](https://github.com/ClickHouse/ClickHouse/issues/67190): Attempt to fix flakiness of some window view tests. [#67130](https://github.com/ClickHouse/ClickHouse/pull/67130) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#67272](https://github.com/ClickHouse/ClickHouse/issues/67272): Rename (unreleased) bad setting. [#67149](https://github.com/ClickHouse/ClickHouse/pull/67149) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67441](https://github.com/ClickHouse/ClickHouse/issues/67441): Try to fix 2572. [#67158](https://github.com/ClickHouse/ClickHouse/pull/67158) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#67416](https://github.com/ClickHouse/ClickHouse/issues/67416): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 271065a78fb..027b207d3ad 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.4.49-stable 2024-07-01 From 8d2b804c670d0941acc4fff059859017c1bd93c2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 30 Jul 2024 21:57:08 +0100 Subject: [PATCH 0507/1170] fxs --- tests/integration/test_recovery_time_metric/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 8f369d7759c..628f2e744e0 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -37,9 +37,7 @@ def test_recovery_time_metric(start_cluster): """ ) - node.exec_in_container( - ["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"] - ) + node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) node.restart_clickhouse() From 4aedb9d40298c1a3204bb72a3288ea711eb5e2f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 23:02:22 +0200 Subject: [PATCH 0508/1170] Update test --- tests/integration/test_system_flush_logs/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 2022f9d4a89..713b327eb76 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -13,9 +13,8 @@ node = cluster.add_instance( ) system_logs = [ - # disabled by default - ("system.text_log", 0), # enabled by default + ("system.text_log", 1), ("system.query_log", 1), ("system.query_thread_log", 1), ("system.part_log", 1), From 51212a414fd57270694c2653f730890b77714949 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 21:13:20 +0000 Subject: [PATCH 0509/1170] Revert "Rename bad setting" This reverts commit eb4ec0912ad3a1e89ea7aec424366bc268262e11. --- CHANGELOG.md | 1 + src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp | 4 ++-- tests/queries/0_stateless/03013_json_key_ignore_case.sh | 4 ++-- 7 files changed, 9 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a4c873ba3f9..730346c0a2f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -65,6 +65,7 @@ * The setting `optimize_trivial_insert_select` is disabled by default. In most cases, it should be beneficial. Nevertheless, if you are seeing slower INSERT SELECT or increased memory usage, you can enable it back or `SET compatibility = '24.6'`. [#58970](https://github.com/ClickHouse/ClickHouse/pull/58970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)). * The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)). +* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). * Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)). * In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)). * Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27b71558bd3..0aa879fd9ad 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1127,7 +1127,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_case_insensitive_column_matching, false, "Ignore case when matching JSON keys with CH columns", 0) \ + M(Bool, input_format_json_ignore_key_case, false, "Ignore json key case while read json field from string", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8bea0b1eed3..d38c8025227 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,7 +64,7 @@ static std::initializer_list Date: Tue, 30 Jul 2024 21:14:22 +0000 Subject: [PATCH 0510/1170] Revert "Merge pull request #61750 from KevinyhZou/improve_json_each_row_ignore_key_case" This reverts commit 3229cb5874cc9c8b04e16d2fec3231a25a3fc171, reversing changes made to f838c25d20bc87ed9a788eff0659d26be42cadb6. --- src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.cpp | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - .../Impl/JSONEachRowRowInputFormat.cpp | 19 +----------------- .../Formats/Impl/JSONEachRowRowInputFormat.h | 11 ++-------- .../03013_json_key_ignore_case.reference | 3 --- .../0_stateless/03013_json_key_ignore_case.sh | 18 ----------------- .../data_json/key_ignore_case.json | Bin 123 -> 0 bytes 9 files changed, 3 insertions(+), 52 deletions(-) delete mode 100644 tests/queries/0_stateless/03013_json_key_ignore_case.reference delete mode 100755 tests/queries/0_stateless/03013_json_key_ignore_case.sh delete mode 100644 tests/queries/0_stateless/data_json/key_ignore_case.json diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0aa879fd9ad..8cc25f42cc6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1127,7 +1127,6 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_ignore_key_case, false, "Ignore json key case while read json field from string", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d38c8025227..873578013e1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,7 +64,6 @@ static std::initializer_list position in the block`. NOTE You can use perfect hash map. Block::NameMap name_map; - /// Hash table match `lower_case field name -> field name in the block`. - std::unordered_map lower_case_name_map; + /// Cached search results for previous row (keyed as index in JSON object) - used as a hint. std::vector prev_positions; diff --git a/tests/queries/0_stateless/03013_json_key_ignore_case.reference b/tests/queries/0_stateless/03013_json_key_ignore_case.reference deleted file mode 100644 index 54683d8fbc5..00000000000 --- a/tests/queries/0_stateless/03013_json_key_ignore_case.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 77328912 Ben -2 77328913 Jim -3 77328914 Bill diff --git a/tests/queries/0_stateless/03013_json_key_ignore_case.sh b/tests/queries/0_stateless/03013_json_key_ignore_case.sh deleted file mode 100755 index 807e743b22a..00000000000 --- a/tests/queries/0_stateless/03013_json_key_ignore_case.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash - -# NOTE: this sh wrapper is required because of shell_config - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') - -cp "$CURDIR"/data_json/key_ignore_case.json $USER_FILES_PATH/ - -$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (id UInt16, reqid UInt32, name String) engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_tbl SELECT * FROM file('key_ignore_case.json', 'JSONEachRow') SETTINGS input_format_json_ignore_key_case=true" -$CLICKHOUSE_CLIENT -q "select * from test_tbl" -$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_json/key_ignore_case.json b/tests/queries/0_stateless/data_json/key_ignore_case.json deleted file mode 100644 index ad8f7cb450780891d64ac8cbbc19de17b92e7db5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 123 zcmbzd5O8HN>)lvsd-AZT-9KOAYoL6#t4O8 VnYnO Date: Tue, 30 Jul 2024 16:50:37 -0600 Subject: [PATCH 0511/1170] Fix case sensitivity for percent_rank, dense_rank, and their aliases --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a1b46c8e36c..5fad68e4968 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2726,18 +2726,18 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) From 3b12fec141fda8cb2a3ef68ac96e6e58f1fd69e3 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:05:48 -0600 Subject: [PATCH 0512/1170] Update dense_rank doc to mention the denseRank alias --- docs/en/sql-reference/window-functions/dense_rank.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md index d6445b68c55..2c8617fb668 100644 --- a/docs/en/sql-reference/window-functions/dense_rank.md +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -12,6 +12,8 @@ The [rank](./rank.md) function provides the same behaviour, but with gaps in ran **Syntax** +Alias: `denseRank` (case-sensitive) + ```sql dense_rank (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] From 419a5e7f730dabe514becabc6c24ec5b87325e28 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:17:01 -0600 Subject: [PATCH 0513/1170] Update window-functions doc with denseRank and percentRank aliases --- docs/en/sql-reference/window-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 0c3e2ea1cb6..27d4bd763c7 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,8 +23,8 @@ ClickHouse supports the standard grammar for defining windows and window functio | `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | -| `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `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)`| +| `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()`| | `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 bc312eb046db07d901e208cdc1bb0abb1df3eabd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Jul 2024 21:27:50 +0200 Subject: [PATCH 0514/1170] Improve check --- src/IO/S3/URI.cpp | 32 ++++++++++++------- src/Storages/StorageFile.cpp | 18 +++++------ .../03215_parsing_archive_name_s3.sql | 2 +- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 33a4939c810..fead18315d8 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -55,10 +55,10 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) static constexpr auto OSS = "OSS"; static constexpr auto EOS = "EOS"; - if (!allow_archive_path_syntax) - uri_str = uri_; - else + if (allow_archive_path_syntax) std::tie(uri_str, archive_pattern) = getURIAndArchivePattern(uri_); + else + uri_str = uri_; uri = Poco::URI(uri_str); @@ -176,22 +176,30 @@ std::pair> URI::getURIAndArchivePattern( return {source, std::nullopt}; std::string_view path_to_archive_view = std::string_view{source}.substr(0, pos); + bool contains_spaces_around_operator = false; while (path_to_archive_view.ends_with(' ')) + { + contains_spaces_around_operator = true; path_to_archive_view.remove_suffix(1); - - if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) - return {source, std::nullopt}; - - auto archive_uri = path_to_archive_view; + } std::string_view archive_pattern_view = std::string_view{source}.substr(pos + 2); - while (archive_pattern_view.front() == ' ') + while (archive_pattern_view.starts_with(' ')) + { + contains_spaces_around_operator = true; archive_pattern_view.remove_prefix(1); + } - if (archive_pattern_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Archive pattern is empty"); + /// possible situations when the first part can be archive is only if one of the following is true: + /// - it contains supported extension + /// - it contains spaces after or before :: (URI cannot contain spaces) + /// - it contains characters that could mean glob expression + if (archive_pattern_view.empty() || path_to_archive_view.empty() + || (!contains_spaces_around_operator && !hasSupportedArchiveExtension(path_to_archive_view) + && path_to_archive_view.find_first_of("*?{") == std::string_view::npos)) + return {source, std::nullopt}; - return std::pair{std::string{archive_uri}, std::string{archive_pattern_view}}; + return std::pair{std::string{path_to_archive_view}, std::string{archive_pattern_view}}; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index efb39f90053..8c079aa4600 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2258,21 +2258,21 @@ void StorageFile::parseFileSource(String source, String & filename, String & pat while (path_to_archive_view.ends_with(' ')) path_to_archive_view.remove_suffix(1); - if (path_to_archive_view.empty() || !hasSupportedArchiveExtension(path_to_archive_view)) + std::string_view filename_view = std::string_view{source}.substr(pos + 2); + while (filename_view.starts_with(' ')) + filename_view.remove_prefix(1); + + /// possible situations when the first part can be archive is only if one of the following is true: + /// - it contains supported extension + /// - it contains characters that could mean glob expression + if (filename_view.empty() || path_to_archive_view.empty() + || (!hasSupportedArchiveExtension(path_to_archive_view) && path_to_archive_view.find_first_of("*?{") == std::string_view::npos)) { filename = std::move(source); return; } path_to_archive = path_to_archive_view; - - std::string_view filename_view = std::string_view{source}.substr(pos + 2); - while (filename_view.front() == ' ') - filename_view.remove_prefix(1); - - if (filename_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); - filename = filename_view; } diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql index 3a7ed0b864c..e34be475c5a 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -2,6 +2,6 @@ -- Tag no-fasttest: Depends on AWS SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); -SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError STD_EXCEPTION } +SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError S3_ERROR } SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); SELECT _file, _path FROM s3(s3_conn, filename='test.zip::03215_archive.csv') ORDER BY (_file, _path) SETTINGS allow_archive_path_syntax=0; From e664a144788b48c029f56548242baaeed82a80ff Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 08:49:14 +0100 Subject: [PATCH 0515/1170] fix style --- src/Databases/DatabaseReplicated.cpp | 4 ++-- src/Storages/System/StorageSystemClusters.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 06cea65d62e..b2be593d326 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -356,7 +356,7 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); } } - + try { auto current_zookeeper = getZooKeeper(); @@ -396,7 +396,7 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) ++global_replica_index; } } - + return replicas_info; } catch (...) { diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 4b9802c9089..d03b600b6ef 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -116,7 +116,7 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam else res_columns[i++]->insertDefault(); } - + ++global_replica_idx; } } From 5152248d438ef9162845507b68e18f1d8541a250 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 09:25:59 +0200 Subject: [PATCH 0516/1170] Add test. --- .../02864_restore_table_with_broken_part.sh | 25 ++-------------- ...ackup_with_matview_inner_table_metadata.sh | 25 ++-------------- ..._clear_old_temporary_directories.reference | 2 ++ ...kup_and_clear_old_temporary_directories.sh | 22 ++++++++++++++ .../0_stateless/backups/mt_250_parts.zip | Bin 0 -> 265998 bytes .../helpers/install_predefined_backup.sh | 27 ++++++++++++++++++ 6 files changed, 55 insertions(+), 46 deletions(-) create mode 100644 tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference create mode 100755 tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh create mode 100644 tests/queries/0_stateless/backups/mt_250_parts.zip create mode 100755 tests/queries/0_stateless/helpers/install_predefined_backup.sh diff --git a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh index 08313e2fd3b..bf76727f76f 100755 --- a/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh +++ b/tests/queries/0_stateless/02864_restore_table_with_broken_part.sh @@ -5,29 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, -# returns the path to the backup relative to that disk. -function install_test_backup() -{ - local test_backup_filename="$1" - local test_backup_path="$CURDIR/backups/${test_backup_filename}" - - local backups_disk_root - backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") - - if [ -z "${backups_disk_root}" ]; then - echo "Disk '${backups_disk_root}' not found" - exit 1 - fi - - local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} - mkdir -p "$(dirname "${install_path}")" - ln -s "${test_backup_path}" "${install_path}" - - echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" -} - -backup_name="$(install_test_backup with_broken_part.zip)" +# In this test we restore from "/tests/queries/0_stateless/backups/with_broken_part.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh with_broken_part.zip)" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl" diff --git a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh index 8d987dbf1df..2c70cb1e3be 100755 --- a/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh +++ b/tests/queries/0_stateless/03001_restore_from_old_backup_with_matview_inner_table_metadata.sh @@ -5,29 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk, -# returns the path to the backup relative to that disk. -function install_test_backup() -{ - local test_backup_filename="$1" - local test_backup_path="$CURDIR/backups/${test_backup_filename}" - - local backups_disk_root - backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") - - if [ -z "${backups_disk_root}" ]; then - echo "Disk '${backups_disk_root}' not found" - exit 1 - fi - - local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename} - mkdir -p "$(dirname "${install_path}")" - ln -s "${test_backup_path}" "${install_path}" - - echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}" -} - -backup_name="$(install_test_backup old_backup_with_matview_inner_table_metadata.zip)" +# In this test we restore from "/tests/queries/0_stateless/backups/old_backup_with_matview_inner_table_metadata.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh old_backup_with_matview_inner_table_metadata.zip)" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src" diff --git a/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference new file mode 100644 index 00000000000..3f3fbd9ab58 --- /dev/null +++ b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.reference @@ -0,0 +1,2 @@ +RESTORED +250 31375 diff --git a/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh new file mode 100755 index 00000000000..e0c8f08e695 --- /dev/null +++ b/tests/queries/0_stateless/03214_backup_and_clear_old_temporary_directories.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# In this test we restore from "/tests/queries/0_stateless/backups/mt_250_parts.zip" +backup_name="$($CURDIR/helpers/install_predefined_backup.sh mt_250_parts.zip)" + +${CLICKHOUSE_CLIENT} -nm --query " +DROP TABLE IF EXISTS manyparts; +CREATE TABLE manyparts (x Int64) ENGINE=MergeTree ORDER BY tuple() SETTINGS merge_tree_clear_old_temporary_directories_interval_seconds=1, temporary_directories_lifetime=1; +" + +# RESTORE must protect its temporary directories from removing. +${CLICKHOUSE_CLIENT} --query "RESTORE TABLE default.mt_250_parts AS manyparts FROM Disk('backups', '${backup_name}') SETTINGS allow_different_table_def=true" | grep -o "RESTORED" + +${CLICKHOUSE_CLIENT} -nm --query " +SELECT count(), sum(x) FROM manyparts; +DROP TABLE manyparts; +" diff --git a/tests/queries/0_stateless/backups/mt_250_parts.zip b/tests/queries/0_stateless/backups/mt_250_parts.zip new file mode 100644 index 0000000000000000000000000000000000000000..15310b79054e09c75876efd20845bbdabdfec2f5 GIT binary patch literal 265998 zcmZ^LbyU>d_qBycBi$g~(%s$NAT23KcS(1Hv~;I1q$7e1-7V7HNF(_UqxgN-`pxq% zi?u%M-gC}A`|PvtD9Jv3hVkghix-bx^WJF%st<_?m_2%wHSzor?4w7I7>$igtex#v zfqS3Ho1zSV(_TH{Cl!!FHF@ii#D+%V3LtZC0OSLHxL*cIVv#w>nQ$yQ9NY-Mexva= zsnm9PanzoGNa2X{s=+tY_fqig;^!Uf?MjvJWrObx+4Wh-Y2Sj^#lqdqkniWShC`Fv zt)Z2wuDh$4yI&2hH(Rg4|7q>}`&`gitX1627ys_E;X*!oVJD{b#`>!7BCnwp z5NcB2diVRi??vB&kD-F#E%2sm;r24*Y(wzQFHxWDrtP-M=x(pd_pZ<2u5aP?YQg8~ z`%dc(_%D~lD=|{P8w9`joc_H0cq=tm5Ofjw6)EO=o5_KTWB*C!Q_VN}HsCysUt9*{ydw3%9QbE_VLxz9JxlSYTvsU!JC2` z@GTqMuHN1FMi*U+3KeMYb9n#?$IN4@M!K6x9wYzqd~&&S5hDl?yxa1vYbCV2Y`wdD ze|ISeQa`>UtsIf~pxxGH{UOCPg}U%=IGH5MbNTY%t^oYc$UzsxB3q#WZwIY7$}x*8 z>xCR;f{*Ji|zH3p5wOmh>kr;4%G9~9dp16!nfsKEx!}JTUXy@%xiBN8V z@O%3F_lUc8gu%JRn7{d{`_=AeLkwC54`Gi>FR6bLJrB z?06B@h6;tTG{nPhNw6&-;xrJJoAH~7rHb0yWYb|svt1P#wW!g(EaMs=EcT`*QR4jb z33_?etnsa_>lnFR;mqr_gI6Ka@eBv(GYQ2t+^52C~kX_T-~!NUptBc=wE5O zX;fD5-bOq=9S6~H4uz?xZ8<5{OwR`(oY0{tR0^n(3~_i3W{|BI?sApK*UlvzNAaZ? ztkl6u_HTxdnhiVxM5h*}+0|IIRJ^&SKPMdXA-c}qTWu3RRYGCBrgF2Z$@*mox)@nE z)o~*KCek!zl*cp@v$PDsA~?*Nm^z8Kwb8%{>#xx#(UyWJKEp0Yi@6@fu-Lcge2rR)Zm}I=>~Zc+ek^5f_3%-; z7&|I5-1Gs1j}XwC*Dl;jeTO=-6q> zlAWXGomf0ILXOomjq=rGW+;HTE|agZm}t8#=dN%_+fL zpsgBb5);jPpT;p&wODG~0Esr^_+_BW@bg;E@I+1pFZ-eJ1+tl;4+!t5p5B^z6yN!@ z3%2r*<`g;Cx7{9}W^C}o1Y7&*t z4e>_ve`8;X-)|$tZi~avU{-N@L87F(lT6ty592|GvTU8@EapAt?Jz7MXb*9hAfGI_5eFaDaWW<8ScVASYR4 zAM`*C6Q*+rOGloHHO!wiFqb_snIdXRUWW!R>xmRyS=zi-P9Kop4fOVlmqro@PN)9? zF$4E%zlMw${!gkqzy=k3ipX1CTF$SRUpMdT9*i#@*ns^_7~N2+M`Q0Q3iYk~KZQ|a zyH4D8)O~^#wBU8OQ|2EJK)8b=(km}=L7y1kt_eY;6^R#lRB@V!$uuwyPUD}Rb{~A! zggYXY;d_K4= z#78&{A;@{$w=1i&HJM`D-}cV}-ZDM`oV#e=!z+hMMLe2or<8 zv+lW5OF-8w%A!|OT^<7N$5m;H-ZHGg8rq%ZfrwbY316kI+u0;_ zP!;a=pCimj?0c&6kNLd6hP^l#@Gv7Dk&&3lGhOV?(slAPQ8}T|w(AD5x-2~2+yBP7 zsfktIOHqg}8`jZLW*Hn>cNg=1@EokBrYfQNor5&um`60bY8&6AK}&7`Zj@taz(S>rtdpx!qa zigkrD8kyR&Rnl+@G?bADPC}+ipw~&*|7~PLyQOnS{ftc7IZDAgrDwK=Bl#A3+5bq) zV%aT#Jnl%!am3=gc+p2Sr&8_a2b|eLORk`jU{WUg9vJ7w2=Lg)-2q7wLgyDNu)M4% zje=$|pZPwkNR-YyoyDa+RHtoHhiMVKnS0?Q0^>)kYRgvJWGi3(vY3;7W)M6vEGOP1 zDSd_b9vXEa9ME7Ebs5@){ot1<#FBZ+M#~h%BJYse{#{yjJo@3nyalGi`%^Rny4G04K1|vYVd}_R>rG3oEoOA9}hlq21R)N907HGK)hk_OJTlP&6LW6}Q zYEzV4f0oL}2)e2H$2RvZd5#vYVFY#E{>#L-c=pF-sPzP4#7*<}WJyaO9F|v}{flZk zA34xn6rVarv*7}s=v{AKK_g4r>{aA!%x@>+d#Sgyq}L8@(oC`_9{5FXK;T>C=e5EG z`4^AJ3pr$8k2D}MaAIyoK~RQMd1ZXMprJ7`X*v~O%w#jsv&;U_IUgFDnN9EwScoFl z5M5~n-^gU*26`FF!vB9VD zCh2%JU;A5y-FKe{(l=-NqR4KKEAKqoVKYw(Z!qtS+~>Z@5E+;=XuW)=Vddi% z@^MmAcEt(Qa zVxRbxXMCUf<kLe*G2pqQ?>x^iB*=Q$S9zJIp9vxHHRRaJrpa-5iG*v(c@x<8Pbc z)vb1|n(#n6CU_sBfs=eXeclv;@M__{YNYO2xdIz?7r(?dE2Ad!(e64>nuc&4#MTOh zEVH&)p=+cd0h91FSH5lib~7TQFAk2~|Jybjbd(UzW;{$j0i=5Ks>uok*e;LC&WS$V zpQg2QYtAHk>Pv48%Ba#PCIY$rtY(9Y|_Gdmun&m!U^trfsW~K#~76>w8 z_E~3~8NlV;XnpKL~`ty07UnHaP(cXnJegA6# z8MW%)w?*qjyi~(j6ln9-OFd=->g%Gj*);Nn$`N!&X$n-;K*4iu-@~aDs)(=0y=6_Gptntabp|EDwmaa%1!Vjq>xJ8M1??{-F8wDarXa7?3 z!GMD1!*^n&Gv@wNORmpH@>cb}K(M(O%*_aaY~9&;iD@gPQIoL|VzS)@Y(Ztj-=WQ9 ztV7~-BZkH<=kGu-95XyA?IKE zYBn7&`gy&pZxtv@gyC}7$d+!^k|*}={nt+^3wybgJr`UXhHl&9+}u|1J8UwR$Z4*t zp2R!$9#OUPJ0Hdgj^uLED{E2qOPq*RLM2+ya(19{n!>9wkBsb&VX9i`QY|kR3%;>K zO)x>x$)Kk+F#DHtffgY(>e&dzqmzDyuL?ix2Rg8M4l8QF+gt=m+juD zABXglz6{1Jh6`w1BZGmu%QMG07AM3Y-pqzSHEt|XWA&ey;WX*5##)~oGr(|M(*kF) zpB?m*9>O-klhZyd86f!yOtZ3$48JW$=_JvV?Y7t6<2z_GO|56Yl471BD0jXl)lRxZ z-LW%e(C(oZO!(J6CX5kB7_?2w$<0o5$Php=ga{2Jz)p7*P!bB%L^1~_PfY4)N^!b}<|egw5#4tpWuH#DH(>$Wu8;?o(TVU#T8BMA_9 zj5kvH*FAAts10v6uU`6GI%R%l)9PVW&yS~74~j1Q{!L1a z(f0gWminh#^%rn;no^=5x7HE45*B;#oxG;dIoze$J^^jyLL-9XmH~@%1oFiq?^&he zuriktd5#(z>n;d~|B67NUIcF^^8HUrLFoSZGtuA*@=^thK-piwvYbd+$xb2>CpMg+gDk%)6=gCtGUz(~l@HuihYd8l8BwhgAC2|>X{ zqPNWiHPs>opP8wz&avKnYlIjI6=9q9o3n}-pmg)y4)Lr#&0exi;9IImj_hW z=)30$Cy`f54E{o+&nBe9t*}N%=-4t%T@yq12F}rsTgKKYhN$6FOe0@bbbliP8Y~re z(#SYAku6+ly}sSYF$ZeO!Ue9#s^{J@l`GYJ zEFQGwK|8r;X>%Xm4_DE1(S~|l$0eCXc%l+3t+;#_pRkLD+}RSCq1(ki>B^_1jS}_2 zvQ)4aHYYlE`|@A?%x7l0mdC|HEbC>_AWKQ^s;PZW^50}Oa7Wd(>7r|7`K;KNq4U~y z)+w|I`4qyOyrRb5hsdNehp0Ygi1*l^F0nexSABFb;|Qi`|5s;R52F@i#es>9%@uI^ z_=H_k<7p_o$Uo}bASxYRLK+ZJ?OqZfi@8&4$0Y|D0rRK0cG?IEd&WEySGa3B2_+I= z%O--EMr$MUDAc+XK0=i>tJTVoV2%7p9Y0{sD}d;nCf2s#g( zMSytfCn(~4zoW8fiZQ^Lj>f2uR&+W+w*Hy=|45*o&L`H5sayS69sJ^fTq~5+jzGw* zO-^y`2_MT)a=%-_EL|If_wHZuDjPpE{&@q1Hz`(RD+;`D2*?PRH21f^>(_Eh%T|97 zfZqs8eZT4_{He_P$j(fdGj6lD?RAzD1euk<_}?5R@FFR#qoWd#v?5m)zg{CuqS(2t;@)l-TnBTQ$&D=}-I zf1TxX>&XR~p^~9`ZVG2S->qgEt?)+82G$X}ps=s#B&a8YU~^?ao$$$vDpf zB~E8be?vop&VYf`(vw1D?qZJ(jkW0@`$3K7xra@_1mACxw!%1D^LLvSn!=Hlv9OL1 zoffRpTo-*vp<4T)7D1$5hEB68Fl?9TVQ&R}t_jmHlFA=@SEh z&eVW1gLxf@fepB=rq81XgPgY^AaH_Lq^K_Q^jn&mNFc2Mf-63tHuMM16o&-Vk!C*p zvD1sH$P$`w)r547<}#`i$5O`+wRy9+YxIq+{Nv`A5{*9i(Ts5O*Vsd8clS&KOlU5T z+p-El&a_SatC%x~BXk3ka;oE15e;t8C_o@ALRnsioB&a*L&Z3^WKb4UZ=L_zztgZO zl(T!Z>F`M-6FoVcUr8o+e+_}>wrji%&4G`l_ARfnk$EJqo4D8t#&MIXDzL7h@g|Ld zkZ0+J7iL6VI})m!@ri|LC}Gmx4_3@?TbEOf ziH0H!D?U}Hy?Gh! zC2d`Ht2q7#m=?084to_tOTZ7O0x|NtM+Bj)e>f8uV-&^nfRoIs)<*3aTn&6H`cKPH zKSpDx??c@+DK@@Ni%We=$-w{Af6X#ps3Pn{?Iank2Kd{@*k=tmT9`zQA>K^&Hq#;n z4Us;~y?ORt2&UR%AFRSkG!nOR@&TxyGwTspi-ZJn8}K%RH)t;FS1>kY=fILI%T~_Y zr`P9Z{9`wuhOuIQLdwLH9?Cbbl-a0KiVSA$?g)qXk-Y+Z8bUavib{Lvm`$)!yM9#J zrkRxh{;V9pD{oh2Itf5*E$Jhysd7J{`ilbsMefRi-_RnXO%W&aiBzDbw~iXu=}V<@ zu7=%4;_-t9kmbJ~7gI6DLWNl)<=9V4Sb<2u+y@!YJHXA^G8-TEB{K0L(htOC57`0F zVD8u|VBi(B(TwlqTGnzQH;OT-Cy+HZ2zbhvtNQz2Rc6c^Zr7AZZvm2cnpQ2onOh^} zk|aa88CX3(AY#o4b(T%OutSQv&}Ho(@$5qC0-!WQ6`YQf0&yG(Y1VCLIYh7dx(<1S z(YC~*9+IFT-D0T6arMnuf2`+hm|TGXoZmMP1e907%t_crjn!}~Lq@zZ|IY8(dceyf zq-F_mdX^>((@)Q7X(V?>r)67bP<%vwbOx-kD|l9GPv9M2xjmh6Dg`~0!8>=)GYbAM zg>KGdh9s2%KHzND;eGZ@_~*$C-d7KbMsGlBbWCM1ugZ87pNP0puAZk0579DYIZq9S zS1Xx{KDT|(nb?gn**qctAR8|^H9^ra-7a)YT7YY+8* z8bIJm$leuXV#aTV7Qs7Rr)>B90McV1<4murYQTt$d{VwbhITYMe?B(se=X_8Hn?4% zXK$yr|KjoWG%49DEa;F^AQ{*Jku%as&Hn`Yx_APw>G4CbF+9Z0*fZR$2b1< zaaZ$dLv;)IcaLv9YqOXQ7vVGyqJqZqhY<=zx^9z4;^e>-@Q*r!bwcEff_FLGm{l6n zigIcufCaGOq^9do^@mrX(zG!nBeS1`_I&n4lxCep?! zL(5;O&MNQyu%Ow0CoEQ`MGybViZrbP$@pnE-W1jer1}rR(<~-ByVkl>2f;aZ1eDx&O-;ux)E|(Z2xi5k zA$GIwf6O7_93c3(_W+rLTE-bA+M%y?Y&jFsJUQvKT9S%MAUSYko#FesQ{)3kC(0j@ z``edj4xq!fm`W9NVE$!T<{r)BoIKk1C?Ib6{kAEdGWP!tGZ-{33pYiafm_ zAoua&AN$X-b;g2t#p%3Cw;eUsO4hRL;tKaxjrW^Yw}^9S1ArSUT5@tCDr@4iVsaog zmE3&EWctItnMp1{IogJUvu4Gw46&<7OPa@_$Kk;>QwD?1&JwP}Ctk`T<+E2d36E4_ zXXEG(HTsDKn{tunxO0*#e4%)Saj3tzvO2H+URV`hXuftG9;X6~P;Y)%*1&Vq4}1_v z1`EJD!hJnDM8L-^Is?5h7hK$k&Pre18V{ibr8Xi>YMg&tUZpOetMe+SO@RMWgyD2~ zF!|t5^li^^a-}ssL<7*~!Lxsn9hnT4E7>sQL*RYWWB@L8j1fbj;!tKUB6_%=+$b)0 z>qYSjG-Td_EmJMkHlCRb$3+9k5wSCczd~64cNnhlBmp?duufFN`jN89cR=^v-4)3E z?+unRS9z!jJ&(1_&7M5(dIoeYsrU^FP=zKj;i9N)`~ZCHMHb3Z;h$e(${b#AY)-cM zD*+FQ+xRD~i{VDFQbm#CsM=tKv}-QNUj}}Tx}TTKWfvFqdDK*U02>(fG{YVok=H*# zAevUmx=&BBrxl)G_nau*-30pj-}uC5*?sb0J?05ORej4XkVJZ*49i-J_s=$?@Z)Vd z4;2|!Wy+Vyj7LT%J)`BbD)HwvS%u)x0JoYbY)JATG+hnG=TYmd0ru3%2;$T)PiOupGdS3Tei9aT??}F{Ye>*hE5CYmndlBFOz^<` zW;~j|E`pMKf1^lwxgv@Y|@(230bWH%7muZcufD^>!2u~F%vFS`<* z%roqBJn!E`UI+$+Y>VztG;R0g~VP9;f@ zkd*2Q_LEK|aU{ppkMD#{@)oeO7jiRLZCiM8s=vP_4gFV zXx{C=H)i0@w9nT}K}<`CS*#wc)0-c;&o>5fG~CNFwz>NNX&wQDx~)`|)aOW<5)<)N z0zS}~3>9Cf^6>lyb!y z>|FcoHd*(_?72k#KAfF1_Te)vN+Gs8v)sr4bVgOz$#{!o$-xyRPa-DHoVHiDZ1{Kq zdF8fdn#Epa1*#imTV;w{IpDlXb>N@+$%VYaYsAy4x>-}1L^dQ-^=32pP--mi=dno~ zo!(6F^~V$!$GIwttBZZnQ>hK_U+3rZA*X>?xhFuD)^ZjFkui&`hvpp)jEy6w5D8HBS<~NExj5AXRW~`>%`p)(zaFZTtNaH05hKZ0> z!~Oa^$*- z5rj=Rd}hwSbYcun#&|BBzNeE0T>v=ZA|F%10}fsQFdAGkaxvz@wHhf5JQ-;T z1b<7Ij=wObjZ5gQ5l%zPxBMTTm0@t=0;AQ0fP=J|%kxjU*IlYA{$p?M7{6Fu?7rt_ zJ?nse`TMA|e@2zD;M+eAP)wi2a2{9vQUW`gB@gz!a-&VM800(2U=aBHg*)93vocUs zZdh4Y9Q}A^b9t{5S_fEayMQ~RPjj>_P7Z7jL??HKzY`*Qm}ZVhC@|q+@sf=zC`zC1 zQ1?)*Ykh?58Sgdb(+?o(2@9AB*fsKaQ@7-laoQ1Q124iYXkd0^_>vy|F@-2MWu-%6 zn21;`|ARpjOw7CpKB_rps+jTWPo1s@JI7OWA4AqZkcr8PuCOb-Uya-KS@-e?A!V73 zq4d<~HB^pihvxYb^H<($OLbV{_n5Kg`)WrIBJF{|AAz_ zcv7d)_5lPra+$$hWq9~q{UK?Y*UD;w&PgZlH|FjJVMXB1gWF7-t$bz zWcC!SZIk*mVRV+Jwjsx{QrZM2}mZ z%!B{q*9SA!aESm7-~^n&g9T&?rM}TL)UmJjX+b!^*!ED9TVc}qkEI!$XI5?V0nZaX zs20xzzi=~!O~dtp=+!{U+s6)MP!VwJj?zx%oT)qUwu%%p4-kegS$ zZ;DB!#p(COpE+i!K}L*zh|$Prcc_<#Ce8upg`cpNyD`%Ge<%!>!mRWO>O4r%pDdDA z0RBvdAA6lJ4tf8+WvECyU~#}VHLR_jf*&wC@62mzvv9|~gWa1Idui;f zv&|k121&5iw9?|Z=Ka-;u+rr00O~MNNj*b|&#dhdZ$nXA#$P)nY9};WS0Ynf*zDAD zm=XGGi6J+;V%7kHQ)A-|J&%dh@Xc(-!H3%aDQ;LxC6xJi~K5o#fHQQ$OE9V63-g&35-`#w*BhIf6}u5+G+vgKxvj=g}>mt+Q`Z#aRmIsM5YZE zF>z5HDLOX1MRwkjOT*wxH>6hDL_)X!60N9GZ`=3Jz^BN>#4pj3mJstQ4g912@hNaT z3Zf}vbbgm{?yqU!O=siAA;CEW@8p@tuo;2}WS%}D&=|w9RI0bN(UXH}V>=bAJEKNk z-1cAf&Tsd?*i2Lj(tjxYhKRHpKxD1L#Kjn@J=a+j%~JlRsx9p>_z@F0k@uOO5bA(L zdd&KF^NuUixK>%IssBTd;eL~AtVxU5Wt)tVyrsf5B;fI>;_bsTgCtm7l}}{KPeHp# z%|&3@c~wCs9S9Lh!SmQ;fLpn+@Zz%%lRsmnOO<5aS5EHy$I3Sl_OWbY*l1A>JMS5h zMs<9iK}pHf=)+mogyGNE!oq>ZLH6%-UA;Wi!z1-ZL7DInrzvpKFM zGH5ZXZ1=vkv>ky5Wpw_ToyN?vLc}P#=*FpGA}gozZEWJ-&=!lTEmn@10`adVbn=Le zY{x`GwIqwpA8>!)DU3QWuSI2RJQEfypX;zK8|81JLV*9?`j-_cHrBynurU4N@6k-7Ircm;=UV=zvMZ$Ax3E4e!cxMzEd* zua^6qHBNL^`C;<2)!b{NXv#|}3FA(2z<=|mS@Ov7fyk7@%4#t5n9+U9lR@|$7CG+# zK+gH}UmuwcDO?ewSb0}rt~vSgG!>st;TiDXL=JpuKQ3sImiIRI9IV78C{Q?O z%N5cZ8Hse_FqXuS)!B`PY_Pmq^-(J}Itwm$#VOPy`Zh0=l$i76`yjqQWe>221=Qat z6@zU{BU)CWt4Oeq|K^dI<&?e`)D5{J4pWf?@(hh_48X;yuU52C(H>*zezkb7`y99^ z_sMYc2(EU!koxoG$-mt8XL7)U(6W1Rc`;O$y#~Q5O@KOM)7fr2#J9V$BK`u}GIlUN zl(f4^F1>(`Zb_NfsxCOS;2S8D;jbTSjy7z?w52i7DSz^fb=nBgy5E}LYBc0CVB?>F zsDWx_Vq@@jf_Ul>^l^9_oJ%w?#S3 zbJhWe0}+b~gll1cJOWerXXFE~{XCQ;HlBBNmBo7wq?p?e5y-nmwX~KEL0w{|Dn4h8 zDQ;)biQh+)UnDsagX=pa(|kyPxuSq3QT%e20#z@?lsw@JDl0&q`NNaU6G!Fa6?b0@ z+Pxp!hD^_x*!?@dS3=*%DORMNH!`ML%dc^50jR&|BfhPG8>@b(|B5t40p>^6#)XT| z7)Rhgui-7^H`qfe#s@57mIrESv6g(s8L;O&{5F|hkEZeEKc4=132atrwr$&ifnFvW{up!4|!b|w{hqnSk@wjyK z2GNLp*RI*=f3E!V_9t|>18UT=&{?rclpn>+W#RYy8!ffB$rL{cmAID71hN_@Ib}4O z7R;$-LtKAbs;>OB8a>Zgdoh}2BMdOmge;h&{)1#Fx>W;4nHh(Y^LRK{R=W2`OL;<7 zW;q_pfB#t6NqCp*5;ek>LRuy>k4(&Hivw~Ru)KWh6vmN*`YLXPUN0$RtcE*YZqwmE z-b}1}E2mgWF_-c*Ye&`RYpPqma9{kNxp1yIYz9@9mVf5k(ukWV{^)nfE{>mhQxANm zl}a_$YF4|u#-vH;dO{r-D#!ZS$>uKQKSa6X>cvn}3(|4z(4^o_u}-7%|9O~p!>pEO zj}pC{XrfGow%f%ie>Bp7{nEd>1ntkRi!07}0x88jURui$@FR~Wdh8Oo9omc#r)Tlq zJ^$I`CrISAeSPptv3enw% zIjV()nPLbA&*u6=*5zIIo8O#)Ia$Fonxf@64dLZ25(*t5@fH`S38*GxGi<=JS`2nq zk9wX%Vr8z0DQPDM(*vR5MOfb+C9$M+j*ev8sUfG3m7f)ndVreK>@$^Mc&i|PGEIU& zy&O^2_mlqtHB@{ry4A!mjTzGmXO-71)P>exnEoqGZ8UO`0_|BZ0@>y=8H_EH<4HDr z(?8-L7-&vOFiJZ#;LPRii@FF~q!}2F#&X5KKz{|;8GtNS@hcQJ!;qHaeBP0b%6oPH z!I077!^HDmj)B!|rvfFNjhi-CiHHUl!Zf7rnTA(+cuGmWzTL9llNm2l8l+{1tSJE$ zXdad#mzW;p5sc2$67KyhW|@$PGqEmPO8p0IX{~L`bUY8cJfQ_~>)3ggQT$fdN&wQG z!K?Bcz462nPbc5?fZ>kml|d6Baq*Fe8>l+=lCbqK)vB%4fVCRpfq;q~M{P;)12;d) zj?|`7z%ttLu7^ut+gNd=JD~aFVff9iAP9E!H_!l*dn~l_O*fy`C=p7Z7(^1Mv};@sg_W2K>k!WVq1m~q&P72! zw?}2gmHl*Hf8|J!rdkUfxMc34=z87{NEq~by3#DW%Ox$)1o=SNOgA$p z(m;B3{-}1bF{ZTez=?;DfSC;Vy(BvAY{*E3;i!&C`1i-tT@c+D{ER^xC7zXpPSe$* zE4YHWO*)sI05|8&DfE!EjX(DPyp@Qixj9Y^LgkmKNyW>1u#{C26X;1|`{QC`qC92d z=zK{#&H)Dr`#9L~bLq98CBr|d_-K7T7*IT{T%9d)93g80U5g{lW@EFX=l5u{khg%Z z3+#JCHN+5q3UZp=gozbde#MB5wJm!YJzjm zb=miuFV#-w8#H0k%a|V(BvOpZebn*BRx}^O4>L`{4nf6XYgYsvSz(G7xjaZ8XRH#W ze%yUXYUlL^KYcfE0h(t;&2o}yQ}?FyOD52ef7c0Gu6?Q-6ME*j_!w*zxHGyezPtT! z=YMni3b-V|lHr;`btDr5d{U=o*d1GWI3%q4KL6fejj|4CS@VAM;n+!?FiOH&)WoSn z=VkL6=Oy?R$H=<|*Y1rra#K{R*P^0B-$Py|(el*seaTu0wcy>!G{o0_&+I+sfpGhD zQa9h#Ho-NlcyJ}~jc-km_U#9bBVzWN$zKxT({mmO$1}W&w(ZLR=gDA7oXb#stu8!# zORKf0TkzAgxAj+7#Kk?-X$;CI3hY(CCBSlK`ug=Q;OOOu%Or0b$LD$D%feHvYuNGz zsrOn#z9LH2)1{X&y?pCKv;v|xMaVy9@M(-)cNqGu>G+j-*Jc|hALHcf!ilu8Atf^H zLO#Oxb5GTfK17)Q)gT$6m!CP6td`e1*lm>eaD1dQgC11V5|g@e3w87UcYdc>TbWg! z0%~Mk?qx3i;r#vbZ8IZ&uMqt>RXX-3byKEnyOQ)VYm06r6e)EVp865n& z9Uc*^KFmsFom#~gW`x0*i(&vq-w$y=b2Oa zd*yv9{*RxDZ;+DW${s0uW#PD-&mJ7Tk$fYuKP@u=VqZfbo{QU`7mI(M^ z+d6c@!|h?5J4kcog(R#tK}~~m={TFmC;%leggZf@`WAH~D%OGJ6ISPH8F+{QdT=7o zRWvYF%~|oTi(u|yl%|93Vrwl&nPOmtMl|medFfq)=DX5!JB`=rPuUfFoX;H1hWY&C z1ZLnJBNDW#5WyUNQPwgVfJ{LBw$ncqj?|})B>*t=%YpRnXj=LB!%3cdbp-ugLVcfMJ z&%0T3_sU)UZA^in6Zsv63ly#Qaq~YC6#8peT;jj^YBXtwjMG*N62+K@a1t)<+5&^z zZT4gqjeF2VrIkfA`zew0MRD@aPat1m`?;5(l}jomm(%fbBQv4rosc|JB*VI7U%+5! zeeFD4ecUP57-vPpqM#vA+`}AXwh2--AdxlvUOZ@z$o-_A7Gv~pwa88$+cJ=0D&oF* z#!^2BQ0c(|uB%c*!qQheS5K5I#exl|uKY@g(lV6&lA3rtYr;cP?5Fm@v!C4_5nJh# zy1$64=ShJ^z*z^Us*CadFmc(SZB^D_6~5QUwLg_({v@&~5JO(KPaq_IBXX3b|CC<( zsaS!+_5E4+)Ar8r6)?uN>{ii)1Mtny7%o@e1`7>Y0m-0TShYUp4a)Mnq7`dhPS%rW0cyJG|TCLm3$e!2CQPWU&jM`1B)p zU`3O*n*CS)#15a$<&OXmRER;zUi;l#A%XZj*N>)t(Kq>tJ)%3AI3zAT9=QD1?z2ef z95Ro~^LSyK0^c1e+#e-t?-oy?Mn^*3^k7!OpTgc^R;ByLA(8VT)Gt&;ps1hP2FFFH zq;AMs6XD4glF!cnO|V*h9B!oWTQA`reR*uC__73!67JC8{l*V4-{$BC!opE>NII*1KnK9ozvl3eLAfuE3t zEq_&Ux`%oNSH`gN!X~))ok!e7C1^h>oHg!ND{&4M&eI_u(oT8&i2?g^ugQ9;VJ<6Z z!PS-iTHv$6lB;9G6V~jZW?TJ%S5HB&db-{rX+Oc(8AAWQpLO+Sz6AAo`m+)snn~=3 zczW#cA8cwK7ZS$}69PdPW-Z?DJ*0tf&#Eay^(THOM&*>6`Dw56Z!NOs&hedNzat(a zl>6i~_qern4|x>-t2JMs%=?On@K~jCqhj;OIHssGnegdZH`w(FZ+QK|l2o0?5syuu zRqYf9r?5zQ<+YyI5cRuU(dpyAaIZuO&k~^KC=H^CQ(SA3z8P_jHHVVPwN$_^%&(Xj2a)von z)~qB^qSbh|I2R4)P3lZ+_A~_&7;N(v?-(US03xRW0bbd&kMn%nb*sMz>I7SO4 zOXe)+bjW>~Fj%{}A-;tXHpkj=+gPr*1_otF$>5vVG|qX^PsHJhKB1HOkY6~$s_Jup zZTe|@E)_*NCpZ$76$wInBfpD3)M(u8)WSwUt4r+~OB*Ri9vu0!t14ZilSnxl(lTHi z>7OQsCOe^*KxMYH+J1~k{kUZ7%Zs9jB}lJ40{6-dMNAy>$A&u`AFIgYZJ!_t1R!N; zA3z}&4Ypc1Hn#|M&W0~>d;5XX629~r9{)HD_NIl3qJLuw9)Ddns z<7D2CeI3POM~Z7lm-1&hN|EDbRu+{6h2|&@(17eX*iou0MGKVeUXM_6%A)o`my&bw zA6TZRi@cyM-Dp|f$0wb*;bdbD9@+X!U|J^ea|WYc8O|XHL^Vxuo%5GcnY_MR)#mmV z`@ii4e-gbkm7RjzJU>yIk5eppR7?01VZY~J981vJre5z+D8P3w>f_rtE2dDyLNoS+ zB=eBV)%bu>(r0p5I+j`1NwpT8aS`^Q|2T_dl~SAvt$B4?@h`nMuSU9#aG2u+cyU)> zm_vxt)hXYGk@7M}hz2G-aC~VF~pG>Hojl>7u!&tx>ANa@T zASCAFE&h2S9m=;6vgnbwPGta=c-_~=C_~Y2F7}#yjvMB-2NqnXY;OfVI~=<@&Y2-* zUK^zZSW*!;HW`&2SAp!UKW~H?xenbhR{bc~2=x+jwte!}h3N%$Oz*QeJpO|n8-fkr zp7SAMSpb!?Q&WfSECo*@CE>8liO#`@NeAwTZPpb+easeep{h03VB=F$ zoB|UOP&XD$lFaOjJb~l6mxJq_yi8TYQvcJJ`^y3Y%nfFh+HN;mJX;Mjd?9jV#^o3D9RB8KzoC@9>tNbs z^zakr%du*#QM50=j-B*m-v8RIE+osi=Si=%QH)n3@lr#LHTt&|3BoHV(cdOR_&(@| z?UYAWn$(lzrso<*o9uzYfKF!3o?vqnEa>G@yXJ1GpVXmZ(;pNl&35r@05xj$FS=0d zupgqOeeTTVuQ?rVT9bik4ocK<|BEhXNC=`1Yc#m@8g1~V$UP3_!7(k+lFQYK|4AtP zW4jAg`(Pt>|GVNDy7-vIzdZb7c@d2VlK=cH+pJFC+q$ONc3PQ22;E!zVOwQJ6SQ$k zvVN9Me3Y1j%ETmUVfCN9g5_u-#(K-<1y z^!K+t&?2I_1B%zlPg4~d?|kzjUj6msws-$Z@m5P{vOOGNs#-ihk&~b1P|N1@XJ~P& z|G99Q?Xe*vSJ^adg6->(m&kf`vggd_8OsZWf@k7mc{Q4eBs@EIusmVg^%Wgcp5vkZ z*CW~ahgb*F_z2CT$ev4;0Gi%S`ro}cg&f8C{zKhwLuhbfbkZb{Fmz8yc^8?I&96?Q zp)l9)9!c^2y0+DtRe~uLuCdc9A;OM@U;hG&DUge6Vo?0sMgaRUr8m+}YxrH->Ad9~Ab1s~MwLqIk*pUNYu772`eD(iGyAx=ty1oJ6 zt_&9`V?>6?lzAqUhz5ns5t1o$GLsCYOhrU8B}3*YnL<$%DKexGC4`7FPw^eM=Un%5 zd);$S-@0e5_gm|E_V(=aJOBOP`|Pu~`=wrBN!X3&_N9W-@w3A{yOcu%>;@v$F4OgR zy`7m$zuX+`KK048(D%59h;OSK>GDd6+>wdI+CZ)|cKP3yhrWI6GndUE5X?TpTaX_7 z_PWOzlci8=dG=c$)pV2#GWkQj`Dl(g>ZJX7?yV|*?{^^*WQv|28{FruE=ZB1npc-w zdAMTgZ^`}(?>H@bCAN}crB`RkzrVjm#7}ncB{($ZdDxysb~H?q#IKB3AzmZ>>&W>% z6TeW;Jup0KW=s1-TN>$|c9w6iCa~Q~`kYtdSepLJzhz0)yAiE z#W5)FH)M7F+Ze23D$ttrZs4wKqs@M1?}=WTKMk1*9EkZZb(wkK7yF$iB;RsObUrSK z{e@rqC!_HJR zR(LH85#QbN?7yc)Zn?NXT@Ywg29(Ho}vYQYky`HVbuskv98(nQI9Ti-ghfPAU}x zfjb}AH1~DHk;=QNX!40Qp$#8km&7T zIPPS_ZgkyoM|SzV#M^`5_~?L5J6(42!ON%u%1sS|gmnxWiU8%rALSjR6mRwfB_(l2i&b9ffuap!WWZ|l0 zWYJ3ZhW*UgPD@4i^RG?uT$+3Ky;w`jzUJC?o9wjqu8TUk+tXBO2JTE<^10Nujz7<(uPu7>r_&AkT;AwrR!fRak>e!TRc5!mW)^`{&i<&1v2R$G zE$4H2%YvTWNj{?yQa^g`=spqg_pHYzPUc&Ax?gVltmwWrDwnoh?#RG-E#C-uaX?Cp^t9kA>Ab_=uN>E@e8YJ+N%U>;6@{&(VRhD{!{F?W0?p4uW=!8M ztB}W^J743qVq`~1{bwMt7K^Xz_!no@k$lp6psCfEO}f*p-MZtiS%t!go&@E!cJgY0 zV`s{>aw4x3&w}%c96BqJ#Td3VUI;8jg zt8#-ju zPd3%1PTDK3&R(oo=pOI+Hc{au<;Uh#u)y!pe0#S1-0A$Hmj?0NwEQf5B*~6ljT(?TTW>%-D5?2+jO040<@-j>6SnP6TL8w~Y(L>2d{W%-U%rID9A zZ}0ecFM9iD{XY}!xARETy`&t=cNQYkZ^g*Fc{;ysej2|*X~6(Y=z8Sy#MRs#R4puL z&1%+Xr0IXRb?Ud3-?Zw~-$q2X3XBcWggNbcGVwe$irII_>Jp8>DLn@dlQdqw>*^1s z_bNZybE0$g1QX+4R2MAVr$7ymk;n{-%9IbUkR zks@DC?=}gG%0b!l;2g!n4m)nDFwF!t%Y^Ozx5$gVj0^Uk=AV|1G+s+td^Nm(?iFS% z7Ru1oE|}#wQgF{_fpl^B1#nmX_5y@Lo}bvCpA*S^KaRuIA?jX240=SwK(R!3_7-7G zzE^1Q`$w5nH6~v=b)EdSbk=!$m2~Hxa-C=F2q)Y^XrLQzkge*#c=_Z1iYqf-&i1p3 z)h}`>?}nELWCb*6Hf}HaN&aW{pmgLQ zhTC1#4M%?>i1M2$1kO(0Ssn}g>m+y~>p-Zcs$!#wORz?L`{Nt$_y=RJfl2+J z*9=#hmaVcunrW+q0=r8WkFCwLi*^5yPOK93ks4Pa<%oLRx_igh_MR)VV;R5yI@{Wu zr@_kjbzioHC<^(5q}w}SW8mm5)U4Fb48@!rlcSj@z~|c`6FOl$+r?s9zsO& zS;ahHDwcYwbH=mgcH@89N@;VvV{-Tf%KXEmBgRqjdd2Dwxd*Kioc|V9hL%MK+N;J2 z`^cIf+XP zLCGvJB6{IFPR9@dW&T`@rgPW?ZmDx4Pfv$^JFOUI^g&GMNJsg=yh<$U0 z{^D1QojYC9THe;EW`pmEKDej5>3FcjY<;Mmwd44qs`zcijcF?Ne;c8gDX14e%m}RV z**ktxt)7hk%uZwR=iopn#^grTEeUzbOg;I-ilX$ixh?_3bKp40i8^Q3363-R!Nu+M zJB|1hzoflZn9us_Z66w!pR6&oH!<(py-hKgMzF+SudEoDa=V)gXtq&{sC+r7WS|yN zcUUiLYE}sx29?anI18G^@#F@|zDmCw!xVJ(#N_{&T>3W7*I^zubJ9 zs$;WB*WZ$&j-vu}<~y=7+_w;lIbK)GjZ87h0K4B=KHkIkZxShQVU|)=x_clo_dN}; zu62PKm|g{56BV1x_2sI!8Y#{0HqLSu5B6epkM1ur{M#h1bRN&vgQtDDs2;cT(zplg zk4PD>((?JQXZ#^m6?!L3xo_TO?)nEG0a414FU>55U{EQT*{8N6`V-f@a?p00*Rm-G zI*Iva$7cTa`eq`pZm6iXLdd(^Z3_hLKGtf8s!yz=$ezh|pgXZA|&h@bPYYoNYkac;-z9HD{F6Bn}})Jrd}(O=a>_jq<4 zy{W0ps9UrkBce9bw^A<-^eU)X+)$km#PLb_t^R?EW+9q~e~vAmKk~P`-{jz(FG_L6 z(03b6aL~KJ^FqB-Q)6$z`|qBupQoc0URT9KpuE>8|190(s4J+)<@dN;!B*SqD}ovC z?!9Gx6go6?pF3xyPcQK^@a?w4P{n5thq3|U=1?Yyeu=gs}QbhXfLDZ+1cD5BqVbaxkd$Gz#lXJ&72DlWeE6Zx+U@UR@e=)o_nz`x4@T>OyB@Dw_0JoXPX2!NyVRA@=HMda z>gkI9f7q>@E;yLEvb$SeaItZ6+{Z3Ko~+xL!eSyz_IvxYp&kSNR zb0;@P*R>K5_KCVxBk|D}v$b^$rs;@R2`J8AuyHWE;3Z^Z;YoEfqK+|}?#J1%u)>Z& z*P3eE(_3W|Bds0H>23{OlA@vk8cer3DCOlzp8ARm&RQt#;Q$9yA3;1yLA9Ei7hfCGM|Z{a1Nl5J&XuX-WN^G#M2-*mlBTF}6g zvm=2wF7g#}G289o88@dTc4~81^v;G9!o5mj8J(sAMDe&7Ed3M(LRDy2!asFXo87Mh+z9PmE z&aEd~QsUzYIy)Q5!JaGN8dlG>U%-SClmp?(`R`AKY)4jdT6aEFGd}X(do^7uy(_Tn zB;K%8p%Pafdab7}aU-?wFS`!oh#Ro>@=3mrK$_MrvphHKy_z-Pd^yvD zf)Tcz*6UKcKs&gfI`9@@=;=4}Ct%%F4}_je7U;QvYoi|`@}1||05@xy4}_izUCJLp z^KKJ^$4fb8wFRRVm>sFO@WmuA`PT@YyoYu516djvAt zIcBs{!I4lpfMY**XAT2wMCMmdUj-3CcmUC@hrvj$o&Z6*Y1f522uVwr0>6!j-0m0i zLJvY<87M+aK0lwfL=)+YpQIML+~d*qBH_RExh4!N2$h~#hy(@UZUe9y$bOzod->$D z={HRz#YflPOf!Pzhx-e~*(0t^7W&SF?`(&yYVh(y%N_dYSn3%YXTRSO>NB5Q5y;@y zZF6y*x`_!#x63M}pB;ql=YA*u4bTtnAOyA@hMp;)3lsjwAS5OtYr1xA^z&k$MjG@W zv|b8Ikle!$S_*0}2fMV?>K=8m4DAF3^>J(=+8BhYKQYIGgm4F;^_n0f%Gc}csz833 zwwNsTRhB}0g@uSgD*Ri{8*5V1PhY&kMdRY|LZoI@AX{!&He7D+F+%9e^%Q~pyj7G# zZ%DS7aVM|n?sH7Gun~2HRHJ(`+z;S->tQge0D=(1KL#OD5mVr|5z(7qwnrc$e%zVR ze`TNu@k>-*^f)*xRxEs=?DPj>dDHiMYOAZO5w(=3w*uqM|KE>oxu%u|62jdG;NJ=v z(3K<-65#6}J8#{q*TyIEU%&!XW-_05K_@5P+Y2~am&f`TUV!#rz84WlqIdR@gL7t| zH0copGUqavYtJDPEoXbHCHy3l8AVTlsI)bN@^dF?2m(HIVg)_yoF8 z*1e5?64DP`qRk*4R=cDQ6gCvvVz&_%bua2H;BCVols`QALE{JC<57T zr+nn?M?O8dDCq_BHXsLqI|ZcuhV3|XJz(2m=y3t=hxMjsx->9qqgF%*w9g=VN=CvuDnHA^3XL8#|G9Y#2#V&=E^%|2hO_eN*!A5j!*Wr2)5rpuKZaoY}xz_&v-v7!-Le^9i zxHg6%hIDfWFbqLpgf8U|qs2q1jU{2y&D^fVZ?ehLK7xTL?lsn2klMpTd+<<%t_ZRs z((eg74^ose7)K>|35?%9087zyd%&GW5_i_uHMPCL2g4KbQq=zLG?B26uQo|pejWar?23t|(s(~6FiqNfxp(p{kC))p-2}uBmungU8cU1=s z@#79?|CNCf)aczjQ}(#7apIW9=apIYrBU>3;wvmcr^(BXfPx@AN&i&?*-usI&(A%E zKILpS^;@pJrILo_NB%BTS+Yxvjr4VdY-)aD6kdLIR1kz)^j3Wp8y+Wp{?b$$f!uYz z-&AvCCZCO@#eiglq6xO2Hw8tVpdZ}BG2kAfZZn3Sesh0<{r?z)#DKY`wQFMx;s|5R zg5H&_m+~hWYLsv`-kY-g(yO?}eIeU)KYC7x9=j{6`9iA$62je;t=9w@(cJr?n|U=a zmW0fC2qm^;alu0L?eLE?yWf4)GQL>z`;NG1H(rQzpE3Ei5`9!6(2T8BPd#LgKpr?- zJAXt+hFB!{sA<&l!%EnQ(ha?YKt#A7!1dO{U{vv(0HL3<+3O74`7|)s1pGE4QWj(J z0TJO2LSPvvLi|flE%mdE$LN3aOcaa=Cy%#+!?B0h!?D(HT-6{U+>HR%0~rua&Lso; zgIhe)Or;6N&r05f1&G+X;@3ejhYQJKWc4vpoV|Df+J9&)`nP{>*W(JGF;mwuTe+>h(f@%5m3Q0Q3wbK zcP6w}!kD(`L@g?MqNF~FRqC~$3L zgw%h%9Xg}668@BeVno~N*RR(ms&LrtJq24+ksN~r_<~~}c4tOUdes~{qxG5~Ga5ON zy?0NkCp*h^iq}Kc-}7NH3XmFW8xoXA{(j;6FQ>QS_INS+)f1BNfJ^prH>KhjPZXCT z0_l)8X2E12_UzZkQttt3@;TUy8hYhrpfg%;Jq$)hK#n^4kBr15M1eRWipA(iJAW8- zM(d@Z7>z#rKy&wko5t@G(X;*UCtneRZ(92}v8Sj>WmsyUGg_|+GNa2%H|SsamOVWw zPg%)(FjfQ>Bb(S2hU0y=9}zbYv)!>)rooF5BmLs}ZyW}<<6cY0=Sg|AA&@K?5~tcm z?U%j{#hsbH=&}zsqu-Gslh7Hhw;l$gC18r0ZPS4n?r0pfjE+9Y|CIB+kap+A#S((J zYAaZb1k!)nw=>h-Ts`f6*YFfI1zwEA3Qmm*)z(svNt|bM89maAKn}({f8|G6q3w4z zI*!)N2>xm_WKfj|K8(QD!(>!LfH<&eG{T))OGuakzl~j5E;mmi^zl7d#-9!?-;Bnq zde6$t58Vep&h8HQ4vz0rUSN&yk5(!Yf`V{&>|iyJ{nURdxhc+1mi{)hEZX%7e=jUQ zO}xriX99;#Og&k=xBp2&6JCD4b}#J9<1YMMGjxSw>#e$@2;^CD>nq<)p3(%|bQICn zYBqu$h8Ax5p~DdF%o=Pv3_WeYnh%yuGi%&>WW-E?-$p$%N1S}1>jBF^>5(%@?Mbv{ zu==of>7ruI<6;vqEoqJ|=7>$|{<^o+3dWa^ra zKCoc7-0vCRC^kBcz)MdYyOW$=?c-v<`@rUWkIfem$i+jSTyvK6h1Gr@xcWqiQWv(K z5v_xd!RUiq57>4XdMM8N`?GJV2X_}HDh{mT0#C)c@4Z0q%8pwJ=>FDxPgI(`v72GruKheq?-6$8f9vdraV(5Cn zw!_e)cE;bI=pXfn0eLz0AS~T)tN?oO0n0$?2{Rq{UTNuNTb(jBib;-7t?K}Tj~I6F zk+>$R4xJBJ4`edOQ!s0`{J)o~O?+A(DPKhfDG7lQ?;&V~) zyma$(KBbs}nuDg#J1Y=KK~qgHv)D^ce)h-NkLSF+2b+&|V9r(Oe89HD;1dlb&^tHf z!;7=u5fhaHUXWpjpM8iWT`>GWn3_hHf+D27(pBK_BcC?9`RuB1Pwj&#FaVvujFm(i z`WEwhj}Ll4JY^%H(fEFO}hPd@y-Y8I9u^AYKwhU;ab-a30f-e+$o!pqRe zsZbBTzVhG3CGCu{ifMz;Aqa}lrJxigOtmaErrix{%c=5M?sRSLEj=0>5D>5> zn7K(*apMb)JM>Lgh5Yun%8^mrk>G{IuDS;GhZTe6v{*cLn(p<54w zksL5HJ+vt!+=;X(u#O9R*_qqPRcSC3@#73=U>PVvK9(Kr>9h}T=~l~jrPSnI)QhIt zvSkY?l{8WeOHj*B?!%xU+>a7i4P-w#ip^INH3h~mrqVePo7HZE<;N&TNK$lW?j@m> z>7kWh&mZCCM|qHRKP%(IP70w6qu&DdzzZ^=A)RD`SXOPP<84g_!$&+}cV!dr2kAgR zxbtYR?J)GvTKE$j{MW8bT-sC&xHbl%c@0Hz&<}(lgf8WeARb=r?o?S{w?{iiiAv`W zl!IN_&seOk>=c*gMNklgt_)ofWJ4@2KW>L=aA|(^=@>FhrhN;0%&xaaM6Jq%r@zluXohVF*ep5@G=%&4 zTf5PJZZ?J@pGE?N)4#sf5RnABGD$2&Dn~2J@KA&<<&PrQQ>jWz`e!crHv7Hm%?`*0 z6`jXcq!GR#frlb=MUWK*2AWEpk+~#VZ%8h7nu@F%mLfAN;)I!35j`%3Z^*NcRNuf$ z(Uw{HdP7-14rZ~vUyM%s-a;TFUQrEBm?#aYuO@ZIWNF=pqiwu|!;qJ-6GEjo}trIgnKVO>s@s8w&2APjl8Tgw1@;>ZY z#c7cqc90P6M+vM4GN8)zjPE{q;srHr68qkSY}bYbh)z(ODk&y;tY6iiY)xifmzO=1Kd3FwO11EKGKfP|U*yM3dGu0|LU`m4R)C!AINL zpTKR?t_(LHNm*0iw~>!B%j$FJe84hLe8Nt>xKuUT*;MI9sDJsnzOYFzIH>Qy9@O79 z7&V5@2doD&ABkA8v-Q*afzf+J$pRku*2#|?(@atu=%jte!dHx57>4Xe2Rb%IP5m%!;3oy zmJu}-1+I;N3Ze?!p!Z>GCHyG`MabodW=_-R5`EnizH<$OMBB^3mG>X9SKePLrey#H z;m(BCD}rn&V5uZ?ywAQnoW!=AfXyQiRtS0)`DVJ31!k& zEQ&|cAnXGLqJVitMor|BuJa9dSCJd@KkEu$8@m0CIs-I>JAALV9)_ZB;GP(5+J`|< z1gsGf{mVKb9p`UBcql@bf>P8!r6Bmdsw{upH9z0da4?ev%!J+nz3zt35IY?h`|(hO zt_ZRsEn}qpO{Q~CA7*_Z`t2+hiR-_6R0dp#}0P=fURiCRm>R=Md;SUP&5jxeKpxM7UAy7fUk>y z-^NIk&q8_>dRGRPffA&aJSFVbA~fQ6?T}EiCujV7bXV4j-Id)Fl`sMc;eM3BdLRRu z+#fx9#lJo4f_!W*34a9vEI{9hrKqNTyyt?+8?#fJ?p?$S&`RNC?Y5Hhy%aQeLkun7 zwjz+n;;FeVDt{{~iwQe*j*%<_HXsU@STrEqnGo1^7<{6EuFPgrJ`jSCDDV*(aBTz> zkbM6I2nhF}eyxN*rJx8U%-P=UZSF3r$Y;&IcD&+gcqTZr_qk}`x&(G(*B@AmV@ zAdq+N^E&Oz%j>u)e(88vq*M!RM*gjnsn8j%w;l$gB_Jg--83b_-I+;2ZIQtg8KQ>FQO(UU{^c6(Wu9sc_&n#2Knz3cm}wZ}n1xH~hj9>{=J zg9mCp-13R1I8ga4n35abhxG~d4Ix$^&zWZxPO2QAQk}sIkfp)zTXttIvJwT(*ku1& z0rvGsTKKqiF;|G1;%jA5QtjF1*N3JbW zJ!wn#(Rrql(h*qU$F%a=TmT8eGn{}o;Vb7`GQ9L~=lZVF(^qek%+g(&s&DZ?AWM5+ zE$f_idi~K~KjyHkoH1-Y=V;FjfZ+#1D)irN#?TXO=}&O#AN2rh?f6U)dp@p;C;s6BYwu8V9Ta_AY{_Az%QcKp6I+!Y5_^xZY>q=E_`)P2wv>6+A3Z(A)>I_H-LMDx_2{Ix%;gG=sy5}{& zCokW;_lVj87NSpaTRRtn?Bv}g;yAAj9h1Tfk!jek(>(VbKOLv8t`TEOEFIH}H;rY14P~5EzL|0c)nQmwKEex>5$l zAqb4nrTpo~Ix;@+j!XQWTk(JEB9*^7p9>Di_F;d^s-BY9f(IjXO^_Mw2;~CeRfmaRqu7Nd+ELKLy$>JPi_&6oxIkDb7a(O`Mn_uJZ>nG*qCy}aRzs|X~ApXQIe zxfI{Xf{zWuvwX*3GYZHIQwF;-+-bnto&IyTF&K>k%NDn78jWzLLcmVvz;9zLDhb*m z4?PtE%RmvLaMGn~<$t4XkofdV$}uxrMR2JHqp0G>A^o&FzcMHYcN>7!K=xxW7nR_C zZik*{{h@aOCuG@S`B9fY8yb69>ISF49$z;47Dl}M?60mS_2wxvxx_F-`MmHZAp-f+ z?%`hGjcrcS>k9vTY|G;mfo|DIvObmVwgaV$$zo^(}R#GdwqbQD-z_7X0q;@?PwIO!81zD0DtxJ&^erYurySF!vp@ zcb7GHQGJjHi_g*l=TY_)Wtz%9_GlL34EzT2OQXcsXjWTU74M%7w)jG&1q|d>ZY~k| zK02o5!ttakt;`WNpRECE=&ufO^8woqgO4_lN$YOP2X`MPCIT#)2Cj{Oj6JR-fq-!L zVe6%!2$_eQoT`{ zBHU33ECWS||A`hazxd#Z#Tw%UJB_&2CG?gaE7-3#%$EqCfrM~30$2}ZKpzRNZ6WO4 zu_91l!1?3J(+OCB#ys*SD@t6KEfBn%nFB$m@d6a2GnFI0BUJx*#AQkyezhM6 z*+nJ09a7#eZJyFa9)b<1ojj@%1cW;Zfo+GuX9`&4;rFjmNCJ2@23#8f-Lo%v00M## zh0vv-2#tO{cITF@dw}fW@qzXO`j*=z!4K#?u*dX+VofhWLJ*=5x+chojA#qRNX27r zYu)b==qpVQfQ9Hfp$f;cxW%?4PkOO7EsZw35KWw*{!#20$1cbbX%~Bbgb0D$#%KLJ zX^5Ww8*NBiPrpbHY(&XFJZAfotc=psTgo=WK==F zstV@P5E!9LK{2u;|8hN7I$6?1S6{{Z>J1fBFq{4az~Atiz9_${4i81>iXbaGb;80S z)$zN;%nN@m0(FK(Sc*>PJR4n7aZfODX{ED>=I6yr(USuA++QoiRride?rmA2IEX+p z4-4|9FmK68s!^&bh`K5QTakuCJ^I)U2#V0HhoPvVnE)~OkBUTrPpsFjjf#j~e{Y2z zjMhs*DKe5?>a=ZfvM#Z>G7&$RU0z=f4(m%qu?A%tlMC|D8Lig@nNil|!#&Q5InN1n zS_Hb)y5LPQs?^P9`jE7bUi1E5>^0nY7%xV?>4wddY;XM2o_j5+H&eSIkdlYT$u8zk zE~Y$<^dhvU&4kV9s;=JJ5nQOVqi{1?Z#@h~-7g6c-J7OFxPy_XB(P=(xHdAH_>^Ca z2P1STC`RVT79LJCC(++XYB|7g+JbiA9JqDx2zFZ3|AEUC4@T&kAT#pfP4Zl!oTR$1 zvfsaOUl;;bG}?jK>J~UPyTI{YoQXfFI2A8Oj1<1f$3B&i>ns~z()149i$LCZbiU|~ z{=59cA{w$Y4TR@mGfM60L7$-jff2g(Fc>8OeOlYUMk8QdttfD9WR$Y_`Z)AHZLNeq zrJxvT&-N6SbLHffXwGyDhU<~u0~at31F!Zr^l1dbvWa*oLRSP?(ca5F<5PQtK1T9a z64aIE!~3!|%5pWn?@~_ST@5Q&?m6)zcqw8|&^;m1RNomL)?Ax-m-P{_FGHHf)B)XX zY(|T|=387v9>P`>N%sc*x(tFMbn9U#8nyEGcl^f$TXbzxS=6qw|NoplBkCt3cmyMK zDJVtV*!$z_l?Lg*m$=;K2x83W|}7 z_VxvzXFoFI{U*y#P<-C+iQWJ<0ecBrN-13v9?=M06J$ma)m>NbTykpQws0A-(Yx^( z79$atZi~nkRyCq$`_dmMhc4j7D52pVPpyLNNa)!HlhIx1+TN`)} z-J3Rn&1m}S1bWI5LNr3R9tNZCHw1`;f9=y`foKF=8yQtwCoAF6r=d&v)2E3S$3Cf< z5$g|M-4arN)NwbsoccWWc2!>u7%lNogsup(BI97>_^_&#w zCOx-JZ^CGc`7gB&yc8W*&Rv|8XXZU)cFuf;`6(F!S?#>CuOLe}PoS}rMKgl=8f--u zB$;>P5sc8ShoMLg*aP<3rU^FgIY(fdec-pzkSF)d888&_1EYY=PmTr4KnarT%A1KP zVkg(qi==%ZA>5A=SPx`C&fm6G3}kpIQ{P{luISU;3JVaa z=7=gQr>oVu%eVL&Ttvq40+jOmeFN{c*SaT8RAp${R3Q<_GUk~$>H6hTH$sgGT04IM z*nn!^miXQa>zA7NQ4D=`+8T1J@}Q^vnT!Jrymqt z-UAzv!6|+`_QPLqJq$((z--afzxHLKz)_pPwULqf>pO~gFhZAtVq|9|J>vV~{#f+q z=lw%Yr4nClz+t`;_SCg&pqmgFiJ)ZG;NJ_Gk@|}rSGk%Ux_-IF(q!>pu!hA5kv%5f zIlAQ$?Ps;$3NM}SN&)vpg#Bbs6ue512!rhrkNtgn^jfgruS8YH2ZILYF?{ z8i54u$%MvBdJfdo=;hR;*o1Uv5wsv7+>HR%0~wI@)*HuT^C~5Fq*(^l%Mez;0`$|# zxGSdM8C4Zon^sOg6@DAW)Rhn1dv|tJFELhUi%80T;04+;?f29loiSFT%!YxfEem9@ zH*8v(UWub|K6fY6ztOF+$sRD4$+7|mYewNTRCXf(>D1@#FGNQ%oulk`5_s!?+rp+Q1?2f`h zHR%0~wH#|EnL}3eNer^!BzlmomaT@weV7 zgYk`DK9S!2zQ!Y*RjYUb>ewDwyFIRaN!KH;Im&DaI34JX+GrkgJH0rauHiOU-ZU!M zQD~@SQ5pnANGhW{hC@nbUt7`konjw*VpZ6K2{*r{(|O;nHx8(_+x$2M!SMvH^y;i z%2Hp)2mD54PDeVv1$e98W+jlUrSG<$0KSkW&`|lr<-Je;NNcH3PU;!hd=diYuR`Yo zwjBl^3TuD=^PBR)-HC|-OC;B>jexR=M7=>kxI3}+Qc#4#BKf~`ezJdP9s7#>ZA`a$ z0eTgPFm@hvo_M+fB!s&YTdxT+qKgPpf_sJ|=_Df_wf9GQNMRx3^inyd?Bge|&wQcx z;@+ZaykgK9z9K`Z+-xsoyuhQvt?7;k?%S}z5~$Xvl~rul}2+fDbYw}YB? zFC7OzaJ?6UwfWhlREIL?iq20b5)}iM`vccTMd6f&^U#CQdMPMH{LQ{)(>?uBE_2p%0_9Kn?-zmT z(0A-~h$EW_y{Zv+Fj}t(G9#0Y4tfbgsnPoaCtNwT3g9h_wKQ$Dr8!Zx?Hft#DfR=> z_zlbKN8}vrnzKEl&qy`x8gLRrAQ2Cn?0fj03sj3g&)iq%)(M-D%JrZeut&okjMiHZ zgHf)zzyH2X8R5>TfsKRyvilHWi$8h|2tUrj9asj6P$PX`nXZIib9%oMePG5z5+j-SB3~u~xDV z`RaJZp(CD8RV+^*dKlJc7^-Jqw*!I9>$rb8E%TR}^yPgqF_F!LumR2Yk>WFiKaPok|R2W zn+h*P`l^pexmr*1e!lo=QQ-GiIUovkJ^vcs9% z+Zc?x8we0`|2Ro4CI;;J0bCn9v&!x9tk7S$u9t#hUnX#4cOU+cvBLAbklo3y4cyKpd}>TGjm8&6}F-ycj@uiIcB}}FckR! z$J}lIXGOre?Z50`C110B26`}BE8$NmC`F@Br#t19{6;D-9X7kpzY<^#Cf0qT*uf|$ z;8rU>jKG>8Gn%Rk__m!O;-M~8q;(r18@#n>mSVyQjxA62bXN*JRd_&k6fZ_ZLCCn5 zpJ#XU=N3A1Wee5-TbqkZC(|TE?02^$57;tw2L2BDdR3A6V2_478(6%rbVlnn zL1x6ibXxqzwh!kk(?b33im>Uzibgc=&6IPCSUYpx9DQ*ZaYG(2MzJ(p)ag3kSFe$k z9go%D25g3wNY41vjfJYd>4D&*tZ?B9*wILHp9CdzM(eGI!N>^s#LhLjYhZ!Xws6Vz;7d>I^o+~ zU}D6NbASexfg+T!RC3DR=U|D}lh}d4`;t>{zz(e@0V^%yQ#tS&6ofk>gVjLxKTZ#Du{Iv{ueVQh8qwLLghI z@}Dg7&WxyAeJ6O|l6C|(qDh2WB!~$21GwIL7>t&HzKrc3`!Z2!8DL#C_E-1<2ODld zXS80*pFmW2*32QXontqt)JR+Qo4GcSkr^#Eqg@|uc0p&fUK37o(r|#1>QyDD`zsSV>s7H3G3H{EpSR=bzrG zoY@(<`&?G#JJ^ht7zDMTGg@yw3`X6+ir9pIWh4fCat2%*qtVubV*X%n2I0*Ix)c$kG)l@bfIO-oq%We8IVnQDLOfFTZ+okFy%(~L6xsxf`P=Ecubj!{hE%u!Xldh z!%5D|uodxLEJyDrgF6_l-RM6z8$(epFz0yeU;8vMF<^j>{r#)dp)ZEegV9 znn)P_n-9fhay96|Xub6?7*znfqs?s^jBqE`z^-e+Z(}G*|JgutvbE) zr{6t?C8=lgMncwaHNcaDUN~dFJJV-TN1x}4JF#A`2{IyswvScyJXK8ZLb^sr$c`4m zLbNyaWm5Fn8DjUHrKbH2=eY0+LWB`_+gityI=fz9V!NoGG6an7>jP^)D^|GOvnDVK z2#k`1-Io!3OGUpu!~FoRw;l!~Ibd}EYSYm@FU|lW27ElSc5P(zLD#tnI-~VcP>i~N zeh?q%oeQ0Gs922utz>!#{OsB=;v`WFEJT!# zmwj4R*c6x3Ws@a_G6wMqLg)Q|(=n4Vk=-kFO~2j~3+zioEWTj#V3w_tgirFuPyJCZ z*g=S#WCD*#qV@ZP!AK2QN+j{GjAVh;0l>A9k@}-_T09t`OF=QRYxkU4-u-#;k=B;& z`Ax)3ndotu2==(l_49j8JQ$&Cg3O5gC(D&p`7;L%KSn>U*ZQ>pi;=Mw$=1OZ)t&SK zrE^sekJI7BXw`%}mAp;=Wl|$qq)?3iJOar*Zu`idkp9qzore~Te%Ot{X5^dRiO+V! z=+?tvBnP~Hy|!szhC3w^1rGXLyEZb)phlSB!3bRnic!MrmG@&OKc)t{4owNFNk83& zUX^wnJ0%+9ndirY5xOSGjQD1bYL0(vHgZBVcD$3KE{Da)cK?l*(Xlc~ZbWkIRyW;1 zycp@K*moseP5khKEbGn*ZEjHnGVsi*Lo4adopR!%yDJMha$qw`bm4jhc4@fNf%UtM z!Ke|KB|84E(Fpj4L=?C-GP+sP>Wc>>bSWrC{2mfdqv97ue|0ga{TlXos|Kb;W;EE* z=FDEYSOYLSTe$Jq$((z?60L zzs?d#n*OCr+o^De6GX(1bC3rVfGPt;$fsjfV(D2@SNd@2S33pS*a2{GrYw(5X!$#j z8Au3sBY^cl22@qoM^U(|(!@w_I)ga%CA?#VY&(v!gu5lejAWya5$q%|JY^$-iONO}rB_-@QWJbJ=8;nD^Q){s8F!+R7`}=$T zYZQ_KUYr5f#wavBmvaXM1mTSbx)cNrT#QVgyeSn9)pY`wSkq&t)`!#{ zm4k#Jq}J$~AR|)olI0sG9zM9gz2;5QJ$HIoQK+;{|F=oem!><#fhtooJ;(7vBzUW~ zMMBf#g}|&^b@#99HxNkCT@mJt?aORslP4NOx2uH0MwIjPIUY;W*6#G5yN$ufsD=Q+ z|Bq2fTG|x&ZAA22zL|$%RmvDO0qARcB?IYLE1SOmuG!6Iv=0_R2e8fVIlNZ3k!nhdnDdkes2(d#tlwXU*^N^#Pm+S%!1Ab ztOqh5A(k*!`q(t7mL(Tiopin`SbXgEpR`0&>g3NShds8D&||`jPv)U7EY_z&DSsS! zD}9<2Zuau;@u@vz*vy|?7(9=!DIKC~U_;l1CNi7M-oy^{P+ z1k!4V<$erH$d_+_Tpj70f(2*a&OF-%LS@rma#=jW!K<(Ef^+pLE2Zab93N|l*y_{Lcsm4g zo;BjI2Mv#5h_=b%gIiqWWOz6rx zChM75q>$s2nfZ+CZ+^of)HR{NI+*UacWWBwPH{Ec?|2cCmQQ%!*-OV(;A8vDFQ3XE zfjs*B(`q97;n(-=?P}-_gx`QoXm4oLE-?1sP8!$l^q;$p!6*S(H&L`{<_v+63@{A@ zTpJlt4@=>(XTw?ve@a0yk~9AK>y=CCx7v7$na_>|iJ!W`?70H_GwC6p-~ccRL12Wg z2{NO)qi3cNe>tQhJ1{hUr$_RG`EiG;e{5B%e z`BfAP#v*>45e+N@MW{_z@a?4+9MN-L4tWDPKf<<4|9AMy<-dI6%B@rt1`@*kEP?ev z1{Ba!M<&Oy(9Y6Jzd$+H{RI}F)|lYD%I^biT}e@y6tz_N9oq5om2-QwM(=^-mD9xys3b5l)sjWTw&<@;50N8dIe5QcTOk>l|3^yOtT0<;8 z3CE65LFWUOf#PH2^?SfJ%xUoWC*uI4Idye0@SFt_QS3ZumLv89bUt7`koo9rjj`Vo z$Zc6b{k(;hG*<)`AA7Qk94FH5zW8pF7qqP>;u~J^r>7a+;>{})JuY7=GgRrAd!;HWyJox-q z28xgQZ>#S=Dr|MBUtM#t$@aVJgN{Bj*yDHaGv^-S!RNnvAoB^^D!p)WmEt!i=kJHR za^x?;;xjCzVyZyL_i44k)nO|KZz*1Ubh3gjyl`?JcKTFv^$p$AVBnCBaP3@^prG7^ zS@Ri|)H{b^N1yOw<^VkS{MUAve1L6voj2tJAqNrx#x}sUvHNN@zRn8*f{;q1OFJC~yhcWI%qvhVamNyb#fp1gjV5sfX+D(d~F|8nEM#&c8Ae1quePjf|RS4&26r5xNu0$QbW#&m$d(u>jXq#m|vn z^W69{&Tl67&QW~#4c{i2zM?7jP!utMnu(Pw$mUYew?W^SO$uaiyTu? z=x6J;inrplQerHp2GNA9un9SZ+2b=A{omgT84!nik&|8-OXkf*7HKvG(mq&#bRDAJ zCa^`XocujQLnTj*-&WL?#ZgTJS$-KgjH^*xr=)9vb!H*=4c~Fe`RX~!9~;lSl>{45 zmO>=@!+PAg5O|L;_=p3!kocy2aCc^65)#0QX>35+e)>!xAl#kVdMPME35E|C*uQ)% z`5M_A@taMEO$7Z$RT!I4YR9`i=uv3BCdh~gGkhj;N;CUy3PuN*3+H%XA$q>>X5ZwO zs-9gRUpE;2Vt$QRXND;3$vYV{^=q->xX1M5{3QgkLyaepN?KGbsWt zBGq6u}ISh*t!dO(C>i|!FY!p(aSNZadWq;nnu-@~QJ&a3k}!9Q1e@vKe} z9*oefhrvh=I9*}rUl%J%Nds^Eu^H8*wdsH_$8blZwG#f6f?^crowstY!%vCv+H*nW zdx88-;Onu%M6AX4NBv9ALRYk25oAT*chj5yii*+g_})k_a|6i;OVLhxr|Yu;WLH`e zFTV{KdDwtgFdC8Pve;Uz4b5@83A7bHU6U_G2m0#wQFNAk}zp0gsy146qKTV$}shZeKtOpABJQsyQj&1 zfQruF#!|FRm@WjmqV5-c2 z!Ansv!@=zxBs59XA6IuTKSBs2kh%P-qzRc{#B9GxA2~?n7Yj3oTLLQJh)8 zu@D=+YTX-CLj|4DdQFfSrOj2{G2yE4ZKg{R;;e-Ab2sfkk*27?w zd&b|N{9hAn;BZEFMj~CklJL@)ioW@9TiYh>6v*{(%(b{UXt15dw;{xA2Na zQl?y)5f^V0th&;$sjiZrL?BD_`sC|x{cKVfbt$*0Eu(-Pjh=WeqtAQ6&1k*#Fc?(; z$4h_MG{MF_%$Jcc1%4Z2kw=Yq1lXhT;~eILWuOSjHKu-fnKI;e+(cuHLVxUX64;+n z0mFd}fyl{#`4K1xcN>7!K=u=L^yQOJCch`Wnr#aZKkMLq89mf{GWV0H#GV_>fg#i$ zH!|_^W8f@k^m9=5RK|^+rGY=LN+Xcloe2iVOhRXc?IdOPt#Vnw_7f`h41Ibc?omG2 zb{KjnfP-GB{xJxN0_PB|T^s!zVAwGSJqWFrf)ZqY=F=%g3E#nrZjH~5UbCwNYT)op zzznM|>wRB@$59LGH9*}tRTew(uuWsn~GfUq5LhYJV_RK zAv)Lnj;6v_6k$l}P_`$I2$(8T%Ty=TCM@$kz*!$}E#U?qk)WprK3m10?=}V_BjB-C z`^O+8DQyb;HX>ROGDU9`gF6U;WuOR!b^ZML`h!o&;YNW=S2IY}d2PY#5&3w88>3!I~JWZq}V8K&WGr=4y`L?&izO1BSO; zztHU*bK?tt#j6P{U6k@U&Z^7gE%S+9O83Dc{G59K5kvHW(zv4#c#kmn6j}HasBIdB zaQ9(AGX0l)I+~(K4~#*$`G94h`1rhBoLy|OJMxbC3tTf1QIQ9kf zP7JIEG9QBct-H7;9Bcg|_>Ff_&n$;y9uaq!y6j4jKA?f;Q>-tkoZe;jvHx-AuLqex|B?_Eifgvczi zcVy3!B85~WrAXPUkj#>Z5E7!utb~y4mEUo7j{Cd!`*4OH_w#u4*ZX~Z&hxz9@6URF z#OW~yv4>QrQ5^mO`;G`7S|B29Tp~U~{CdETRDCWb1-wR{0)aLN_*AS)|N83rZv~}^ zD2f}2`z~z+Q=qfJR2E4Jv>~4+3OPNX8^rX;vy8HOGrdh>*z5K@W?S_dO!Rb@2I&Sd z7O&NKGkIBN$C(eP=t+36{(Hx3KbI?K*Wsj|TBhRgLF@fYpBD#SrCHCNE>d1%hMAr- ztkKor%J&G2xh4Hg@9cj``h#@*MF{m^0_6)`P!LzpNyNj{i zyD9vot=-CH@1EH!a`du(B0i`fN-p_3Zp(Vo;l4{A&*02=U=Q_#6VuMar6 z3aa2RL-hKcHj25$h2A5=$i0XHXR~Muga9Kxpu!N)NM&flu=95m7(rGD7}+0dI&ZFb zwQTVHgb7o?&QNXe{7;c8c^WjmH$xT$Mvy1O80lwA^B4=waLhgvQT1s{5yYfL+yC5- z9xjOpIg@m{*oE%qD^wT-Tz**j-sGfa+s`{YqUHF%;PB0XF5&hKRioAKW{ynGyq>{~ zk>J;JBA^z*4`t`x^!(c<#;BYEr?kY$MLu9Vz+5B6NbYmSEfg3*RtOmJ|EWE#Ry#W} zTUVM@XA{AC6spQlLB8xNzalsh1xAo3#2D?69?IVI{daS`ltmfMafb((Fq)}Q&9vmZ zQ*o4Cmtx&x6?7*$D8)H<4yN^P6+0C^bKsNNcN{)uF!pS#o>!Q25 zenx>2)O$o2b-TM*B*H6%lfVH_o|*Bm?_%7 zP6$O&hK1fErU)o^CcG#`@Z&NOQ9a;CDn+x!6+@sDaZpfD80kCca~m0(=sQ|Da9cYN zuI{wew{x)P*0-{nGeEFGASk?MvOp$8u48#u+?st`oPS;X0H#H^$b~4nQMCgUB6vjx z{U8SD9rrGgdykby2d9K-U7Ia5F#)vwsLxhj?oGwz)LsM1wO3A}0!T=Vj#+X04(IRz zn&Nd0LO|KY6~a8YfE;&T0k(a?-ulARm;qX5{2N-fhtI6Rz9YiNA1DoTbcvZYFJO1B zkpjeb=;1>UAb1g4uriNOZjwS?Ptm@l%>}V$)qgg8g{}g2C%>5^@@y*)h!A{cz2FHk zL{1h?->-R3+K>5+bkLUN;`S@_g*bQ>8{jcP{k_)M%iY<-eH zrDPD;lb^gS{zTo$Wz=3hnl~J3%Ed54WNpy30(mT3=shBgnt=6I_eE8R6;?rbd4WPz zKqJLylSV@$@>sTDWj>buz9BR{c_AqHZQUx*1oql+u-;h|dErGGE}Bjh6hV)Rn4+ke zSAjGdhWk&m+5WEV@W7T>cU-@&*uTOFxQm5apLq?XCn_o8b$e;6d#OW@u4+dd^Yy=t zIDGG8wT-qq3>uAHRHCax_=7Q1RQaVGMPj}1nusXkF!Ay-SgIHa0Y?jgMoN+EVFMJ2 z^_+!yD+G${O|uG~omU7m99mlx`Q0w^6v`Hmb=a$y5t3c7(t#8V?=Fq=5Wo$ zD=&O&zH(j`QDnp7FY4j3N^mU%ZYHms zx}Di{S(Wocv>^Q(rybbGXZ+gbVJO51>OCTibb#g7iX{&7d4W66=Nc(S%Xt6#powXq z6#_;X9)HRvor^EY+OyllT+ddYfwskdB~Od)Z0U_ap+?a2BF1Rymhj~-K9nboTJoFy zEg1G=!idVuHF&0X7mF&6Do*^TdlxE7LbY1rE^OlkXv_WUrz^ff3{hF-AJLiO_#c`6<~m1gMg5GhWk{- zQ{nys4z2Y3xRLJZHgFpn9w<>lGRE(6j)wNcA{e0k^Me>51(*CC;ihWpyRQdx#Ppfn zz@$T(-=}-q^P^vBr;!dHd#m4>=R25E-VWohF zdy@WN9?SswJn@8zz#$^F3wB`rm@MS(+rVoO@#Tp-r zUOh@}jy<&wbFM{G0v75$B8-}V-5)wjR3TvL>p#}d!q?YFfDpl}5NLyd5N-3&*?}g8 zp}2=xj&_X`=5f&Zj(GCr`^ePy55PbMuR@?7!~m^{(Kn}-h<5*U1Rv_n|GXO$KuQzQ zR6k>+?!H~-pImPccmS0OIjtNRD}8(6WVj;b1sRjAkvKeK0K?T9SrHMo9k_#6JFF}* zs}SXeYtT_ecohQsjtC!B;FI3GR22fM@$mwURE0KDAG!nrgg}KLEAzRK2h))ceQ#EG z#c@qs%W5dl2R9J;29l@L$AkncL4**D%pgyQA)1&Kf1CEfwz-&xGx%!{KYk@2ao8d;L4JQqmW_Zc1s) z5cNb`jG!2qLA^(WQ4?@0Io(oIYGGh#4m47XR^UD9&|m~wAz(C=n0;;s-{8kqN}5;v z{loXpfpyS)*O6m%eK!M|W7E*{BE~4HA;0jKb74A`K}3p7&oVK{VD3PcA%5QD4`@Tq-vaptb%uw!5ghpOSA`_3My z4E{=TeA2&jGK2Ua2x5>Y#0d4Z22GFfT3o(m@hI6ZbRq^5LaDRgWr8BtGt!^49G&=~ z7mo^|A)3P5X`^cgd#T%RUrcW|#o@>D+aE9<));2wU7orO*R=~XLOauR(bWEcdXEUB zCSX7>zNq>jSnJ^72a1Z2Z)SXT^so{Nj36rnjA%EB%nm=RJ99^{a($$dR^#&&dB01S zE>Zf?eI#Epd)Dz0)ukNZ^CHIR+pplv`q+A=J32!b)6Ha`V8SSW%LbFIL#OE0%0)}A zH=IyJg^@caUA9&8@uHVQD-twBQr&TQflWN-59ujgjrU#>ru}kH95Y5M?sB3jqX)ew zB8(J(+8+*!suBFW5HIjn$RV;cHrkh3ATUWke z9eXk=ex(0WJegg=wl3tw2i;7WwQ)E+bM&OuNozkB^Mki`J1u`QfSI4K^urgxBnUnS z0Q-)Jo=#wzPIsyF%vGczi)1%ad{saRLO{tZyz*ELok>K1&%eGS#s|3O z?%bmIAc$eYz$_imNK>KhKW)oFfDoiYkQD+#ohO^}KlGYN_#2m13>6f__zV84<5|Q` zzS3HLWjhy$5Q0<)@`M}z3 zr9n}Cqc@wIM=xp|5@@_B;=U1wf22merKdjaXSmVTX6rk)xtJk({y@$Q)F1@<4fP%o zMmj(#*rcVVLO`8#UZ9a;WcWy&5(P$(6#_=$<#+h|s=gf3kNln+v$DZL!W_KP#fN-6 zpKEExjshdd6Jm^L=z7-mpRV3;m?=8HR5Cst6Gr*5v+0|z9?RQuvL?Y@q)rSKM#sKI zzh=Ig#K~S~d5c}RB^ZZS+8d&OWT!4mA=Sg1ryg_*W5!4?Y19SJ(KUs__Mf!+9# zwGIbxzkD_W=Q`52&kQW@PY}wJs)!%FLDP$wpSL;nbRa+Qu?*}xB6@~^Vg1fUV;KT^ z`1FLySLt`eAB1Y-!>2-k0fG$zJsdi0)>-4JDK9ULPBZ8yq>q;Y;L4zI%bdL>j6K~ z2-d|c-2+B2czi$`1bmX7=VbgUe){A%t+CW$C*B9In!yd!*U00SQR{;`az@ z&*1Rf*AkZbZ#QxAGe|iS%ir@7Gd?C`wWi4N0sD>!9|d3o_2DJr!wXzPgOQ$oiDQ|XY~~-kKbb?V;I3C3aD^jD?4s|sfU+NW@d4c+rf25C%9svm zSvFtZ-*)~DPflVIpEvSqvze0TH_PGqn1d>lAEKhiI$5rOkw)2XowpJ5aAfsiplTjn z(eDh}hjE5Xk?|r&pYUR)=da0Q6x;8?z9XWC*1(H`X^Hgk^XLITQt{cPaKs6@_<%MD z^bB!V>y17aaQZxyrCKbZ#Bc_>%+8Knd^CQ#^dT1?&<|pKybDVB+T{va4>dVQovnS_ zgb5$dho|IQuX}FhmT=2uG0$a3g^$~@tF;cM?v=|o-}ZQE-#>`Mf4DF9I^c}>?(MYc z$;^dQI+*cseZ-Aoa}d~fMEC>)1z<%N#fKGk{*Q0&v_8;C0rEaCfTGOVf|dF7r@nnE zXLfy2>SL!>BMAymrJ!YbadLzjGL2|Kgy4tq3!V@|^ojdr(}V79zzxcT2(R_d@RxVU!1qXq*=v#=~dOLOgoFj})S14OdnnS0T^_0ioJ)d!>=L zv8%#m8>$1eMvqLN0r!_Rl5Y-buvlFNA_U(FpdZ8l^=tRhY&|Aq5TuL4=XNWtS$pMhV1!_0qT5f? zSj+$&5ATHLf#3&VVBZnplK~9C=oVEWczpQz^nf2JJ`D}^50R%ppbY{(8Ezl{%1@j% zYwOA5O>`}f;09MR4qhTF*O^+!+Ju}Q&<$dGJa!Ii+DISCGoRs@`qtRnfr%c`xRxPZ zpR!^4`VA|*IluX%qNjYrsPad%onEi04rC{WcJTxEl6h#Ros!5Mi#bzYHm>RAftj8Q zC;u!%o2v@g+P$Kg-B`3d#!>2IxPPQIi{Eo_0N>@?&FF+}yJOq5>Rmt}u?Vc~PeNFo&zL`O}z%vYu9HK&K&=L@fS zqKpb6>hDZh_REGg^D8&A+|dqRfy0Zw-=}c*W6D*V2fM|r?uOmNEJ7^j*djrQ;6K2* zH$DHhi7;{pP99b-DnjtFj8_EM)B!Y7jCg(Mp&MP`W7&e0`E2OEa{8;>ERCQi#U3%^ zUBLt3gpj^CIYwu{G~ED|2tJlActVVk!|>OaQn8N*Pu`XLk+)WC0A>UtQD-hZskJ8ZkU9MmEX*r@@)0Kos2m7#!*t7_empNci7z1wAMzs$?du`i=)` z4_2X4ArDJMc9j)(3tL%ChD~$teZ}Eh{SLGG-8=sK&Yzfmo^VI(cT1Hh{e|ki!he7Z zZ<`1s9iRwi@1iP%Aeiw3n>&C;iqW5SatFbj8UaR-6#_< zA(&O00lS(=vTDZ%%e_%h1UW)XQHlJD@YlBsy4UM#NnObQCXR`s5XBAaKmHNh_%oIA zPuw?FZB!J|8|B#eCHCwOS@xXuN6uRp9NtE#-X?kd<-TO9HpV0L@l2Sd$l|8hVHB|p z>OCTgEPRV%V)>ZtlMN>{9_v{W{en%_kBWv5!8D`7>NTl z7v+{JMxuZi0gV)+b4Jv$XfOh;5HPZLy8r0Av+;eJCU40}U0vZJaDb*SO^%WJc$5PQ zHG-ZOF-DEC`?y@h@dY~hqgvb6f5Ud6%SK%2YN<&_7Im5$zB9cmyisAqdb=*0-|_O` z>+0clpNhN`;N)0Q^E>4dKGpYOn!73b^$uai$o_xx*h-p(x#fTqJ zBcPFDEAy$5hxYgM;^!P+Cc}3#XNmi71NRHb{Kx(EcTf08puh<7gczfe z(5SMjCuu_Y?6*D=6}^&)DX6V|X?L70UGLAQYw*gcu`5*OlMv z4BhCjUNrB~%1#Nwgi(Q=qK`LkOBP3tN^_X!g$Ps_6+H0%b8~7;y)H?!cl~G;P)Imb zt%j|@@H2nLbikUH0lV{iA=ULglSStRxqLL=xfp2)DR~SSH{#yc{6)`}Ul0#NZ z^$KqZ3}n3DZ4nAmIgn$AeuL7;V%jv;*j}|NoxaJmP`;K1gSOT2{A%uztJuIqqS{$?$b|Pu#=d zGlEKJlva7TTBMI(uRs398#6=&tL#^U5W#5&Jrmn$7^ zfn$95)AaKe2>6Kac<1OXIuc!dN+a%1%%OZvaOeGNKrczg_xlwnp{g+OAJM!M#N?du z2oABZ;O0F0Znh#Esce1sOIh7~sM@CpLO9e_qE zKHjpFPf&;tWQ9Oa?`-Dpp!7nEOHU2NvqX~>?n2Ac(kFe4=2V|5Kg2!Z%Oy+?#mII#81e$f;N9wR;;;BHKykz!Qc zNmUC95PS?kFN5*+^2gGv!0(~`Hi zGu#}9&i%!HMOmZ|wJ&mv7J82eqh13qFRrD=G=5;=3}~bnt*U2XM}rY)g@930b9%FZ zckk`nK2MW2?Ng8Jfnu5yIYv}tXQ3VW2oh@Oc@bkoIXND(Ta3-(0CON!M#mFdOfl^W zQ%IyxP}cDPzJ!D=RxeJW!pNJxkAG@h#_q(|9xI-wn@wen)k zm<%>~i?P}`W=SxneIP&5dWX*w+EI;wBFGV9ikLV3X|7RYX3Dag_PB6JuMHDL^i%7U zc1&4&+a4dLtFyl(kBXx01$1$joX=|JHI7DUns|=k@IBV%sTW3yX^hps8NUc@YQs#? z^!MMiD8vZrJtB&Ffop#mmr4;YkYEFilp?AExds$s1X&?aR1uc$F&lYr;Ax+GMvunZ zTTQn?jFic5j#idvL=)2>Plz$<-n>icOYD{GZ#$|>pG7fX+e_4-`2DKsNJ;souUpu{ z)j0Z4VWb^>Woz}aCuX^Mt2Yds`l64+JH9C$o$4Nv_T04JJ=pDvI%bTbI7Xq16A>iX zQ1208!~tMLvqX&efJv9RMv76*OD7cTuM1WP81eUrMmbDO^tqnQi2JN3nb=(e9&ybg zzdF(39W$zHroiV#jL{Lt!}^Y|qIPV^VsUVeWjKgQjq0*GU8y3o)t~=4tg-d*!*i%G zx>;UxF=FM7zTL-MXfDrW{>9;cPfnZ5p1#7--t+Zyhb2olW{f0XorjLt!(+7Yo`^8Y z1GZOZEj7ay02a=GMv75*Vq7%}HG-@VFw*fTjj=A+eD&M=zEX$G>>bac^;bUf_194e z3L_L5HslF0Miu&d)(`nLH(C^O6uh~5xfc^gRlASPB*~b{naYa!q?*eAL4{Gyg}-|a z-?lIO%6R47#cvs#aQJqsJR9Nq_$SQM)w6r=pP0prQEOx@G+_;o(cGJ!f7?VDg#&lq z1uwe(3cun3l(+tmTIOM1ZBZaZ9I!J_pbY{-3fesergmMgtn3}9oQ7J()uFXl3UY+< zFGQo+rvL9*5d##}&wA$LiFbFTZ1sXYdk*?y0!Xwa$hLFT(RWLO1UJn^2|Ox*{C^c4 z>fRSJb0K*{8qHMF0USOqp)@gk{ql34lVk_3xIW*H8K5!S;`3lu1V32|z9S-h)|~b7 zqF8b;12%5+l4sU_VfVy9fDp_U0T$-15D>~^cyCa#CED6`vf`e>pr!XyP=(rQ$Pr>b zUHT2V3e9<$_kHk@t0II+{`z@|e_*w*@_qmrWiVYKXu z;f*{cDNn_-2G2&}@Hs2G8qTlm%h8wUJN;?!#&gUR^*?%mrkFX@dqfmf0GT!Y62*v@ z4|pv=Bc>kA6#cUDBfT#wii~?B8-iHcQfkl$jD;3(K-_BrR-jrG6< zWgU6i`=yjSingf*@;x&6idlgUk55D8&)`?_=X;I_8$;m4#FBUC^8wfY0*y3=(dOSs zLleV5D+GLWY~6Hcd?;mkm1yefJRD?Yz)UFW7Fj05v*$=VNDqRkYUpth6BHE9_gJWn z*YD2|Wz>pY%VjVT#Br>IK|@s1qD#%QI!LQzFDin%r(XQ5qc&~9X$EooJ8;Y3@cDx` zGt+E-9I0sZKa%q{2m7&Chx7szXYc1;^!&>vqR0~14tr!#@nMA>k@E@wcjp0(l%hxN zEe}8ef*+PGSRqgp?zA`m>?fbAym$8`o#=iqr2)+`dXkIKL5~}@C@_Ma7coZZQd{lL z^L)|iw^dp%p3Z$A6GpschbrSKj+C-Dncps;eprPHqvDNK^d|fEZM%}stu8JQWrM@N z%rfZ^Ih~_-?><#IeO=dD%tF*r>WJoy2h@8+81(`h(WI7&5ic<71T<2N=q)2(pb#U- z3IQWSmET)8-K}!EPkYXx!lYuj6ild7&Cn)NhV&k{Epu4H2KNsqsYj-s%ZYjl5$64LoDh87wgJjo2yBHARe^Y*v( znxVXhR3xPC`eCLhHCG@7MJ5FG9uY+xKqho@sT2tTXYheWO3^85hj2!t!X!rP zbstVp4E##9J9l${dL!p0RAQ9#sa_~$t5$%>CesRPmY2YNcdmAmPZAy$O8(6$chh5t zOvH@Q!E5%3C@_M0j|d|jV5ZuC(V1%an8wE|3QXORXGY;4MW&Hsv|xpRQ67)1vG?1C zCTfqEvRUcrV9zEnvCbkNmsv>+`y$6^!4qPPwyaIi3A@7ZYl5$Z`jh;dhnO(ROMNg} zNa1^phRHZSA$tqDvP9=`7a67tuHI>u$*p_R@DAADDEn8ytwG;`J!-Exev++H7BfbC z+!@doStL`9;M*p`C=a+qI%`pk;KyYmdgK#CHs&RdL4@GPW%CvY_%u21;cV!8TjS4v z?yXeP=gesru&kaJego<0XPMwO%<4DfZ|>GS=2+yVGHz(h0#t@wyY}JXj25}qQ@d%lr?q3h zfBb2+;&D)c;Kyb2Jx7F1=UFcb>P3SX{5lyho%0`aLOJTUL%>9c12!uMZ4l7ueMO&F z5&fJyUTQR_-uQ;-B$x*6DkDpS_J3o31d@Z`)A{$Ph}qc_eXeNyCu?Kcj-4Dn`%A4c zu~W(+-aIvR{+ngcadWkS-RPFeaY$*ydo@sY;OPNv5a_&d*txw?aTj=izSJgwrh^ zGtzVy0nYh{XbtsY;K2r0t^QOjU_ z;0|_{>isQMPl9~ujZs0={HSXmUfY`$PdgPOTdUZE!|T2FIJv#CmcOZKI~T{7Br41r zlvI%i#V-W<4ZSBKjPiiNvBsjQ4-$-k$iPd!vhgKbv=fYCNH7Ae5HK=y%lT3#pr9gh zQ!-jjGjc1oPkt1~4g4G>F2)+}52PHf!Vt}|gLQk>T4X@L-dni11Y0?1`KsN)fJcy-_ z<9o#4L3i~11}{_q#TCZ}^WPp?m(Epwiq1a?D5>XNm3f$vOV>HH>$?38++OV4@t1d~ zLEG^WWYquqjtHM%11}1;Me%`;WxT+}O>>P@g+>iFLOU1XW7&cg0z&Rns@~P(x+b4` zizehR1mc3ANyT;K2)Vf&5&;o{Pl*;hA%0#_%SAVy!K;vvpdRoeh3Lq0R_H_ryb6If2ndDG*gpAv{ZZ&us(=j5 zj1=8S@UBEEIr770x+8ZgL4@E{2=s#(prlxzWyQ}F6(y%n)!gtF!*=&LyUQfCw|}B| z{;QYPG4xwXPytl$%hh}L)8X*!w`M$R+7Dd9;itr{PpBlUwyBy`nz^aG8T$!E#s2$H zHUzIiVBZnpV-Jk$tfyd_f?^1D}S90!r=`*a_Aao%V%Xz9-h&03&4IB$ceWPDq96l z57>7^^aPuFQMfKj4}uuR2gER-k@6!caxDf-f)KSGZE7((3{cT<^1Mx#LA$E zMDW^Zna$)eY_!~XABYfw7zTMl4AGiehQU=j+#`(*0xh|XvP_sD8VxoqFU-5;6(^1&G#R57+Is;;*pCGXoG-|y~~K?GhyjnrorYaSB`aMae!0P z8-Vo}k|DiW*a%cT4?cu}ZV>a+$b85z?RMG=d!;`spP7CW!o&~L+nIg04y`Dyy;RRB zcjNvQR3a4EbD+Do;)2S{xaYk}&XRj@_~-J+znVufoA174UY*gnuNgBx^m2|U<`%)e zBcjJ2Cq@n8g@atjo76|0%G%Jhr&Du-XdhOdT-mgVp0UoWV z`j4aaR)z+Yp!mRR@Vpnq;IN!f;I&$QDsH(<)~WdDiF261p-jRXe(dTkcHjsp`rXJiUbj`+D#sa{jV;RTg9s2tFuJ1dycyhWiR_atU;M02k~M-buH!{&RA z2pdBnb}fFS9$s_=`SgGvY2-@njfWx^eC7k%AfS`Seep!@kITpKMM7!9Nds}L(2B+? z^0>8TCCvzk4!r7seh|a6SMkL~p&xsA^^<#NTv*1i)enw{@@zOehAVYCt?f;<<6|-^ zc>Le>hOk?0JpD${#JA|A2rUk8wK~=!JK5zFL#%pz!qtaknBjS`bMreeYQcYYVBZnp z!w=k0M!Q7S5!EB#$dMc!@e)jX;8kbd!aO>kb@sEXUaZl!?M;h!UvK#oM2GS}%DwJ8 z#{tbj!~gm7UJ!$G!cnCs|LsGmv+Ir@>}Pwt4U_7Ks=oWMyYc1m(@u;V!hC7bO&s=k zuJ#P8J7Wgtk!}zd2oAjJ%=a7-Hl4tD zYw}xs5I^^sgk$$y8F;)fu~X2f;>?4;bF0)}E!?_M6uMX@xB6?JTz4gY6 ziVpmEix03n*GT#4a1?@WriUMIEm)aPdBQCdKWVh`2u>d|Y`tF>R|_qp3X;dI_;lK8 z5Fz;S)`BO*5J{MH)>WTjNQ}+6sK|G22e!ki)%rbd`?7m9o0qK~&Qp!@L>0Msj&j*N zs#g4|Gr%8iecZJQhmXEtWI5s;hsSA0U?FYCADose`|NVWJo(PaOiJ}HI2d9vuzX|RQBX^ zhF1Fbq43N0A1b~a03JgUWFE{%*V0XhQ=JG-$yEwV|%xJy(-cIra(oT$PqI5 z%FcmYg%&&^hRE;rudNTJSlKjh(WiU08)6r+^5PwvPunzJx(g@DmiQ-8!WA6|0{^UR6T%``h(z%|q&N^*<_ z89MeL$7sP5VvMFj1DmEkoAs+sEpt!a$SH{lBa3XWnA0V50FLKi^)JwK{ebQX^h+jQsLf7b3@K!4qPP zxO85rIBnItRZ#ob{m@v`A50h>QkE^g@aL()E;`NaJ=Phes4!Zu^!4PB$+jzEx6<#n zyq+os>b)lZ+;VQM?wK6^^7le};w@RP+~oh#Zw0m;uAB#c zdHSqb-A4Dx>fQ$b_(W6yJqum?BfL9oYNGsTDg@94h&%M)) zsc*6q#Am;RG#=*|0%Ms=A=#Shefq>5$SGQIgqWi0Yw?$hZcxQP9#M-6iF>;S6GacM zO$`kQREK16YfPRf{$fD>H7bufA95C?JCF+2W%_@Z4d}j=-i+FLu*r{ z#L%byb49WZn$XsuK(ehtpPX3DL4x4FB+w0Fe$*V^h4ysv7Q1Y?t($(zo(B^@*A>)- zr{CTTF!dNIY0OB-E355HiiKCva=P-#RNhi}tcF|_=p&*_TPLgr`rA##}c zVc=VP7gQnmPzLrL5k0EFWiEn?hBA11fO4O}k2H{#{=9Mv3}o>1fHny9(B5mjv!}g& z)ZQt8v8&FxS_s@=e_4urn)-&DzW{Q4KtG7_sr|VuO4V`O#4;IrmXPN=OEBSMLLcDf zPg`OsE5y9_aKN1xsPGYcrX^eTH2b-MK}*F%-Az|RUPN0zj^quY54^Vy(WI~V?0z#@cT+WwY$@I7U zoBOW*Wx3==a9hx``(y$%MiV{?5`-WTf*c`c=s|hKBl+)r{gqaH0aW_aH!(4E>VSf@ z{>S{?^3zxUc#c`Np<<}sgzH?-XWeMbE!%B8qf?g^r{}7djA7#0%_C1R5ztZaS7iC@6xg5Gd-Mtlk*+yK7*i zZR_7#uCu-M;Cfk<7a2u;o;T%DPy{(bOp$U(8`tZcDSWob=@a`Obz{3zGg|Q7$<#)t z)@wFd6X)!b#!ykjeyhnxg~vI{r~a&t7VWYI9Da?VqqQn^9eYfkvfK;BC>6{~)NCFY z2`UkM5-|6o=U+AvMI69EpJj_`5qu~U0nX!-&o2hQ<8c5Pf)8b&4FW-yp(#dq(adv zMmY~S|N6X%$=B@4+A1?t{AgSpJkJteR@7Xv=a+BQbw(UMM(fMCcz?7r|MEZ&zk2oJH5aED zF*B4-*9mPlga7^(dXI=Aci^3HFB-|VYNuzX7`y@+`zEaVDHLPMK4f z0pk4}CW2gqz`i5GCj$ryYZetD1o(*P36h5}1Ciw4DDe4bgMd$ln5)O&RjLWflP9#C zBGrU_!FlP+|8Yp8vrTU`3Vio`Q2PV^5*`$5yeXpMElZe57uQ>M<19Z#~ zK867=ned*8@iFkC;8-F)z=c%6k5qjcE!hqr#|N}Qz=!`Y|KM1{bX>JtPo&t?)A--e zt>Wv**H;4y^A01&2lRs&AK%dAm!<-*_B+d46funWpT>kwiqfG0-1JRD%e?%QFQ!xR zsPM^95xBWm;x`=+W7n^$tefmOeBhe*UIK6K{}_Lo9$}I*gT3U!l~2o2ENg&$M}$u; zpg!vs#RonG;uQeuCjyNWAh|&tiuvdTD+Gi(*PKvw-~Bl4g-V2HfttI%1{lAF`N`r} z_$qcZgL&w25i`VR8u9q}HM2KAZsk4ItO>z(%EwYT$6M>Jqdfy-`YyrG%p9l~%1jxq zk$rep*VZq^LU#X-1fZ5{XmNd3dg5d2 zER`Z2U~vp+q!iIVKG}H`n2A0GG%=tpzPdVRVcv?pv7NcTmARX~gSpK)E(?2`b4lvQ zs;b#4D>>BDucxY)SG`wINk3^Puc)xUl1=5KO1Z*&#TBd7yuxB~2H7^OdSpQ9cR=NW z=uJaSx}+y+q1tK>?nSN)NnAlPf@5lJ8pZ+7HP<9iJfF3(gZ|v#qUU66XFvCd4lWKn z$_kXrH&HCZ<0-hsXEj-UT#G>kh95&L{ON=b$X@DOy2Ioa=RGOkI4&wBuqc`R+qCKX zA3N?cGm9Bk0}nsfWKsMP|5my63~j7!?Tqc`{tn>(M#hG7e~paNVObg0lLLjfK_a$DOcyw(}=WH&Iah!yD*6=pTaMDGHR{0~)F9y~*!aM)B=KRtVp|`+KHy zJEx~day;4OawV$8bZx+Ie>Zt0bj6*8?I=D+$P?nv@knH0W_(1CvrOSK7rOWOI!vD< zv(@HuvDNx%>TB<%GJe0X4%O#)roOhgyj%NQ_4Iog#^*`nIDGx;j=eo4cqzI` z_5Q{DIc_}l9ZmFvdXMOH6bE+O$}PH@3y%>Wa97V{!%cozQ9N<)V`+ppHG{4LYt;xZw`hq9K7`gZqX4wAidiSI7R%fs4p>|9d z*>9Ho9mT(mO5u%PhJ)CiZ>TWRxm@G^uvc66$nuGjkL!*AbD5h5e%o8L6jVR^fQ$H) zl2(HmBeCPj+JJsg07ZEz;3aFJ_lPiZ2QX4vB1XKTz$81+NHOZ7^Uy=1MxYe}Mj7k$ zEdLD4KJ1uf3*DZgC#V5VCKTbwM}uy;Eg-Jc#{23EQ zr{7atf5Az$CnBftfx>|w=TTv_$?S<~*nV%R?LI5Z0xDm$;_%WgTC<{BTv58u_@#aK zQDQ#=X|*-t8*+>m-Zl|N;=rcum5X8oKOZG3pa=X&A=-u$rvj(MIRMTV&*VWH1caJu z{(ji~PO&m!x}4RyCdP3VX&^!HpAzT>F+Y7=PExI%4qa4&hqKZJ zwXr3ap)Ch*xy4^Ur{DhYZknKBGb(6n-hW7tfcNP?padbH=bsG% zJrz>78RKHI!mXdY=zghi(Nq-NTOzZTd~eB+G~NpZKL7k6#wTl>S@zed`Uo}lJEMla z>#<#J7SSoxXS6l;vV7`sVCxfS369e`n*ZVi2?C}&!87PAx3DKpHpb8*JC5Y9<4xH zN0&R85IQOLB9pd7%TRqfOV{ky5W2hZ!tV;x|0+wrC4v|7R+W^$fWuRGANMtoj`FUj zV5DAIN0EdXp~9EDp_()BVSl0bh%gF&M}ebVbR`!7MnXVF3N%uT4)WVV3WNY7$O-`? z9Ze7S8Xr0O61n8nc2ieOFM{hSzOv-wc^lERwJ0!xJR!zN`~021=hbM9Y!3O*ww1U4 z#)OgNZ@+R%+ChgAcjg(<4G!p5R3_w;jvcJye46))jmF!!eC-!>6OI>3d~{!7fDf#TTzQ3&0DkT`Ge zvg#rJF3!8(qUC)1o_(zcFA|z2M<{yBjm;oJ2)?C%&x#nJpsS|#DG4GyM~-$K5+69D zfeE0@Jia@k$@WE`Lk%3vlUF`R70SxfIP;545?_z-RE*~)r|!n#m%mr@KQ+erbt3py zM|(}icFX`3Nu7soEQe15z;{H1PbaXycFBt=e87?%&`4FNJ2n+fFoUcR5DFJLe|=cW zsjE9E;$p(s11;-15TSZeBI7HSi>FXQ#QnV@M@Vv`2jlqfxAY>4+`yG)@xOVWeHk*Bw(4(wv}LRh zGep9=5@Fyt48aEo^&SyMNx*?Mi>2m7Jiz^mKqJLy#7iR*)FA}H46;JNXsBoM`Pmk2 zmG9qkY3}waJ>v&cY8fH&lzMN&F-{a1L7os}w2|Vps@W2{4_eawZ>@t*|hIE>(ob2lRs&pKQ7@&SKZwrN>`xt9n;l+l2|AgO7CM-(+sF zE*|E3e&A!ra#Z+iwm8#~T4ihClrHCYxh--v4lgPm+fo-(%XLV4=Bbi39rh~LTsMs9 zLH&Wxg}}Ze!p9!Sg`5|~2Yy7x#}AZ{0U9Yl^5#_gK>dLaVhdIX2sNE4a2&fu8TzXr zbkK5W=APpV@YJ;~?*Y;o=~I`Rj(`dTp9?K`LJX1RO_ANJayGt7{Db!oDk;HXg6P*S zFL`eH9p(jBsr(KPH;15tC{!T1q-WkeN;{6d ziuIHDXs9R_{0F$udqfzyzXhtQEjl7YfDsQ6#ehbNQEMEfB?^onEAtrTCJkk_)qmf= z{<4Tf!OIhF zBPua!QTvd4IxIVNr;6wgzJih;z=gpxr`w;4HH`?I(un`Era%)jMg~KBOp#->(0fD} zH37E^xi1>i;0IU*>i>WIh(xVjx0-Y=)W4ey&V*y z4FL2c>-2Z{+MGdx-~}0UgP5Op*QRZ@Mr4?K2$WGY2feDn#LpEk`uvYkO-wDTdqX2u z<(Q!2=U3^UXbZXMlyUI?e9Zo%QVFnF3uRyCx{cKtP1RJd~rGe19yE`I{~flsW# zz9XV%4KS=fyVS&*2MA?ABju-qbIW>=9|WNcvNE3$jop@1d8=@&Fu*cO>A>%jU*Jv; z5%Rk|POLs=3lfANuZA2UX6UDQki*oZ%=dj7UhAGHux`P`kkY$NF}k8=(p+>BkJ@@V zs8BIE^d1pK zmcX7Np`~gOQ0ImZXrx-i)8>5^IYo08=B*GYnqq#DZLF9t` zQkJUH?x{G&KITlSy^lq3c=MSo#aV+AmhIZLY2Be?t(Yll>1mZlp+(SZBBIE>lma)f z#C%9dR1f%(GW438%M?tiIbic4&<24Zchd`9*+Mc|JFosOm3c5=ss!CPq(MH@nB13% zVx|%FgBYOoAH>XmS}Tbvt?7Dc@g=+!6F}W^D+He9?%JvCEo^qb_W?Q?T9w=zdfi6K zsgPy)*bn_a;21Xjstvy-DO?$DPT0I1)Vqj%)s-i?{WAyD%i9DK7g;K+?@eya9A8MMVPi_j7G)LxJw1fdLi*+dkD1MkFs(S!(oP$nd# z2mDAGN?Coa3uFj^2>r7`Ac*5$mR!1ZuSUTM#Z!@vToYa3F^ilZP-3-F>Xc@d5oH z#^-X2_<7|MRC|Mam{m^P`#FUPADV9?g?D3Q)I%N9_B1y8*rLKm`EBj#cHp`Y;vj~1lV_$OQYdZ z(A6BtQ{I|)o_-KBK0995pqLT@`;G{oV4xJlszvc(g-w2Wg@Ju%KqHM_@*Pb{AVBa_ zLJL+12!->U(ly8mu3pE`zmtx!U&J0-YyHGWHkvoq8=e3Wf}ayw@Prtm09z+l%hLwC zR>ra&y<$+c91}#*{O&6wj9T8QbOf)xu}1)12=ic|62Psm?3Jkc1Kf^4|>}~80i2hkmaH&5IjbF{Gz}(oIH&2Eb~7CCPDBR zEm$F7L|f^ue7o@bwzRWZzA4jNWV^vc+NGK-k-q1Y(vO^?1xJV}I`(e-P$}!4&wbWX z@~S7-U|X?|DCQEqCnazH+`d$=+o~ZSl@#?$Ts)Gs;lh)UqmFvoMk3E}_)mA9Ppm#K z+sCEaFTf)E_zq@@YTOylAWx(hdXI=Aci^22ERiB!Az))N&`8B-El1KWH;*HyXu%O;iWpjk#ce!R@onY&>O|>F`x28D zi8;1BkY3e5RW>wni97CM2P%qY_eJVe>s{G-CpX`1P5e`F9A0evgw$ATQ}>QEilJO8 zqh**WV(tz^F(!r1iJ?xpdKN}AOc;6ZFOA$QFxAm) ztsJ$vbKPxJ81XW!JzvVH!M)KVfnBGdj}nLHZGHOdMJrz)bDf$jr^)77%ou6B>pqTL zjOO0-{M#nNC>)qt99nd05k92x@&bj4fJQ1tX1M0tC@_Mo5HRYjSe^MLyYojCzhkx3 zsiXVuLtCjC$+uDu6oo_A!y?FxAWw)fvNPl?J9@^6Q{GSU^X+d!6__v@S^k8DsxN6~ zt@=3g$RCf2y}y(ADVsV4krkS|Ma7S;eE*l8w-Xfm_a5i2+rScp!ymZk<6>WO)8$~9QH{XQCvPzG<9s;e zHOLSAC_gAqjkscutJ>W+wLHPQ#+sNqwZ4l^DHJfl~FQ~k5@mjQY{r$6{ ztH2eqHlP>@$$pRh7pl3C(*wFeOpjW?gNQ9=WnG_n#M8oQ@GCIUvqrXBBj>(UH>=W@ zao`-_Jyi4v(oPt;-Ixqzo)lr3ayx+p#HA&iet2&gecJ~eH7JS+uy2wSj1AV|T{q@|skpF4WCU;1vq!`WHr zD#qR9^F!5geg{B=;6vDgC&Uo3xZRmn?WWP%qR=EMB(fFTA$zW}8-HY{j~RXU6M54% za}?d)#Y@YS=@U$fq)xod5c=GEZ~%vI^shY-tsyOy$1jp0Bcwi#S%S3opR5KY2wuJy zdXESrci?Di&{8uY0br{J&`4!y^ktPDa*XCI%v&K~B(B4_KRZXwX0~cF@9Mpx8xr6w zwePC`m3G!)RV`f@2fMqwK|$dhNxM6++as7Lhzg2gfvqTZ2dLO$cVc&8fO%Eyz9uSH zzd3qmAJ^UM%&fgXpEJ+>{`h>?@^{{|X7=m}^?>^RjWQZ;Ik!Y2+o(hi7stGhiUbYY z;~G=C(u;|@YLQ!!6W)b#H}klVX6K6??Xs5AREv5^& z`67Olh^f`^#h^~UHd`Zg+vuX>gJEldfXK&f`E%h9Su`#DeE*6iQSCf-Jqf%5tlEjf7JHhkaVH_hhAootfM|E(R0({I>>e zZ#_4WZXd4k+dht$6RJFT{p4rio<9zy9b0sOVxKZ<`=qY9M>(-rZXX_pY@f7IiOVOq zC|R}XOXD;Dt8-rHvQNd&Pd>)%{MmBohAyo#E}QGG$vz)8&dgK$OS)AT%bp+hJnX_G z+jOJyoG8@UxZNk^=Teiq6<@4>BS(1slNv_SoZTPxx!NPtrm4_Ags~7iSJlYce+?A3 zD&jOZ5MhQ|QTb(|@9X#YoP6r*V>)oRar?9PGIUWov7G8oY{WI&QM>{X#zIOYvWfQl zY^>n8+a~Gp+g*>x4r;3J(Ce#b?>ySpFwIy0_s=rsE_l1MCKIi`H|KqH+6rwa1Scnq z9GURWHr)lcxRyK5#q4xhU&!gc`474ILPG}uV_p8l}WVzv~2 zyYN+qd`!d+Gh(0r)}k|G`nhVD9{N|~mxaO`B@ZpVaP)+Ujw^SrY=5-y75=hpi~88k z!(NrQaRGZpP8N2aN*3-K4ooQY#6@bK$p-gG|LY>QHrv43e_eF&dHx+5x-v!Omy0@8*_-^T)9kcwyC!!} zbllz46wgnta>!rozo#|Wc~u{z!A43XvW>O|xm@=4i=2OI*1cDIqdc4IvQgH~7aGNl zOXZa3mLc!TMXx_;vQa0eho&+ChtDm){bNJ;$KBEF>1^7T>_sBaU8}t%qV$#t^Xus@ zMXQ`_G)yv@zv&)oJ9VGuL@i1inb`T1+5x)! zb8q1#O<*@_S~R?sSeKIZqxLGveI4 z_t(PRhK|$ipURpzBvRts5<+f|F?fHy7~A2 z&aX@}*a_)PxY=+dB zeC^o8BD3nH%e14y43CM+@^$RkVO-f3QTEl}f9ldGBIl{{A-Q|^>fO8kUyKGpE{ys^&^|SeQi%~UNiC4X>axK~Z;#>~L zqaJH&7Ao!dqu_}1PcoLwl#u$ysgc#9c6yCDx8cj3UJV-z%eMSO){Smo>jk~Ia65B{ z0?T`@3}3mr?df)|nTjOdPM<&5!1$Vjy~;*ZUO8sK-0_=g?W^t+d)59?iyX~P=h@h5 z@3s2VTW-$0Y~T9v%`Y#y+bxfKi%!pzH{P#4=gzZ3{VTMe6XBk}?3!F5esz{Rc58Vy z)6MbAH|MLJRQI2Hi`peRHYu?zX~E(HyYnnE_lE~P*zNqwn)US`^6-vF+hdD5 z9x!FNyCme!(sJ|89XJ}Wx^C7iDZ3Ot8Cu<@!Kn_1vJSc0WmNx|qw~img#H}aXx;wh zdnV%V57M>pEqC&U2xTx1XPh-NpOwb-WPY=-H3&|5^LFY*bkJ-m<$0xaO~(;CHf? z>#Sa_$hYIqYHtyqLly?LpVQz~X5cYrWs<=>y61nQ!Y=orX8uQkMS;Wp) zCU0%CbskJcdWtiwi*=D5wHe*Pya%wK~l3VimNC}oq0d7RJxM1|eZAT2~*+4H*c zzTMn|^FL8xx%$zW5TToowEz9b{JZp*1~8g_Kcqw8iLqOzbKqsUAp37r*k+<43j4&z zYP-0*q4{$8$w>T8R0z7UZ?&?#yEh;F&i_P(HS?#*6CuMgvQjE;zHJ`Eod3x1101sh z&zAi+D(op)69rw(v1?`y8d|Pde;Em$tPpkFG&gVAo~(?p zrY<9?GUOld=VPBGX}bZ^q!m>Z_}n)+z_||=>6sd&g~+>4=sbbD8-Ayk2GX-9LWd5w zA0!Ll?qXseHS)o$ciQ6)_akefz;`$OIsEXF!+S~P zjci)W+|;;_Rca3A=q)zfs6iD49($+YJPsUolz9frcvun^BEK;?Rw*jb;jM?zLlB`u zhsQ_*m>1#xenktr%JY+oZsSD-*6_c{nkev;6C1DOz#)we!7>^=St0JNuTM|L9X@v` zHGnSIucY#B?k4t8``c742+X-y{OvqUmNM5wVXyACG&K%;K3P}@yVd!k%FY2;POgSf zGbh4^<)qwjnY_WsPMTm%jk{dMTJyQL;kbK%22~Vz|1&m@95^f|w=_r#kvA$-AKboS zF-yLYbSAo>50nRRVUv}tsiEhM?(WaKObC6q22~V#_sgQIGA@=;;S*i4T2QbdO07Y!n8SXMqs<&Eq?!rasjFA<(^EnZeCkC7#9QAL3t zxu3H&2M){1Dh<*?%Yj20&&a|;+@tNnm6HWw$60Cul?D+ubog+oywQa@oHez>Hz%i>%yGjSenf*R z3jB4$QAas&=x~RLG9H$Mg~(r(ya;ZEw-goMN%Ro9pdXb6FtF!Lb5lc4*Y0dq9D2IR zva~I#DD>QECv?Z557Hnlgns5;6_0d`bT-!D4{6I3diJ`Yrw^9}FtU}e=B9?;;F?1? z4!yYsRTO$!kHSZA=zBFt3!(orWAb|(`j@G6Cc2<|O_K*;2S~d7-eL!zIZK_bXT|CB zG^nD`?_6jJ*1cGM`!*5ugoV)4jXwx>zgj|6n@$fwgbq8<`O*MvC(ys!i6HU~W$c#V zL&V!;O%!;;12s}`m&4zEtr;>JJXs;`uX9E#9)}(1sF~CNx?o?I%6k}1?4vF!pBe-V z$6@E6B}TUtp(Hox?nGq z%6qz**hlT}O?9`v#>>fFvL*_0dG3%8VY!LS0CDhD`uwm`avs5Z? zV7uj6Q+vCT?e{j^+we)Vvj$ZZc*l{OLpg9*yH9A479uaXLo5cZsn)Br@-_WXDA>#_c&RNmlWV7K>SQ?r|AZ61u56GMzFX^Scf zeALdIc{y-c!xw3g79yYjB>O7vZ}7j(mBX#h4CZb0Y5TMv9P9)}*RK^295 z(?6ebVjhHkUxTy|`iCInKpc9Jt#l?t=&*(dO9QYaD(EVen8Ilr;$>w&SrY}m+~CrQ zyBu1uJFq7}*vyb5lc)>}mS~hhBS^ENzP_ z3cX)cs_8iN7!A@w=%I1%SK!cJ?xtr?gbvF}ojtMu>{?y+Q6sO09~+;NafkPWAyd$meGONaA&7JC`*}Z zqOe!b-gKOo5(wL$EG&e5u1kOA!5+e1cZixf5jN~FZ4S%i4W4X)IBV)c(&NEIp83?w&O@^!>Sif;-6gw;2^y6J5}I z$^*Evd2!a%(BIyP*@;8npg|Rdo)}-+ghS7LO-e*bSO|T^-Mh+Wa0tECb$SS0&^O8h zu=j-q^zCNUx3f+7h#c1R_Zn1D=r&HCOK|9&Zb*qJ2@9dm$aP+sS%J_Ovh(L9d)t%{ zx}d+82QcrRLpKw4%D(S8?sSg?Y1)b^3jJ>CcFM6HaHzOKgR~I((Z=aI;^pQ3O?vja zpnKku24Dk413Fj?A2@h14t=x+RTO&E?pzmfr$5&qErkAY!dvB$2s+*F4xNcE=%Mle zp3Ld2scU*{?`5^{@^VsxDhmB!`+UmP@zCi`ccny>goV%txfU#rJKg^tJp>Utd?-38 z4Zt>AqnpErmig3;``m$#7TJHJ!ZQ3NOPXt&4eZXA~EjWlQEW~~M{Kjt_H+<~h zbf20z5w^Vo#(V}=>IX7;16u=PZfb{5^?Naev?8b*>CEqhy(iT+|_}U5+3vl4j;bSyN z3z2u)Y!kqdLx;ycr)N)u4jrEVg)D%Jn~8nY$Ommt(E@jP7qTV_-0RYdbsRXPae^!? z#Qk*FH+PO3I^6aZm4+_Z?WOV_?k4t8!)`t;q6-dt30V_`9X+ve3=TWhYZ(iktPu9% zup}#cfNH#<1|Y(=2cwkvQhD>)&*rAay)Km@gyV+g;XFWo}Rrf=%qeL1F(Bx4gcN^3zzFp&D=2wcY0q9swngl>ryFig`m@~ zX^<8|uRCr0dS1z)({p^HGa*8U9cMpj0Jel2T}7VaN~kh%>!6e@^AqQ0vL*`rrYfSZRyrohcvun^BF|QAiItOs7JQ|L&;|W(X#k^fWeKeMPyCw^t78xt;3-o*Px0*FZE*C zM;v;EpE4qrgoV((D!F9l6&*tNv$2)E5mK1wf__38z*v!;k;R%Cy6vSh-Z=D(wz9M> zswnivr*8z}&SP1=l-tsNE&mr{D>FFVKK~IneFtT+ltf`%zsirZM z7Z&)~UnGMxZABG@p492_avb_Z4bnpBWk>8)7CJ-dCo#gOww(Gi9MOAwq{8XisSXwqpg|v=aU9stxzK z1L#24ku_1^89#X_%blSGvt^ah;K>Sc=e)7o%6ThIvQYyNVZ#n|rBvSF$zC+Grgr$l zVgone4u7se6$Rd|a5LrHEZBjz$}Zz!Nmz*7=E(|WrUibdGjq^G=z{)28obhhF-Dmz6~tq=nGa<{E9K15L<9&z=Y!mK8&8Spc@I5?u!R zXw9u2cv%@t);R<<5S7o>;K1^pj+0CsMc0X;iwRd(C1 zIP}_uq-iUvDD+o$Ne(#l7!A@w=t}}ipW?*@RDlXoUZ;pO09#ODKo?ca+!Z$x zhdxDvDhj>+n;Ok==pQvm3!z`m(nHy%1)&>@(U}mT!wxiD8i4&+(c+T7>cWy7xdRxN zUm|Owz>{Ctd2`^9hI?@t4W6tJw|D9+Ww^&-2O3y{8h{8JcA#ga@&-3H$HJPrmVf-R zej&#VANRABlqGFZMS<68ylpQB4#(vo8l;8D*B?4+(* z8u_tXw;poj(BW;!nkeuWT{rm7$n zcV$bNSyOv@;kK(8IcV5HHqxMq!hVx9O*wuKc91(XNDG0Nn=;GFjkE8a=uC7$Z|p1$ z;K9b-tf`@Af97Uo!RJg3swniQ!zYDthePP!HAoAg+s?kNTu1}E22UfM2@yIRmCTX` zaIa`$A2ssUe{Y}1eGW$@H^`bO@G`XyhI8PMMzxAE8a!Dc?%jR;lwJC;W`(#=1L%T% zMJoU6g%1LIaia#=aM-zBWhrw_6!zy5^Wt&XBgw)-*keoGv~pYFUN>syx?pE>m&&t4 zC@^QA`Q&&Chuxa2iNX%))@LgY`v6&32z%*G z>id!V+yRVbi&v8+%{5Wr0pWgDo-8Mmg@w4+R-K`&GK3xGiR#qMiLhZ=DO5uyZ)7+A znVZ_-twy&iz;VN};-f(o1-|0O4!uKN84*juLg-b_rBjYUfKH!Zj~+r7^moz#Mz%oB z+|~%qRYbXm~FtWw^ ztf`^f4sX^0ujwN+sG`uz%{`J0hyFx^v=Dlk*TaDKglO@eHQQ!gX3vT1UA&mgCun_kY<6-3tAvjdr(43k%5jGqurtp%< z8`$v~=BCD7J#U^bxWgN0P(^{?yT0AZcJXZ*q=m@K4{dH`3G=%abS6aT(BTbR$^y8$ zo7hK<{A&f1vVa*nd>UC31>P;Qo0Vm6pUA>O+|?R%RL+Bg4mY%>($EEal2o4cnP|5; zJLBE9+}{xPd9o%7yW_KY$MACE(niLDCo6=#G)-qKOC$!gr3TOi`;=7Pt)hv2)Xomu zFf|sh-5J}-Qs$Z{>}&^eEXQFFA`1&)cd?JY#Gg5#vp2P;W=@0+Yj^4nGI=Ar(!ku* zxM!VCH1#U zi9^4sK^2AGF!k*9IP}_Gr9_m3h0srLsCyHK9%iD4&;|V#Apo1uLT_2lcKY2QUd>^7 zDb`J%HdjTVkE%A;%F^Y@8l;8L9W%RGS+fk!8_ddOW4fA{Cf!b03BdcITU4`BzIt_PI{5jN~VO;ULSLv3zq+{Ri< ztz6`>N`op2JhIzg&Kx-GK(qFg@vtN;MDFkNRJqv=ey2@)(L?BhzFHo@z!vATriPxY zU3>ywR$ggPMWN@Yki*KQsBL;ni6{vRpRkA=cE;ht=`1 z(t#R%rD-dwDD-`q!<9o2pwpLVkQPF(*zv4#(;Ia9t-kc^bwRJ#Pa1%oC1^lTK1^Zw zF`id**ntkypo&7Do9E>+yu2i8kQPGEk|MvA4Ohke=}dG%50M9OWxL#1Q#<|1&@R_- z=&>49QRo-ST}*>RFV|m6L`hf(-DPcFD?`OT0rU_==y23>L>hpdQHCB}EkwQ`_^OrNZEphUOo-5-!|Mmh0?v&XkE6atT!<~juX%JyUhfk2o8`zE?b5lD! zdzM;Z95;0Mc@3&4@J4Q~7w{VH7A)goNmz*dMRZ>)XV?Y}rH9Z3{Q@BX8zQ=6L&VhS zMi#`O=Nu+ao2#PGr<5#gWm()v4bnpBk6qW!$LCn~htRVpLWkQ@at)USa4})8-hQku z+M5H1+fv#R6;ap^FP~J7#fHE514LONX#4cB%Df8PmhyQ7jRO%fEGLae%ES%q?XS71 z%Squ{4XSa_u$;`)po+rY_PBfryqtX3AT0!5bZb{Dt86_-)0yakK1&|JnH`DEni~3{ z26dX_&=WMMqR_WIjI*)_zgDP}h?1}ndg}S*t!zsP4Wox3LWiT2oALnWW5^NYfsJS7 z<1;I8lu~$%H0ggeQQ$`o#vbJ^hoh8nWMLuhyM;a~chbU|6+4!ixh~jw$4TYQ%g7Pf z5it|2+#%74tck*&w)o3;9QHA?un_j45gC+aK@j%O@l+Z_*szATogkHWXAeEBsmn?J zX9LdS1C(YWJz*j89ZTw4IhDE2Bzg#4(3i>suwB0fbdPfPsLCmE z=!qIsQRw^n^|5jkcGJmHB1*zS=%x*ytz1nLK7}5F2p!h&N74Xn2#$`z&&_P`5U=6( z;j*N;CJOxTA)~F-@abe>A?_1}{Q*lJOdTkt{5Pz32NT|S54D))&)*bQe=1L%SsC6za@vud!TY7FPPRL9H6BeEt6 zyU_E#O5KJ-!N#*>EO@d)*gtx$u<{~#;%sUFU9j&-*i*5*8va)xT^cM-IQ!)+^~DbU}}i1~7QAc~jQZ&<*1YDMuc_vhqQL zDhj>Zt=kdY<*=-DiIx$uBrJqJ`S_e}IP}G<=pl&EVOjYo4ZtP{(K*7Pd=pb~pF4nc zC?2b2NpnpU_%Uza8yq;aU=&$ch`V^v)mG-e@2sI_PJ|81O2xG@c>}vr$=uWq|M~J; zE{+>Ie3%AR6!;Yvw+0+IbajvgO`=w8|fi*L64OOV3R=xbn0s8;TorL zr>BXLrmd)=(2HzpsLZE8rw`B|Ered=xZMpL`le0v>~%p;yICH9Io%U`!PdV=men}) zrW#aH=xvjh2jNcNtwCA{J+Ow4l|yVkZJ{&K1-;o;X@H8%>8z=1`tT)_-{8>aXi!C= z&u=!q3J(2;25BMm2)BmH4t!YCt8Aw;(FJ|3G=Pzvns07u=Ut94g+C24HvPq1WXVyZ7n{NA3WIip6%xlIEHy z@a_>W-8pb*!AWFcA@19Yz3%a94kxsZ@1|x>gbjy^1^3A04dw}c^mW9@v#KGFbgCak0!OdB+l4-?_q z!f2u*3fr-9N@ck+G++AtG7`TN6@qqisH>cr57!pfK0uQvLWa&>AQd;bvh}U3scZL` zx+z!U&c3HX6@}fSc=rI@*$ocLXjl>!0&n=jZ!#|?(Ag6X(L?Bh{+Bd>(U~2~#F`rV zrNlH>xw~OGDScR$wnY_%K6cjXBRKR34bnpBFR$cO?j(fJ&&SfU*9E=w5m^93MOMRE zQ$t_ZziMsV>HRdQqR?k=oHiAQeqDpK5PIbJEE902=R8Jdq6@koApo0Wsff+7q`zLO z91eY(22~V#nlIgym&>r@EOcB>geNS7o;R+h6{mMRK@Xt|`gVB$c6qe{y}6p!xdV9I z&v8kXG}lCd9~c#F<%M$ySy+hs%Yfqp`M3o>aqYTH&724uJ``oXB9k{7*b8UY)DEAt zU|}VW8;;LgX;4Lhm*{-z5$^DV8l;8DPgR<@pCgA3{~kwYq6>QKtFizFXEr`(O$|MC z$$l$uh8Ag1MWOF2-+BQKJ>@le!b0e8qLx_MG+XmJJ%ldki{t@}3_5FS=ym#5a>a*= z_cf@Z&==Z7x#7?o-jEVe5*9)qUUH;zK@D{J#CUoLUC{3n0|E)ivgpz9%XK#l!S%QAD$X)Kg-(f~#SJ203vb$MAF=aGy(?g*Hd~3B6K)fIV=s}VZz4b0lp_!aNuyXV*5;%@^3{H zwjsFJbq*UoiT5VT3PEqa)U*kI>V%_}<@=7Lep2)^#$D3z6-*e9pJ#>8=#VJWjrhi3z2U)w_hnG@CW4mjvhi6^nHW?te&6;1K--?naB$Y zgl_Y->^SRxk_9lb^`2NQA0MA^l_Pfm z9cMUM69wM6ul;Zi9Mbqe78c@8^{}<_Y8iH%PG6`rh_GQTpCFYtume8KP3`c{KG~Ib zKd`Kv*Px054;fm^g9C@P+$~AQ!;-KNd3^HTe{hEfeWiyWLWd5&APwN=YGNNX@_IY$ zm3R5j;n}~*lIEHy@UEZcDyMfshYuqQ3voAnvq-7p(BV73Q#02EJ5#bup6#$fcLW`- zv~)G^HX&>;vL*`q%Gc9#xw|3k-DF`Q>>-BN%5r@O`@=s}8bsKzoHY6&lQ$SyIbluh z?W!J=SMll%%gGE4swnW!XM5Ul;IN#0(;zKG?s4gdl}ndAY*N_t@DB2J>d~{8_h7#u zC;uS!+~;f>;5{fP(8=4+?-!5XOo-57IhiR9z_tOQJAv$@L)PLBzfRUff%jSPN!gMD z9bUy&MuR6S#9gTVU@H$lLsL)#5Mjf)lS@*0cN4ZYWu0GjW!EyCJIS6>mhf*y6trJ= zpOoC$@RuG;loew3+&58K2n6R&wx*(GO@s^`ohG$RoE`d)UQ)2>QB2f_q zZP(vHd7^|IUK3@7n4`K*ufQ=wN86>RaUep5jvgTuXRDOah02YyUY>^6=vcBQ3fkj- zz+2qWb{S+Oc(Ov&0S|U6&yet&?4FStfCw9oLiS1JjqD#_Zt79U!@+LK$%SwfVv|Xh zv_%yKzII#AOB^^Hh4j@REkwT2vzL`65Nk8jvnN7_4!6xB3*hQ%Vjnf~*WX%2^Fjh^ zc3rY23f#|oky1)v&5j`p3vsX6*itzN6*~NRRw@l5Z0PVB*<|tt7q+sBH8t+B*{4Qu z+|c0@G^nD$eeRaCa(?(b4bnp7+xssn%j-A%PAlY~GtmWoqBMZf$Tn87riLD~=EiUw z`UMTDDD=MzKP-)x75AJnB9?@O(CcT3x3W)ja4vcXUC=K|0~kD+(^*qPUq5}=0vvj- z+_JPSswnhH2QnPNp^wraErgymxSO&-89Mzy9(wk=py$pj4ZzOBGN5N-#TBoRnU7as zdFiA<6@@2RGL&QdfWIQYh3z5e(Df)^dhu`U>!t@Zjpg)iXFtXDy%}ot` z(#K~j@v>r9M3%Ni6@|X2(E0*6^cfnYh0qU9DQ;zj)y1Op?1|7}S+Of73&1udqx+IW z<62MT$YELON7h7v-xy-A+*$=K7)KTs;x25=q}*8r%Sx6KR2sTq_ms-Bjce%MwNuZ= zDfh@i*z3reDC~%ka2M`w2s?X884I4Q5cV@Wk2buZK-f)7Q3DWR!x|nfl{dJvGq6}w zmy^e3w>;tAhNF}h8dOo>Uvez|%z?ui-m0{Whb3Vl^8J~|TN#hfEJF{W3;GLr00Vo& z!?yB~H`;x?q2h z%X^qyv2x<$vGE-a+ud23^1qrW?8N(~3^?pavak@g)Az#4avuo$wtI{8{!@08_>T(-@t(WUX5*2VBhj`LqF z90DY1h=kf`=g#AWGqB{%tuFXS_;}F&5&tejGIbB|A26tAp!wgy{&$a_-7Wtb3%8SB z6FTxz0Y@E|G=vm7H&>~?%JU<1Zstn#e-+|f{*BZNHg+|C6Xw_b@Rffr4FQH$=2SIS_zm{&ANXegn}2^8 zf7r+d*3wP#0G@194QuM3_0a=wyzn2vM-8gzkDzJMBM)##cC8{Mq9iPIWcD_Xt<=&b zRp}uRN6Mjpk_Irc6DZ704LxlOdu2fmtUQ&g$R7YL0SmCQj?8V zt_?`6PS0K!^eQ!E0SumOPJ=Zy^rr`3tsFR{QK7bs22WOq zyZ@Z?$}J7Bv*=rg8bBB9BXW783H7(jn9lFHzai|@b)_l)tBJz?P(58J4%?qBEQCE^ zS3~8{3TL^{tx?<;ot{gR7P+5zph=MlMjw;SULk??+vO>&u z3sNY@*1^%$!v-`CM96S-;@(guZg61-60xQ(AUTugE#jbIM>$f1Dhm7Py1lD7Y}ipg z)gUbde&k?DWlc7$Ql%TyndpK(N*=(Cb(E~Bq1UW9Kyf#Oeq4hp3O&GcYbG3eg(gxW zO2R_u&GK!vGBxVglpcZz9d?u_VJ;=+yQiy>6=ND{#O$P9-Sv5HwO+a zIFKwX#C@#VK;@)BShHf9Q!^*RhGiv{msH-!CXiTDJN)_TO;*ZEBMqu3@UvexSUK}` zhX!dOa`&!P7O-bRn{2GX@AQ2OIul*c8@H4OaAwCHv8IOpDYRWA_c$yoGc~B9(96U( zD$iXGYx#E#(n9ECT0|-LxB9&*W6wrkV`E9!>`^F&b-0fs3b4?U> z>b`xHBaR^Kkz`>Z>vv=I5qwb9Cc1o)l)=tyUx3wpaw(g4n^p0K8d{@0q6{cz|@HK?M{ zJ4CMAheJ=>nVzr^`oa|Jtt=U?+l3xN7xZQF0PIzl5q;gYyZfgoK8pf7&PN(lQRwrW zj!nX$H|;7Vq9iPYp13WOJ1;KK>ER}N2wl)0Ndp+zQXF$rJH1p=t?9Va%XO2bZBa#` z=efRSJq~@225BL5pAV&bDd#Z!_kU^k1PPYD;m8n`g?p(9`17o@DLP0)Az@Y_ikcEY~pBx*j96$;?&>TIeG>EWa2kI-8H#)Pj!kXIQ56kDt%5lT8vRQ*F z3cOi{Mpn*z&DT@L!;-KN`NU#QR?h8i+lwAT7xc~200uTswnhn zIXrH2m&1pkuDxYMEC~yt=c_$dIm-e5ke2kJhtLK6lQe*lEsZxfHFW3uW4htcEBngQ zwy2`eJ9t&>fJ2YgAT5M`@5Nu$@ml_{FFkuAbXdzP_mc&1^E9!K8hNZ!h2FgTgtdG$ zSrY}m@%2tC4?|DM!b0568z0={xM3|X=})CWgbizXh*aL-%H~#BQ{#TQ@5L0{;jtQ2 zQQ)PD^lFYfylj6N4@<&AQtni7lLn$uRjTM-CmH zYJee{vs2)vm*kNQ% z6n5lzyTiD%pOb}!unWXqu`&)XJ%mbw2piV!5mI@BJ9`VLT zz0}Z5P5?Ca4UOl`VOUs&;|XNG=Q-pJCU9>HFW0`;q`d+hS1Xw zlcjA@MWHVo67vR!K1hSK5c-wjg}dM#=9Un8_PU^_A1(`EV8>UQn;Lp1hYiJW=*=~# zqR`jvonhs|i#-~oh0seppJrto{&@tQi7x2PM@j>*O&|tzBgnBR&y#q0nX5q+gBs-x*lbU~jd55R8zGoZKs^|8G(3x|G3gDMLB?XktmVPLSP z*9(;rQ4$tHe>Wz-GhSZCh0#M0p~La`U1;g)RjEl^{umT*e0?j3fpni zr3yIglVo8b?CMoZT3N4>Vgi*05jGqmc96;&&8MZK$E6QwvLFU8Cy^RdQQ(*APpZLz z!*Y^-qKt(BBszo5TU~w{!|)(O^u_o<0F<2%g&L*8txD-OPXt2*0l}y68h&XiHFI6C%S@BWyBJLDqxSdq)^iha*nP>GDD2cV_b7WP zA?!G^un=~5{gYNY%q-KXG>EWa4eu$HH?kdw=BCCSw<^NQJ@XqhsG`6}T(cd;-3@Da z?in&3mV|}K=gsb;tjB`iX{(v^5W1jmlm;+3v)yg1si9wd{@@AsIDE{1uR#@so)ns- zloSZP^DG$=OTt3v(_($P@uC8uFPcpcp$qy4c>s1otN}eGHYDl(N*uc99BJB$Dhj=S za@TUW(^qPc7D6vw&}$S9{lQ#%_C)BggBy07qsW>l@WWn} zw{hUmf{)3kWjrhi3z0uCbXCq=gWqY-Mf4E5pdXS4V6T%6=qu%e&-3rV%Sx)n z(zF#-6uO;P^+7oF{u-o(&{sLxSa}HAxP+cP5jrd@sh7$Eu)SC4_A9r@2439fu&gvB zYofqM*O}3c1BVveLKYU{{+#;Y3|_@yS$VaLN`nYHy)xO9&E{Fn+L0-2_~8>y{KN@t zH+gpO*ZThX*ZB1_{IB!l5sJ*u^pKq(W#gF2#-_OP*KKT~!WLqG{m + +HELPERS_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) + +src_backup_filename="$1" +src_backup_path="$HELPERS_DIR/../backups/${src_backup_filename}" + +backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'") + +if [ -z "${backups_disk_root}" ]; then + echo "Disk 'backups' not found" + exit 1 +fi + +dest_relative_path=${CLICKHOUSE_DATABASE}/${src_backup_filename} +dest_path=${backups_disk_root}/${dest_relative_path} + +mkdir -p "$(dirname "${dest_path}")" +ln -s "${src_backup_path}" "${dest_path}" + +echo "${dest_relative_path}" From d421636a5fdc3b73fa5cb05e83529483d69e75e2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 09:26:09 +0200 Subject: [PATCH 0517/1170] Protect temporary part directories from removing during RESTORE. --- src/Storages/MergeTree/MergeTreeData.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e10f5a0227..ce27ad24e10 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5557,12 +5557,16 @@ public: auto it = temp_part_dirs.find(part_name); if (it == temp_part_dirs.end()) { - auto temp_part_dir = std::make_shared(disk, fs::path{storage->getRelativeDataPath()} / ("tmp_restore_" + part_name + "-")); + auto temp_dir_deleter = std::make_unique(disk, fs::path{storage->getRelativeDataPath()} / ("tmp_restore_" + part_name + "-")); + auto temp_part_dir = fs::path{temp_dir_deleter->getRelativePath()}.filename(); /// Attaching parts will rename them so it's expected for a temporary part directory not to exist anymore in the end. - temp_part_dir->setShowWarningIfRemoved(false); - it = temp_part_dirs.emplace(part_name, temp_part_dir).first; + temp_dir_deleter->setShowWarningIfRemoved(false); + /// The following holder is needed to prevent clearOldTemporaryDirectories() from clearing `temp_part_dir` before we attach the part. + auto temp_dir_holder = storage->getTemporaryPartDirectoryHolder(temp_part_dir); + it = temp_part_dirs.emplace(part_name, + std::make_pair(std::move(temp_dir_deleter), std::move(temp_dir_holder))).first; } - return it->second->getRelativePath(); + return it->second.first->getRelativePath(); } private: @@ -5588,7 +5592,7 @@ private: size_t num_parts = 0; size_t num_broken_parts = 0; MutableDataPartsVector parts; - std::map> temp_part_dirs; + std::map, scope_guard>> temp_part_dirs; mutable std::mutex mutex; }; From 4e2f8576e5a6e8e39a16334d0c697d5cb09e0469 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 11:50:58 +0200 Subject: [PATCH 0518/1170] Revert "Add settings to replace external engines to Null during create" --- docs/en/operations/settings/settings.md | 12 - src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 35 --- .../test_restore_external_engines/__init__.py | 0 .../configs/backups_disk.xml | 14 -- .../configs/remote_servers.xml | 21 -- .../test_restore_external_engines/test.py | 218 ------------------ 8 files changed, 1 insertion(+), 305 deletions(-) delete mode 100644 tests/integration/test_restore_external_engines/__init__.py delete mode 100644 tests/integration/test_restore_external_engines/configs/backups_disk.xml delete mode 100644 tests/integration/test_restore_external_engines/configs/remote_servers.xml delete mode 100644 tests/integration/test_restore_external_engines/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8739414464e..c3f697c3bdc 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,15 +5608,3 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. - -## restore_replace_external_engines_to_null - -For testing purposes. Replaces all external engines to Null to not initiate external connections. - -Default value: `False` - -## restore_replace_external_table_functions_to_null - -For testing purposes. Replaces all external table functions to Null to not initiate external connections. - -Default value: `False` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27b71558bd3..4fc2034b855 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -893,8 +893,6 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ - M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ - M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8bea0b1eed3..9faf77e9087 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,9 +80,7 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } - - void setNullTableEngine(ASTStorage & storage) - { - auto engine_ast = std::make_shared(); - engine_ast->name = "Null"; - engine_ast->no_empty_args = true; - storage.set(storage.engine, engine_ast); - } - } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.as_table_function) - { - if (getContext()->getSettingsRef().restore_replace_external_table_functions_to_null) - { - const auto & factory = TableFunctionFactory::instance(); - - auto properties = factory.tryGetProperties(create.as_table_function->as()->name); - if (properties && properties->allow_readonly) - return; - if (!create.storage) - { - auto storage_ast = std::make_shared(); - create.set(create.storage, storage_ast); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage should not be created yet, it's a bug."); - create.as_table_function = nullptr; - setNullTableEngine(*create.storage); - } return; - } if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) return; @@ -1043,13 +1015,6 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); } - /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to - /// Null table engine. - else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) - { - if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) - setNullTableEngine(*create.storage); - } return; } diff --git a/tests/integration/test_restore_external_engines/__init__.py b/tests/integration/test_restore_external_engines/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_restore_external_engines/configs/backups_disk.xml b/tests/integration/test_restore_external_engines/configs/backups_disk.xml deleted file mode 100644 index f7d666c6542..00000000000 --- a/tests/integration/test_restore_external_engines/configs/backups_disk.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - - local - /backups/ - - - - - backups - /backups/ - - diff --git a/tests/integration/test_restore_external_engines/configs/remote_servers.xml b/tests/integration/test_restore_external_engines/configs/remote_servers.xml deleted file mode 100644 index 76ad3618339..00000000000 --- a/tests/integration/test_restore_external_engines/configs/remote_servers.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - true - - replica1 - 9000 - - - replica2 - 9000 - - - replica3 - 9000 - - - - - diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py deleted file mode 100644 index cf189f2a6ed..00000000000 --- a/tests/integration/test_restore_external_engines/test.py +++ /dev/null @@ -1,218 +0,0 @@ -import pytest - -import pymysql.cursors -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] - -node1 = cluster.add_instance( - "replica1", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -node2 = cluster.add_instance( - "replica2", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -node3 = cluster.add_instance( - "replica3", - with_zookeeper=True, - with_mysql8=True, - main_configs=configs, - external_dirs=["/backups/"], -) -nodes = [node1, node2, node3] - -backup_id_counter = 0 - - -def new_backup_name(): - global backup_id_counter - backup_id_counter += 1 - return f"Disk('backups', '{backup_id_counter}/')" - - -def cleanup_nodes(nodes, dbname): - for node in nodes: - node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") - - -def fill_nodes(nodes, dbname): - cleanup_nodes(nodes, dbname) - for node in nodes: - node.query( - f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')" - ) - - -def drop_mysql_table(conn, tableName): - with conn.cursor() as cursor: - cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") - - -def get_mysql_conn(cluster): - conn = pymysql.connect( - user="root", - password="clickhouse", - host=cluster.mysql8_ip, - port=cluster.mysql8_port, - ) - return conn - - -def fill_tables(cluster, dbname): - fill_nodes(nodes, dbname) - - conn = get_mysql_conn(cluster) - - with conn.cursor() as cursor: - cursor.execute("DROP DATABASE IF EXISTS clickhouse") - cursor.execute("CREATE DATABASE clickhouse") - cursor.execute("DROP TABLE IF EXISTS clickhouse.inference_table") - cursor.execute( - "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" - ) - cursor.execute( - "INSERT INTO clickhouse.inference_table VALUES (100, X'9fad5e9eefdfb449')" - ) - conn.commit() - - parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" - - node1.query( - f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" - ) - node1.query( - f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})" - ) - - node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") - node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") - - expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" - assert ( - node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") - == expected - ) - assert ( - node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") - == expected - ) - assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" - assert ( - node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" - ) - assert ( - node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") - == "100\n" - ) - assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - - except Exception as ex: - print(ex) - - finally: - cluster.shutdown() - - -def test_restore_table(start_cluster): - fill_tables(cluster, "replicated") - backup_name = new_backup_name() - node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - node2.query(f"BACKUP DATABASE replicated TO {backup_name}") - - node2.query("DROP TABLE replicated.mysql_schema_inference_engine") - node2.query("DROP TABLE replicated.mysql_schema_inference_function") - - node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" - assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" - - node3.query( - f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true" - ) - node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") - - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" - ) - == "1\t100\n" - ) - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function" - ) - == "1\t100\n" - ) - assert ( - node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" - ) - cleanup_nodes(nodes, "replicated") - - -def test_restore_table_null(start_cluster): - fill_tables(cluster, "replicated2") - - backup_name = new_backup_name() - node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") - - node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") - node2.query("DROP TABLE replicated2.mysql_schema_inference_function") - - node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" - assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" - - node3.query( - f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1" - ) - node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") - - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" - ) - == "0\t0\n" - ) - assert ( - node1.query( - "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function" - ) - == "0\t0\n" - ) - assert ( - node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" - ) - assert ( - node1.query( - "SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'" - ) - == "Null\nNull\n" - ) - assert ( - node1.query( - "SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'" - ) - == "MergeTree\n" - ) - cleanup_nodes(nodes, "replicated2") From 06863cf4157765c04759109afa756022dc5e9c55 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 31 Jul 2024 12:12:30 +0200 Subject: [PATCH 0519/1170] fix for allow_experimental_analyzer --- tests/queries/0_stateless/00309_formats.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index 0366cdeea5c..b784907be08 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -12,5 +12,5 @@ SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, a SET enable_named_columns_in_function_tuple = 1; -SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes; -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes; +SELECT 36 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT RowBinaryWithNamesAndTypes SETTINGS allow_experimental_analyzer=1; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, (n, d) AS tuple FROM system.numbers LIMIT 1 FORMAT TabSeparatedWithNamesAndTypes SETTINGS allow_experimental_analyzer=1; From debcc2e61053f763cb84e34e48275dbebd5bd544 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 Jul 2024 10:46:19 +0200 Subject: [PATCH 0520/1170] Fix test test_mutation --- tests/integration/test_backup_restore_on_cluster/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 1b7f4aaa97d..d20e10e8a04 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1054,9 +1054,12 @@ def test_mutation(): backup_name = new_backup_name() node1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") - assert not has_mutation_in_backup("0000000000", backup_name, "default", "tbl") + # mutation #0000000000: "UPDATE x=x+1 WHERE 1" could already finish before starting the backup + # mutation #0000000001: "UPDATE x=x+1+sleep(3) WHERE 1" assert has_mutation_in_backup("0000000001", backup_name, "default", "tbl") + # mutation #0000000002: "UPDATE x=x+1+sleep(3) WHERE 1" assert has_mutation_in_backup("0000000002", backup_name, "default", "tbl") + # mutation #0000000003: not expected assert not has_mutation_in_backup("0000000003", backup_name, "default", "tbl") node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") From f9f17fb61e2ab27f90434b5e3fc9081c061eaae4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Jul 2024 12:32:17 +0200 Subject: [PATCH 0521/1170] Fix reference --- .../0_stateless/03215_parsing_archive_name_s3.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference index b4804c82dc2..b27524812c7 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.reference @@ -1,3 +1,3 @@ -::03215_archive.csv test/::03215_archive.csv -test::03215_archive.csv test/test::03215_archive.csv -test.zip::03215_archive.csv test/test.zip::03215_archive.csv +::03215_archive.csv test/::03215_archive.csv +test::03215_archive.csv test/test::03215_archive.csv +test.zip::03215_archive.csv test/test.zip::03215_archive.csv From 4bf7aa1950f65aa82e85962ab3643f7df0e8bf2a Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 31 Jul 2024 13:30:14 +0200 Subject: [PATCH 0522/1170] Revert "Revert "Add settings to replace external engines to Null during create"" --- docs/en/operations/settings/settings.md | 12 + src/Core/Settings.h | 2 + src/Core/SettingsChangesHistory.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 35 +++ .../test_restore_external_engines/__init__.py | 0 .../configs/backups_disk.xml | 14 ++ .../configs/remote_servers.xml | 21 ++ .../test_restore_external_engines/test.py | 218 ++++++++++++++++++ 8 files changed, 305 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_restore_external_engines/__init__.py create mode 100644 tests/integration/test_restore_external_engines/configs/backups_disk.xml create mode 100644 tests/integration/test_restore_external_engines/configs/remote_servers.xml create mode 100644 tests/integration/test_restore_external_engines/test.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..8739414464e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,15 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## restore_replace_external_engines_to_null + +For testing purposes. Replaces all external engines to Null to not initiate external connections. + +Default value: `False` + +## restore_replace_external_table_functions_to_null + +For testing purposes. Replaces all external table functions to Null to not initiate external connections. + +Default value: `False` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4fc2034b855..27b71558bd3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -893,6 +893,8 @@ class IColumn; M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ + M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ + M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..8bea0b1eed3 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,7 +80,9 @@ static std::initializer_listno_empty_args = true; storage.set(storage.engine, engine_ast); } + + void setNullTableEngine(ASTStorage & storage) + { + auto engine_ast = std::make_shared(); + engine_ast->name = "Null"; + engine_ast->no_empty_args = true; + storage.set(storage.engine, engine_ast); + } + } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const { if (create.as_table_function) + { + if (getContext()->getSettingsRef().restore_replace_external_table_functions_to_null) + { + const auto & factory = TableFunctionFactory::instance(); + + auto properties = factory.tryGetProperties(create.as_table_function->as()->name); + if (properties && properties->allow_readonly) + return; + if (!create.storage) + { + auto storage_ast = std::make_shared(); + create.set(create.storage, storage_ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage should not be created yet, it's a bug."); + create.as_table_function = nullptr; + setNullTableEngine(*create.storage); + } return; + } if (create.is_dictionary || create.is_ordinary_view || create.is_live_view || create.is_window_view) return; @@ -1015,6 +1043,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const /// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one. setDefaultTableEngine(*create.storage, getContext()->getSettingsRef().default_table_engine.value); } + /// For external tables with restore_replace_external_engine_to_null setting we replace external engines to + /// Null table engine. + else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) + { + if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) + setNullTableEngine(*create.storage); + } return; } diff --git a/tests/integration/test_restore_external_engines/__init__.py b/tests/integration/test_restore_external_engines/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_restore_external_engines/configs/backups_disk.xml b/tests/integration/test_restore_external_engines/configs/backups_disk.xml new file mode 100644 index 00000000000..f7d666c6542 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/backups_disk.xml @@ -0,0 +1,14 @@ + + + + + local + /backups/ + + + + + backups + /backups/ + + diff --git a/tests/integration/test_restore_external_engines/configs/remote_servers.xml b/tests/integration/test_restore_external_engines/configs/remote_servers.xml new file mode 100644 index 00000000000..76ad3618339 --- /dev/null +++ b/tests/integration/test_restore_external_engines/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + replica3 + 9000 + + + + + diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py new file mode 100644 index 00000000000..cf189f2a6ed --- /dev/null +++ b/tests/integration/test_restore_external_engines/test.py @@ -0,0 +1,218 @@ +import pytest + +import pymysql.cursors +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +configs = ["configs/remote_servers.xml", "configs/backups_disk.xml"] + +node1 = cluster.add_instance( + "replica1", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node2 = cluster.add_instance( + "replica2", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +node3 = cluster.add_instance( + "replica3", + with_zookeeper=True, + with_mysql8=True, + main_configs=configs, + external_dirs=["/backups/"], +) +nodes = [node1, node2, node3] + +backup_id_counter = 0 + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}/')" + + +def cleanup_nodes(nodes, dbname): + for node in nodes: + node.query(f"DROP DATABASE IF EXISTS {dbname} SYNC") + + +def fill_nodes(nodes, dbname): + cleanup_nodes(nodes, dbname) + for node in nodes: + node.query( + f"CREATE DATABASE {dbname} ENGINE = Replicated('/clickhouse/databases/{dbname}', 'default', '{node.name}')" + ) + + +def drop_mysql_table(conn, tableName): + with conn.cursor() as cursor: + cursor.execute(f"DROP TABLE IF EXISTS `clickhouse`.`{tableName}`") + + +def get_mysql_conn(cluster): + conn = pymysql.connect( + user="root", + password="clickhouse", + host=cluster.mysql8_ip, + port=cluster.mysql8_port, + ) + return conn + + +def fill_tables(cluster, dbname): + fill_nodes(nodes, dbname) + + conn = get_mysql_conn(cluster) + + with conn.cursor() as cursor: + cursor.execute("DROP DATABASE IF EXISTS clickhouse") + cursor.execute("CREATE DATABASE clickhouse") + cursor.execute("DROP TABLE IF EXISTS clickhouse.inference_table") + cursor.execute( + "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" + ) + cursor.execute( + "INSERT INTO clickhouse.inference_table VALUES (100, X'9fad5e9eefdfb449')" + ) + conn.commit() + + parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" + + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_engine ENGINE=MySQL({parameters})" + ) + node1.query( + f"CREATE TABLE {dbname}.mysql_schema_inference_function AS mysql({parameters})" + ) + + node1.query(f"CREATE TABLE {dbname}.merge_tree (id UInt64, b String) ORDER BY id") + node1.query(f"INSERT INTO {dbname}.merge_tree VALUES (100, 'abc')") + + expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_engine") + == expected + ) + assert ( + node1.query(f"DESCRIBE TABLE {dbname}.mysql_schema_inference_function") + == expected + ) + assert node1.query(f"SELECT id FROM mysql({parameters})") == "100\n" + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_engine") == "100\n" + ) + assert ( + node1.query(f"SELECT id FROM {dbname}.mysql_schema_inference_function") + == "100\n" + ) + assert node1.query(f"SELECT id FROM {dbname}.merge_tree") == "100\n" + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + + +def test_restore_table(start_cluster): + fill_tables(cluster, "replicated") + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated") + + node2.query(f"BACKUP DATABASE replicated TO {backup_name}") + + node2.query("DROP TABLE replicated.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated") + + assert node3.query("EXISTS replicated.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated.mysql_schema_inference_function") == "0\n" + + node3.query( + f"RESTORE DATABASE replicated FROM {backup_name} SETTINGS allow_different_database_def=true" + ) + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") + + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" + ) + == "1\t100\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_function" + ) + == "1\t100\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated.merge_tree") == "1\t100\n" + ) + cleanup_nodes(nodes, "replicated") + + +def test_restore_table_null(start_cluster): + fill_tables(cluster, "replicated2") + + backup_name = new_backup_name() + node2.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") + + node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") + + node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") + node2.query("DROP TABLE replicated2.mysql_schema_inference_function") + + node3.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") + + assert node3.query("EXISTS replicated2.mysql_schema_inference_engine") == "0\n" + assert node3.query("EXISTS replicated2.mysql_schema_inference_function") == "0\n" + + node3.query( + f"RESTORE DATABASE replicated2 FROM {backup_name} SETTINGS allow_different_database_def=1, allow_different_table_def=1 SETTINGS restore_replace_external_engines_to_null=1, restore_replace_external_table_functions_to_null=1" + ) + node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") + + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" + ) + == "0\t0\n" + ) + assert ( + node1.query( + "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_function" + ) + == "0\t0\n" + ) + assert ( + node1.query("SELECT count(), sum(id) FROM replicated2.merge_tree") == "1\t100\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%mysql%'" + ) + == "Null\nNull\n" + ) + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name like '%merge_tree%'" + ) + == "MergeTree\n" + ) + cleanup_nodes(nodes, "replicated2") From c81d3322b18b0eb4b45b91ac019a8c4f42d7518d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 13:39:30 +0200 Subject: [PATCH 0523/1170] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index c29b604d23d..5c432350768 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-distributed-cache +# Tags: long, no-random-settings, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 64a7a413619cda2edb336079740cca2a3d6503f7 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Wed, 31 Jul 2024 13:42:38 +0200 Subject: [PATCH 0524/1170] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8bea0b1eed3..5e846868478 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,9 @@ 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", {{"restore_replace_external_table_functions_to_null", false, false, "New setting."}, + {"restore_replace_external_engines_to_null", false, false, "New setting."} + }}, {"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"}, @@ -80,9 +83,7 @@ static std::initializer_list Date: Wed, 31 Jul 2024 12:00:09 +0000 Subject: [PATCH 0525/1170] Fix --- tests/integration/helpers/cluster.py | 25 +++++++++++++------- tests/integration/helpers/retry_decorator.py | 7 ++++-- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7d80fbe90f8..7f0a9154be9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2692,11 +2692,13 @@ class ClickHouseCluster: images_pull_cmd = self.base_cmd + ["pull"] # sometimes dockerhub/proxy can be flaky - retry( - log_function=lambda exception: logging.info( - "Got exception pulling images: %s", exception - ), - )(run_and_check)(images_pull_cmd) + def logging_pulling_images(**kwargs): + if "exception" in kwargs: + logging.info( + "Got exception pulling images: %s", kwargs["exception"] + ) + + retry(log_function=logging_pulling_images)(run_and_check)(images_pull_cmd) if self.with_zookeeper_secure and self.base_zookeeper_cmd: logging.debug("Setup ZooKeeper Secure") @@ -2969,11 +2971,16 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - retry( - log_function=lambda exception: logging.info( + def logging_azurite_initialization(exception, retry_number, sleep_time): + logging.info( f"Azurite initialization failed with error: {exception}" - ), - )(run_and_check)(azurite_start_cmd) + ) + + retry( + log_function=logging_azurite_initialization, + )( + run_and_check + )(azurite_start_cmd) self.up_called = True logging.info("Trying to connect to Azurite") self.wait_azurite_to_start() diff --git a/tests/integration/helpers/retry_decorator.py b/tests/integration/helpers/retry_decorator.py index aaa040464c2..e7bafbe29c1 100644 --- a/tests/integration/helpers/retry_decorator.py +++ b/tests/integration/helpers/retry_decorator.py @@ -8,7 +8,7 @@ def retry( delay: float = 1, backoff: float = 1.5, jitter: float = 2, - log_function=lambda *args, **kwargs: None, + log_function=None, # should take **kwargs or arguments: `retry_number`, `exception` and `sleep_time` retriable_expections_list: List[Type[BaseException]] = [Exception], ): def inner(func): @@ -26,8 +26,11 @@ def retry( break if not should_retry or (retry == retries - 1): raise e - log_function(retry=retry, exception=e) sleep_time = current_delay + random.uniform(0, jitter) + if log_function is not None: + log_function( + retry_number=retry, exception=e, sleep_time=sleep_time + ) time.sleep(sleep_time) current_delay *= backoff From 15e0033016eb0e23a7e6f512d5096e50863e3187 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 31 Jul 2024 12:09:49 +0000 Subject: [PATCH 0526/1170] Bring back the strict check Also update the doc. --- tests/integration/README.md | 9 +++++---- tests/integration/test_storage_s3_queue/test.py | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index ab984b7bd04..a8deb97b526 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -124,10 +124,11 @@ You can just open shell inside a container by overwritting the command: ### Parallel test execution On the CI, we run a number of parallel runners (5 at the time of this writing), each on its own -Docker container. These runner containers spawn more containers for the services needed such as -ZooKeeper, MySQL, PostgreSQL and minio, among others. Within each runner, tests are parallelized -using [pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` -to [distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the +Docker container. These runner containers spawn more containers for each test for the services +needed such as ZooKeeper, MySQL, PostgreSQL and minio, among others. This means that tests do not +share any services among them. Within each runner, tests are parallelized using +[pytest-xdist](https://pytest-xdist.readthedocs.io/en/stable/). We're using `--dist=loadfile` to +[distribute the load](https://pytest-xdist.readthedocs.io/en/stable/distribution.html). In the documentation words: this guarantees that all tests in a file run in the same worker. This means that any test within the same file will never execute their tests in parallel. They'll be executed on the same worker one after the other. diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index e3445d14cdb..9a97e8c23d1 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -907,7 +907,7 @@ def test_max_set_age(started_cluster): file_with_error = f"max_set_age_fail_{uuid4().hex[:8]}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) - wait_for_condition(lambda: failed_count + 1 <= get_object_storage_failures()) + wait_for_condition(lambda: failed_count + 1 == get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( @@ -920,7 +920,7 @@ def test_max_set_age(started_cluster): ) ) - wait_for_condition(lambda: failed_count + 2 <= get_object_storage_failures()) + wait_for_condition(lambda: failed_count + 2 == get_object_storage_failures()) node.query("SYSTEM FLUSH LOGS") assert "Cannot parse input" in node.query( From 9d14053cfe7867fd688c08b493c264ee679a4a61 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 Jul 2024 12:17:56 +0000 Subject: [PATCH 0527/1170] Proper fix for short circuit execution with nested dictGetOrDefaultt --- src/Columns/ColumnFunction.cpp | 26 ++++++++++++++----- src/Interpreters/ExpressionActions.cpp | 4 --- ...sted_short_circuit_functions_bug.reference | 2 ++ ...210_nested_short_circuit_functions_bug.sql | 3 +++ 4 files changed, 24 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference create mode 100644 tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index fc81efaac0c..18c343c6ca6 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -296,16 +296,28 @@ ColumnWithTypeAndName ColumnFunction::reduce() const function->getName(), toString(args), toString(captured)); ColumnsWithTypeAndName columns = captured_columns; - IFunction::ShortCircuitSettings settings; /// Arguments of lazy executed function can also be lazy executed. - /// But we shouldn't execute arguments if this function is short circuit, - /// because it will handle lazy executed arguments by itself. - if (is_short_circuit_argument && !function->isShortCircuit(settings, args)) + if (is_short_circuit_argument) { - for (auto & col : columns) + IFunction::ShortCircuitSettings settings; + /// We shouldn't execute all arguments if this function is short circuit, + /// because it will handle lazy executed arguments by itself. + /// Execute only arguments with disabled lazy execution. + if (function->isShortCircuit(settings, args)) { - if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(col.column)) - col = arg->reduce(); + for (size_t i : settings.arguments_with_disabled_lazy_execution) + { + if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(columns[i].column)) + columns[i] = arg->reduce(); + } + } + else + { + for (auto & col : columns) + { + if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(col.column)) + col = arg->reduce(); + } } } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index d832f568cb8..8993830af14 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -195,10 +195,6 @@ static void setLazyExecutionInfo( } lazy_execution_info.short_circuit_ancestors_info[parent].insert(indexes.begin(), indexes.end()); - /// After checking arguments_with_disabled_lazy_execution, if there is no relation with parent, - /// disable the current node. - if (indexes.empty()) - lazy_execution_info.can_be_lazy_executed = false; } else /// If lazy execution is disabled for one of parents, we should disable it for current node. diff --git a/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql new file mode 100644 index 00000000000..923f1e3be1f --- /dev/null +++ b/tests/queries/0_stateless/03210_nested_short_circuit_functions_bug.sql @@ -0,0 +1,3 @@ +select if(equals(materialize('abc'), 'aws.lambda.duration'), if(toFloat64(materialize('x86_74')) < 50.0000, 0, 1), 0) settings short_circuit_function_evaluation='enable'; +select if(equals(materialize('abc'), 'aws.lambda.duration'), if(toFloat64(materialize('x86_74')) < 50.0000, 0, 1), 0) settings short_circuit_function_evaluation='force_enable'; + From 6b7c5eb5da1be1fc31d4ebfd4f0dfa0c6a6e728c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Jul 2024 14:09:07 +0200 Subject: [PATCH 0528/1170] Fix drop --- src/Storages/StorageKeeperMap.cpp | 34 +++++++++++++++-------- src/Storages/StorageKeeperMap.h | 21 ++++++++++---- tests/integration/test_keeper_map/test.py | 5 ++-- 3 files changed, 41 insertions(+), 19 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1559b442e43..0634c7be6ee 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -79,6 +79,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int LIMIT_EXCEEDED; extern const int CANNOT_RESTORE_TABLE; + extern const int INVALID_STATE; } namespace @@ -497,7 +498,7 @@ StorageKeeperMap::StorageKeeperMap( } - table_is_valid = true; + table_status = TableStatus::VALID; /// we are the first table created for the specified Keeper path, i.e. we are the first replica return; } @@ -656,7 +657,18 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E void StorageKeeperMap::drop() { - checkTable(); + auto current_table_status = getTableStatus(); + if (current_table_status == TableStatus::UNKNOWN) + { + static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " + "once a connection is established and metadata is verified"; + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + } + + /// if only column metadata is wrong we can still drop the table correctly + if (current_table_status == TableStatus::INVALID_KEEPER_STRUCTURE) + return; + auto client = getClient(); // we allow ZNONODE in case we got hardware error on previous drop @@ -1017,11 +1029,11 @@ UInt64 StorageKeeperMap::keysLimit() const return keys_limit; } -std::optional StorageKeeperMap::isTableValid() const +StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const { std::lock_guard lock{init_mutex}; - if (table_is_valid.has_value()) - return table_is_valid; + if (table_status != TableStatus::UNKNOWN) + return table_status; [&] { @@ -1034,7 +1046,7 @@ std::optional StorageKeeperMap::isTableValid() const if (metadata_stat.numChildren == 0) { - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; return; } @@ -1045,7 +1057,7 @@ std::optional StorageKeeperMap::isTableValid() const "Table definition does not match to the one stored in the path {}. Stored definition: {}", zk_root_path, stored_metadata_string); - table_is_valid = false; + table_status = TableStatus::INVALID_METADATA; return; } @@ -1058,7 +1070,7 @@ std::optional StorageKeeperMap::isTableValid() const Coordination::Responses responses; client->tryMulti(requests, responses); - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; if (responses[0]->error != Coordination::Error::ZOK) { LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); @@ -1077,18 +1089,18 @@ std::optional StorageKeeperMap::isTableValid() const return; } - table_is_valid = true; + table_status = TableStatus::VALID; } catch (const Coordination::Exception & e) { tryLogCurrentException(log); if (!Coordination::isHardwareError(e.code)) - table_is_valid = false; + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; } }(); - return table_is_valid; + return table_status; } Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index cfbb35ab2fe..8ed348a4f6f 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -80,8 +80,8 @@ public: template void checkTable() const { - auto is_table_valid = isTableValid(); - if (!is_table_valid.has_value()) + auto current_table_status = getTableStatus(); + if (table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " "once a connection is established and metadata is verified"; @@ -94,10 +94,10 @@ public: } } - if (!*is_table_valid) + if (current_table_status != TableStatus::VALID) { static constexpr auto error_msg - = "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table"; + = "Failed to activate table because of invalid metadata in ZooKeeper. Please DROP/DETACH table"; if constexpr (throw_on_error) throw Exception(ErrorCodes::INVALID_STATE, error_msg); else @@ -111,7 +111,15 @@ public: private: bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); - std::optional isTableValid() const; + enum class TableStatus : uint8_t + { + UNKNOWN, + INVALID_METADATA, + INVALID_KEEPER_STRUCTURE, + VALID + }; + + TableStatus getTableStatus() const; void restoreDataImpl( const BackupPtr & backup, @@ -143,7 +151,8 @@ private: mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; mutable std::mutex init_mutex; - mutable std::optional table_is_valid; + + mutable TableStatus table_status{TableStatus::UNKNOWN}; LoggerPtr log; }; diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 7aee5df5746..4b1bcd11cfe 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -67,6 +67,7 @@ def run_query(query): def test_keeper_map_without_zk(started_cluster): + run_query("DROP TABLE IF EXISTS test_keeper_map_without_zk SYNC") assert_keeper_exception_after_partition( "CREATE TABLE test_keeper_map_without_zk (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_without_zk') PRIMARY KEY(key);" ) @@ -107,12 +108,12 @@ def test_keeper_map_without_zk(started_cluster): ) assert "Failed to activate table because of invalid metadata in ZooKeeper" in error - node.query("DETACH TABLE test_keeper_map_without_zk") - client.stop() def test_keeper_map_with_failed_drop(started_cluster): + run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop SYNC") + run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop_another SYNC") run_query( "CREATE TABLE test_keeper_map_with_failed_drop (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_with_failed_drop') PRIMARY KEY(key);" ) From 406ac2279ecbfc24913548dfcf459c55dd450723 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 14:48:33 +0200 Subject: [PATCH 0529/1170] Analyzer: Do not traverse unresolved subtrees --- src/Planner/findParallelReplicasQuery.cpp | 12 +++++------- src/Planner/findQueryForParallelReplicas.h | 2 +- ...5_analyzer_replace_with_dummy_tables.reference | 0 .../03215_analyzer_replace_with_dummy_tables.sql | 15 +++++++++++++++ 4 files changed, 21 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index c89a70be541..1140f30ad9c 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -113,13 +113,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre return res; } -class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitor +class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitorWithContext { public: - using Base = InDepthQueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * table_node = node->as(); auto * table_function_node = node->as(); @@ -134,21 +134,19 @@ public: ColumnsDescription(storage_snapshot->getColumns(get_column_options)), storage_snapshot); - auto dummy_table_node = std::make_shared(std::move(storage_dummy), context); + auto dummy_table_node = std::make_shared(std::move(storage_dummy), getContext()); dummy_table_node->setAlias(node->getAlias()); replacement_map.emplace(node.get(), std::move(dummy_table_node)); } } - ContextPtr context; std::unordered_map replacement_map; }; QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) { - ReplaceTableNodeToDummyVisitor visitor; - visitor.context = context; + ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); return query->cloneAndReplace(visitor.replacement_map); diff --git a/src/Planner/findQueryForParallelReplicas.h b/src/Planner/findQueryForParallelReplicas.h index f5dc69dfa0e..cdce4ad0b47 100644 --- a/src/Planner/findQueryForParallelReplicas.h +++ b/src/Planner/findQueryForParallelReplicas.h @@ -13,7 +13,7 @@ using QueryTreeNodePtr = std::shared_ptr; struct SelectQueryOptions; -/// Find a qury which can be executed with parallel replicas up to WithMergableStage. +/// Find a query which can be executed with parallel replicas up to WithMergableStage. /// Returned query will always contain some (>1) subqueries, possibly with joins. const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options); diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql new file mode 100644 index 00000000000..12d2bd627a7 --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -0,0 +1,15 @@ +create table t (number UInt64) engine MergeTree order by number; + +SELECT 1 +FROM +( + SELECT number IN ( + SELECT number + FROM view( + SELECT number + FROM numbers(1) + ) + ) + FROM t +) +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; From 7160e954c16100e963371e416878837437569d74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 13:32:24 +0200 Subject: [PATCH 0530/1170] 02995_new_settings_history: Update ref --- ..._23_12_1.tsv => 02995_baseline_24_7_1.tsv} | 182 +++++++++++++++--- .../0_stateless/02995_new_settings_history.sh | 14 +- 2 files changed, 167 insertions(+), 29 deletions(-) rename tests/queries/0_stateless/{02995_baseline_23_12_1.tsv => 02995_baseline_24_7_1.tsv} (82%) diff --git a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv b/tests/queries/0_stateless/02995_baseline_24_7_1.tsv similarity index 82% rename from tests/queries/0_stateless/02995_baseline_23_12_1.tsv rename to tests/queries/0_stateless/02995_baseline_24_7_1.tsv index a391473e7c9..6c830da8646 100644 --- a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_1.tsv @@ -11,23 +11,28 @@ allow_create_index_without_type 0 allow_custom_error_code_in_throwif 0 allow_ddl 1 allow_deprecated_database_ordinary 0 +allow_deprecated_error_prone_window_functions 0 +allow_deprecated_snowflake_conversion_functions 0 allow_deprecated_syntax_for_merge_tree 0 allow_distributed_ddl 1 allow_drop_detached 0 allow_execute_multiif_columnar 1 allow_experimental_alter_materialized_view_structure 1 -allow_experimental_analyzer 0 +allow_experimental_analyzer 1 allow_experimental_annoy_index 0 allow_experimental_bigint_types 1 allow_experimental_codecs 0 allow_experimental_database_atomic 1 allow_experimental_database_materialized_mysql 0 allow_experimental_database_materialized_postgresql 0 -allow_experimental_database_replicated 0 +allow_experimental_database_replicated 1 +allow_experimental_dynamic_type 0 +allow_experimental_full_text_index 0 allow_experimental_funnel_functions 0 allow_experimental_geo_types 1 allow_experimental_hash_functions 0 allow_experimental_inverted_index 0 +allow_experimental_join_condition 0 allow_experimental_lightweight_delete 1 allow_experimental_live_view 0 allow_experimental_map_type 1 @@ -40,12 +45,15 @@ allow_experimental_query_cache 1 allow_experimental_query_deduplication 0 allow_experimental_refreshable_materialized_view 0 allow_experimental_s3queue 1 -allow_experimental_shared_merge_tree 0 +allow_experimental_shared_merge_tree 1 +allow_experimental_statistic 0 allow_experimental_statistics 0 allow_experimental_undrop_table_query 1 allow_experimental_usearch_index 0 +allow_experimental_variant_type 0 allow_experimental_window_functions 1 allow_experimental_window_view 0 +allow_get_client_http_header 0 allow_hyperscan 1 allow_introspection_functions 0 allow_named_collection_override_by_default 1 @@ -58,17 +66,21 @@ allow_prefetched_read_pool_for_remote_filesystem 1 allow_push_predicate_when_subquery_contains_with 1 allow_settings_after_format_in_insert 0 allow_simdjson 1 +allow_statistic_optimize 0 allow_statistics_optimize 0 allow_suspicious_codecs 0 allow_suspicious_fixed_string_types 0 allow_suspicious_indices 0 allow_suspicious_low_cardinality_types 0 +allow_suspicious_primary_key 0 allow_suspicious_ttl_expressions 0 +allow_suspicious_variant_types 0 allow_unrestricted_reads_from_keeper 0 alter_move_to_space_execute_async 0 alter_partition_verbose_result 0 alter_sync 1 analyze_index_with_space_filling_curves 1 +analyzer_compatibility_join_using_top_level_identifier 0 annoy_index_search_k_nodes -1 any_join_distinct_right_table_keys 0 apply_deleted_mask 1 @@ -76,20 +88,42 @@ apply_mutations_on_fly 0 asterisk_include_alias_columns 0 asterisk_include_materialized_columns 0 async_insert 0 +async_insert_busy_timeout_decrease_rate 0.2 +async_insert_busy_timeout_increase_rate 0.2 +async_insert_busy_timeout_max_ms 200 +async_insert_busy_timeout_min_ms 50 async_insert_busy_timeout_ms 200 async_insert_cleanup_timeout_ms 1000 async_insert_deduplicate 0 -async_insert_max_data_size 1000000 +async_insert_max_data_size 10485760 async_insert_max_query_number 450 +async_insert_poll_timeout_ms 10 async_insert_stale_timeout_ms 0 async_insert_threads 16 +async_insert_use_adaptive_busy_timeout 1 async_query_sending_for_remote 1 async_socket_for_remote 1 +azure_allow_parallel_part_upload 1 azure_create_new_file_on_insert 0 +azure_ignore_file_doesnt_exist 0 azure_list_object_keys_size 1000 +azure_max_blocks_in_multipart_upload 50000 +azure_max_inflight_parts_for_one_file 20 +azure_max_single_part_copy_size 268435456 azure_max_single_part_upload_size 104857600 azure_max_single_read_retries 4 +azure_max_unexpected_write_error_retries 4 +azure_max_upload_part_size 5368709120 +azure_min_upload_part_size 16777216 +azure_sdk_max_retries 10 +azure_sdk_retry_initial_backoff_ms 10 +azure_sdk_retry_max_backoff_ms 1000 +azure_skip_empty_files 0 +azure_strict_upload_part_size 0 +azure_throw_on_zero_files_match 0 azure_truncate_on_insert 0 +azure_upload_part_size_multiply_factor 2 +azure_upload_part_size_multiply_parts_count_threshold 500 background_buffer_flush_schedule_pool_size 16 background_common_pool_size 8 background_distributed_schedule_pool_size 16 @@ -107,6 +141,7 @@ backup_restore_keeper_max_retries 20 backup_restore_keeper_retry_initial_backoff_ms 100 backup_restore_keeper_retry_max_backoff_ms 5000 backup_restore_keeper_value_max_size 1048576 +backup_restore_s3_retry_attempts 1000 backup_threads 16 bool_false_representation false bool_true_representation true @@ -115,6 +150,7 @@ calculate_text_stack_trace 1 cancel_http_readonly_queries_on_client_close 0 cast_ipv4_ipv6_default_on_conversion_error 0 cast_keep_nullable 0 +cast_string_to_dynamic_use_inference 0 check_query_single_value_result 1 check_referential_table_dependencies 0 check_table_dependencies 1 @@ -123,6 +159,7 @@ cloud_mode 0 cloud_mode_engine 1 cluster_for_parallel_replicas collect_hash_table_stats_during_aggregation 1 +collect_hash_table_stats_during_joins 1 column_names_for_schema_inference compatibility compatibility_ignore_auto_increment_in_create_table 0 @@ -141,9 +178,12 @@ count_distinct_optimization 0 create_index_ignore_unique 0 create_replicated_merge_tree_fault_injection_probability 0 create_table_empty_primary_key_by_default 0 +cross_join_min_bytes_to_compress 1073741824 +cross_join_min_rows_to_compress 10000000 cross_to_inner_join_rewrite 1 data_type_default_nullable 0 database_atomic_wait_for_drop_and_detach_synchronously 0 +database_replicated_allow_heavy_create 0 database_replicated_allow_only_replicated_engine 0 database_replicated_allow_replicated_engine_arguments 1 database_replicated_always_detach_permanently 0 @@ -156,15 +196,19 @@ date_time_overflow_behavior ignore decimal_check_overflow 1 deduplicate_blocks_in_dependent_materialized_views 0 default_database_engine Atomic +default_materialized_view_sql_security DEFINER default_max_bytes_in_join 1000000000 -default_table_engine None +default_normal_view_sql_security INVOKER +default_table_engine MergeTree default_temporary_table_engine Memory +default_view_definer CURRENT_USER describe_compact_output 0 describe_extend_object_types 0 describe_include_subcolumns 0 describe_include_virtual_columns 0 dialect clickhouse dictionary_use_async_executor 0 +dictionary_validate_primary_key_type 0 distinct_overflow_mode throw distributed_aggregation_memory_efficient 1 distributed_background_insert_batch 0 @@ -182,6 +226,7 @@ distributed_directory_monitor_sleep_time_ms 100 distributed_directory_monitor_split_batch_on_failure 0 distributed_foreground_insert 0 distributed_group_by_no_merge 0 +distributed_insert_skip_read_only_replicas 0 distributed_product_mode deny distributed_push_down_limit 1 distributed_replica_error_cap 1000 @@ -191,6 +236,7 @@ do_not_merge_across_partitions_select_final 0 drain_timeout 3 empty_result_for_aggregation_by_constant_keys_on_empty_set 1 empty_result_for_aggregation_by_empty_set 0 +enable_blob_storage_log 1 enable_debug_queries 0 enable_deflate_qpl_codec 0 enable_early_constant_folding 1 @@ -205,6 +251,7 @@ enable_job_stack_trace 0 enable_lightweight_delete 1 enable_memory_bound_merging_of_aggregation_results 1 enable_multiple_prewhere_read_steps 1 +enable_named_columns_in_function_tuple 1 enable_optimize_predicate_expression 1 enable_optimize_predicate_expression_to_final_subquery 1 enable_order_by_all 1 @@ -216,7 +263,9 @@ enable_sharing_sets_for_mutations 1 enable_software_prefetch_in_aggregation 1 enable_unaligned_array_join 0 enable_url_encoding 1 +enable_vertical_final 1 enable_writes_to_query_cache 1 +enable_zstd_qat_codec 0 engine_file_allow_create_multiple_files 0 engine_file_empty_if_not_exists 0 engine_file_skip_empty_files 0 @@ -231,10 +280,12 @@ external_storage_max_read_rows 0 external_storage_rw_timeout_sec 300 external_table_functions_use_nulls 1 external_table_strict_query 0 +extract_key_value_pairs_max_pairs_per_row 1000 extract_kvp_max_pairs_per_row 1000 extremes 0 fallback_to_stale_replicas_for_distributed_queries 1 filesystem_cache_max_download_size 137438953472 +filesystem_cache_reserve_space_wait_lock_timeout_milliseconds 1000 filesystem_cache_segments_batch_size 20 filesystem_prefetch_max_memory_usage 1073741824 filesystem_prefetch_min_bytes_for_single_read_task 2097152 @@ -278,7 +329,9 @@ format_regexp_escaping_rule Raw format_regexp_skip_unmatched 0 format_schema format_template_resultset +format_template_resultset_format format_template_row +format_template_row_format format_template_rows_between_delimiter \n format_tsv_null_representation \\N formatdatetime_f_prints_single_zero 0 @@ -288,8 +341,11 @@ fsync_metadata 1 function_implementation function_json_value_return_type_allow_complex 0 function_json_value_return_type_allow_nullable 0 +function_locate_has_mysql_compatible_argument_order 1 function_range_max_elements_in_block 500000000 function_sleep_max_microseconds_per_block 3000000 +function_visible_width_behavior 1 +geo_distance_returns_float64_on_float64_arguments 1 glob_expansion_max_elements 1000 grace_hash_join_initial_buckets 1 grace_hash_join_max_buckets 1024 @@ -300,8 +356,10 @@ group_by_use_nulls 0 handle_kafka_error_mode default handshake_timeout_ms 10000 hdfs_create_new_file_on_insert 0 +hdfs_ignore_file_doesnt_exist 0 hdfs_replication 0 hdfs_skip_empty_files 0 +hdfs_throw_on_zero_files_match 0 hdfs_truncate_on_insert 0 hedged_connection_timeout_ms 50 hsts_max_age 0 @@ -326,10 +384,14 @@ http_skip_not_found_url_for_globs 1 http_wait_end_of_query 0 http_write_exception_in_output_format 1 http_zlib_compression_level 3 +iceberg_engine_ignore_schema_evolution 0 idle_connection_timeout 3600 ignore_cold_parts_seconds 0 ignore_data_skipping_indices +ignore_drop_queries_probability 0 +ignore_materialized_views_with_dropped_target_table 0 ignore_on_cluster_for_replicated_access_entities_queries 0 +ignore_on_cluster_for_replicated_named_collections_queries 0 ignore_on_cluster_for_replicated_udf_queries 0 implicit_transaction 0 input_format_allow_errors_num 0 @@ -341,12 +403,14 @@ input_format_arrow_import_nested 0 input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_avro_allow_missing_fields 0 input_format_avro_null_as_default 0 +input_format_binary_decode_types_in_binary_format 0 input_format_bson_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_csv_allow_cr_end_of_line 0 input_format_csv_allow_variable_number_of_columns 0 input_format_csv_allow_whitespace_or_tab_as_delimiter 0 input_format_csv_arrays_as_nested_csv 0 +input_format_csv_deserialize_separate_columns_into_tuple 1 input_format_csv_detect_header 1 input_format_csv_empty_as_default 1 input_format_csv_enum_as_number 0 @@ -354,29 +418,37 @@ input_format_csv_skip_first_lines 0 input_format_csv_skip_trailing_empty_lines 0 input_format_csv_trim_whitespaces 1 input_format_csv_try_infer_numbers_from_strings 0 +input_format_csv_try_infer_strings_from_quoted_tuples 1 input_format_csv_use_best_effort_in_schema_inference 1 input_format_csv_use_default_on_bad_values 0 input_format_custom_allow_variable_number_of_columns 0 input_format_custom_detect_header 1 input_format_custom_skip_trailing_empty_lines 0 input_format_defaults_for_omitted_fields 1 +input_format_force_null_for_omitted_fields 0 +input_format_hive_text_allow_variable_number_of_columns 1 input_format_hive_text_collection_items_delimiter  input_format_hive_text_fields_delimiter  input_format_hive_text_map_keys_delimiter  input_format_import_nested_json 0 input_format_ipv4_default_on_conversion_error 0 input_format_ipv6_default_on_conversion_error 0 +input_format_json_case_insensitive_column_matching 0 input_format_json_compact_allow_variable_number_of_columns 0 input_format_json_defaults_for_missing_elements_in_named_tuple 1 input_format_json_ignore_unknown_keys_in_named_tuple 1 +input_format_json_ignore_unnecessary_fields 1 input_format_json_infer_incomplete_types_as_strings 1 input_format_json_named_tuples_as_objects 1 input_format_json_read_arrays_as_strings 1 input_format_json_read_bools_as_numbers 1 +input_format_json_read_bools_as_strings 1 input_format_json_read_numbers_as_strings 1 input_format_json_read_objects_as_strings 1 +input_format_json_throw_on_bad_escape_sequence 1 input_format_json_try_infer_named_tuples_from_objects 1 input_format_json_try_infer_numbers_from_strings 0 +input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects 0 input_format_json_validate_types_from_metadata 1 input_format_max_bytes_to_read_for_schema_inference 33554432 input_format_max_rows_to_read_for_schema_inference 25000 @@ -384,11 +456,13 @@ input_format_msgpack_number_of_columns 0 input_format_mysql_dump_map_column_names 1 input_format_mysql_dump_table_name input_format_native_allow_types_conversion 1 +input_format_native_decode_types_in_binary_format 0 input_format_null_as_default 1 input_format_orc_allow_missing_columns 1 input_format_orc_case_insensitive_column_matching 0 input_format_orc_filter_push_down 1 input_format_orc_import_nested 0 +input_format_orc_read_use_writer_time_zone 0 input_format_orc_row_batch_size 100000 input_format_orc_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_orc_use_fast_decoder 1 @@ -398,17 +472,21 @@ input_format_parquet_case_insensitive_column_matching 0 input_format_parquet_filter_push_down 1 input_format_parquet_import_nested 0 input_format_parquet_local_file_min_bytes_for_seek 8192 -input_format_parquet_max_block_size 8192 +input_format_parquet_max_block_size 65409 +input_format_parquet_prefer_block_bytes 16744704 input_format_parquet_preserve_order 0 input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference 0 +input_format_parquet_use_native_reader 0 input_format_protobuf_flatten_google_wrappers 0 input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference 0 input_format_record_errors_file_path input_format_skip_unknown_fields 1 input_format_try_infer_dates 1 input_format_try_infer_datetimes 1 +input_format_try_infer_exponent_floats 0 input_format_try_infer_integers 1 input_format_tsv_allow_variable_number_of_columns 0 +input_format_tsv_crlf_end_of_line 0 input_format_tsv_detect_header 1 input_format_tsv_empty_as_default 0 input_format_tsv_enum_as_number 0 @@ -450,7 +528,12 @@ joined_subquery_requires_alias 1 kafka_disable_num_consumers_limit 0 kafka_max_wait_ms 5000 keeper_map_strict_mode 0 +keeper_max_retries 10 +keeper_retry_initial_backoff_ms 100 +keeper_retry_max_backoff_ms 5000 legacy_column_name_of_tuple_literal 0 +lightweight_deletes_sync 2 +lightweight_mutation_projection_mode throw limit 0 live_view_heartbeat_interval 15 load_balancing random @@ -461,7 +544,7 @@ local_filesystem_read_prefetch 0 lock_acquire_timeout 120 log_comment log_formatted_queries 0 -log_processors_profiles 0 +log_processors_profiles 1 log_profile_events 1 log_queries 1 log_queries_cut_to_length 100000 @@ -474,6 +557,8 @@ log_query_views 1 low_cardinality_allow_in_native_format 1 low_cardinality_max_dictionary_size 8192 low_cardinality_use_single_dictionary_for_part 0 +materialize_skip_indexes_on_insert 1 +materialize_statistics_on_insert 1 materialize_ttl_after_modify 1 materialized_views_ignore_errors 0 max_alter_threads \'auto(16)\' @@ -501,6 +586,7 @@ max_distributed_depth 5 max_download_buffer_size 10485760 max_download_threads 4 max_entries_for_hash_table_stats 10000 +max_estimated_execution_time 0 max_execution_speed 0 max_execution_speed_bytes 0 max_execution_time 0 @@ -528,7 +614,9 @@ max_network_bandwidth_for_user 0 max_network_bytes 0 max_number_of_partitions_for_independent_aggregation 128 max_parallel_replicas 1 +max_parser_backtracks 1000000 max_parser_depth 1000 +max_parsing_threads \'auto(16)\' max_partition_size_to_drop 50000000000 max_partitions_per_insert_block 100 max_partitions_to_read -1 @@ -537,6 +625,7 @@ max_query_size 262144 max_read_buffer_size 1048576 max_read_buffer_size_local_fs 131072 max_read_buffer_size_remote_fs 0 +max_recursive_cte_evaluation_depth 1000 max_remote_read_network_bandwidth 0 max_remote_read_network_bandwidth_for_server 0 max_remote_write_network_bandwidth 0 @@ -549,7 +638,7 @@ max_result_rows 0 max_rows_in_distinct 0 max_rows_in_join 0 max_rows_in_set 0 -max_rows_in_set_to_optimize_join 100000 +max_rows_in_set_to_optimize_join 0 max_rows_to_group_by 0 max_rows_to_read 0 max_rows_to_read_leaf 0 @@ -557,6 +646,7 @@ max_rows_to_sort 0 max_rows_to_transfer 0 max_sessions_for_user 0 max_size_to_preallocate_for_aggregation 100000000 +max_size_to_preallocate_for_joins 100000000 max_streams_for_merge_tree_reading 0 max_streams_multiplier_for_merge_tables 5 max_streams_to_max_threads_ratio 1 @@ -592,6 +682,7 @@ merge_tree_min_bytes_per_task_for_remote_reading 4194304 merge_tree_min_rows_for_concurrent_read 163840 merge_tree_min_rows_for_concurrent_read_for_remote_filesystem 163840 merge_tree_min_rows_for_seek 0 +merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0 merge_tree_use_const_size_tasks_for_remote_reading 1 metrics_perf_events_enabled 0 metrics_perf_events_list @@ -604,6 +695,8 @@ min_count_to_compile_expression 3 min_count_to_compile_sort_description 3 min_execution_speed 0 min_execution_speed_bytes 0 +min_external_table_block_size_bytes 268402944 +min_external_table_block_size_rows 1048449 min_free_disk_space_for_temporary_data 0 min_hit_rate_to_use_consecutive_keys_optimization 0.5 min_insert_block_size_bytes 268402944 @@ -619,8 +712,8 @@ mutations_execute_subqueries_on_initiator 0 mutations_max_literal_size_to_replace 16384 mutations_sync 0 mysql_datatypes_support_level -mysql_map_fixed_string_to_text_in_show_columns 0 -mysql_map_string_to_text_in_show_columns 0 +mysql_map_fixed_string_to_text_in_show_columns 1 +mysql_map_string_to_text_in_show_columns 1 mysql_max_rows_to_insert 65536 network_compression_method LZ4 network_zstd_compression_level 1 @@ -647,6 +740,7 @@ optimize_group_by_constant_keys 1 optimize_group_by_function_keys 1 optimize_if_chain_to_multiif 0 optimize_if_transform_strings_to_enum 0 +optimize_injective_functions_in_group_by 1 optimize_injective_functions_inside_uniq 1 optimize_min_equality_disjunction_chain_length 3 optimize_min_inequality_conjunction_chain_length 3 @@ -664,7 +758,7 @@ optimize_redundant_functions_in_order_by 1 optimize_respect_aliases 1 optimize_rewrite_aggregate_function_with_if 1 optimize_rewrite_array_exists_to_has 0 -optimize_rewrite_sum_if_to_count_if 0 +optimize_rewrite_sum_if_to_count_if 1 optimize_skip_merged_partitions 0 optimize_skip_unused_shards 0 optimize_skip_unused_shards_limit 1000 @@ -674,9 +768,10 @@ optimize_sorting_by_input_stream_properties 1 optimize_substitute_columns 0 optimize_syntax_fuse_functions 0 optimize_throw_if_noop 0 +optimize_time_filter_with_preimage 1 optimize_trivial_approximate_count_query 0 optimize_trivial_count_query 1 -optimize_trivial_insert_select 1 +optimize_trivial_insert_select 0 optimize_uniq_to_count 1 optimize_use_implicit_projections 1 optimize_use_projections 1 @@ -685,13 +780,19 @@ os_thread_priority 0 output_format_arrow_compression_method lz4_frame output_format_arrow_fixed_string_as_fixed_byte_array 1 output_format_arrow_low_cardinality_as_dictionary 0 -output_format_arrow_string_as_string 0 +output_format_arrow_string_as_string 1 +output_format_arrow_use_64_bit_indexes_for_dictionary 0 +output_format_arrow_use_signed_indexes_for_dictionary 1 output_format_avro_codec output_format_avro_rows_in_file 1 output_format_avro_string_column_pattern output_format_avro_sync_interval 16384 +output_format_binary_encode_types_in_binary_format 0 output_format_bson_string_as_string 0 +output_format_compression_level 3 +output_format_compression_zstd_window_log 0 output_format_csv_crlf_end_of_line 0 +output_format_csv_serialize_tuple_into_separate_columns 1 output_format_decimal_trailing_zeros 0 output_format_enable_streaming 0 output_format_json_array_of_rows 0 @@ -705,27 +806,34 @@ output_format_json_skip_null_value_in_named_tuples 0 output_format_json_validate_utf8 0 output_format_markdown_escape_special_characters 0 output_format_msgpack_uuid_representation ext -output_format_orc_compression_method lz4 +output_format_native_encode_types_in_binary_format 0 +output_format_orc_compression_method zstd output_format_orc_row_index_stride 10000 -output_format_orc_string_as_string 0 +output_format_orc_string_as_string 1 output_format_parallel_formatting 1 output_format_parquet_batch_size 1024 output_format_parquet_compliant_nested_types 1 -output_format_parquet_compression_method lz4 +output_format_parquet_compression_method zstd output_format_parquet_data_page_size 1048576 output_format_parquet_fixed_string_as_fixed_byte_array 1 output_format_parquet_parallel_encoding 1 output_format_parquet_row_group_size 1000000 output_format_parquet_row_group_size_bytes 536870912 -output_format_parquet_string_as_string 0 -output_format_parquet_use_custom_encoder 0 +output_format_parquet_string_as_string 1 +output_format_parquet_use_custom_encoder 1 output_format_parquet_version 2.latest -output_format_pretty_color 1 +output_format_parquet_write_page_index 1 +output_format_pretty_color auto +output_format_pretty_display_footer_column_names 1 +output_format_pretty_display_footer_column_names_min_rows 50 output_format_pretty_grid_charset UTF-8 +output_format_pretty_highlight_digit_groups 1 output_format_pretty_max_column_pad_width 250 output_format_pretty_max_rows 10000 output_format_pretty_max_value_width 10000 -output_format_pretty_row_numbers 0 +output_format_pretty_max_value_width_apply_for_single_value 0 +output_format_pretty_row_numbers 1 +output_format_pretty_single_large_number_tip_threshold 1000000 output_format_protobuf_nullables_with_google_wrappers 0 output_format_schema output_format_sql_insert_include_column_names 1 @@ -734,15 +842,22 @@ output_format_sql_insert_quote_names 1 output_format_sql_insert_table_name table output_format_sql_insert_use_replace 0 output_format_tsv_crlf_end_of_line 0 +output_format_values_escape_quote_with_quote 0 output_format_write_statistics 1 +page_cache_inject_eviction 0 parallel_distributed_insert_select 0 parallel_replica_offset 0 +parallel_replicas_allow_in_with_subquery 1 parallel_replicas_count 0 parallel_replicas_custom_key parallel_replicas_custom_key_filter_type default +parallel_replicas_custom_key_range_lower 0 +parallel_replicas_custom_key_range_upper 0 parallel_replicas_for_non_replicated_merge_tree 0 +parallel_replicas_mark_segment_size 128 parallel_replicas_min_number_of_granules_to_enable 0 parallel_replicas_min_number_of_rows_per_replica 0 +parallel_replicas_prefer_local_join 1 parallel_replicas_single_task_marks_count_multiplier 2 parallel_view_processing 0 parallelize_output_from_storages 1 @@ -755,11 +870,14 @@ parts_to_delay_insert 0 parts_to_throw_insert 0 periodic_live_view_refresh 60 poll_interval 10 +postgresql_connection_attempt_timeout 2 postgresql_connection_pool_auto_close_connection 0 +postgresql_connection_pool_retries 2 postgresql_connection_pool_size 16 postgresql_connection_pool_wait_timeout 5000 precise_float_parsing 0 prefer_column_name_to_alias 0 +prefer_external_sort_block_bytes 16744704 prefer_global_in_and_join 0 prefer_localhost_replica 1 prefer_warmed_unmerged_parts_seconds 0 @@ -767,7 +885,7 @@ preferred_block_size_bytes 1000000 preferred_max_column_in_block_size_bytes 0 preferred_optimize_projection_name prefetch_buffer_size 1048576 -print_pretty_type_names 0 +print_pretty_type_names 1 priority 0 query_cache_compress_entries 1 query_cache_max_entries 0 @@ -778,8 +896,10 @@ query_cache_nondeterministic_function_handling throw query_cache_share_between_users 0 query_cache_squash_partial_results 1 query_cache_store_results_of_queries_with_nondeterministic_functions 0 +query_cache_system_table_handling throw query_cache_ttl 60 query_plan_aggregation_in_order 1 +query_plan_convert_outer_join_to_inner_join 1 query_plan_enable_multithreading_after_window_functions 1 query_plan_enable_optimizations 1 query_plan_execute_functions_after_sorting 1 @@ -788,6 +908,8 @@ query_plan_lift_up_array_join 1 query_plan_lift_up_union 1 query_plan_max_optimizations_to_apply 10000 query_plan_merge_expressions 1 +query_plan_merge_filters 0 +query_plan_optimize_prewhere 1 query_plan_optimize_primary_key 1 query_plan_optimize_projection 1 query_plan_push_down_limit 1 @@ -806,7 +928,9 @@ read_backoff_min_events 2 read_backoff_min_interval_between_events_ms 1000 read_backoff_min_latency_ms 1000 read_from_filesystem_cache_if_exists_otherwise_bypass_cache 0 +read_from_page_cache_if_exists_otherwise_bypass_cache 0 read_in_order_two_level_merge_threshold 100 +read_in_order_use_buffering 1 read_overflow_mode throw read_overflow_mode_leaf throw read_priority 0 @@ -835,17 +959,20 @@ result_overflow_mode throw rewrite_count_distinct_if_with_count_distinct_implementation 0 s3_allow_parallel_part_upload 1 s3_check_objects_after_upload 0 +s3_connect_timeout_ms 1000 s3_create_new_file_on_insert 0 s3_disable_checksum 0 -s3_http_connection_pool_size 1000 +s3_ignore_file_doesnt_exist 0 s3_list_object_keys_size 1000 s3_max_connections 1024 s3_max_get_burst 0 s3_max_get_rps 0 s3_max_inflight_parts_for_one_file 20 +s3_max_part_number 10000 s3_max_put_burst 0 s3_max_put_rps 0 s3_max_redirects 10 +s3_max_single_operation_copy_size 33554432 s3_max_single_part_upload_size 33554432 s3_max_single_read_retries 4 s3_max_unexpected_write_error_retries 4 @@ -860,6 +987,8 @@ s3_truncate_on_insert 0 s3_upload_part_size_multiply_factor 2 s3_upload_part_size_multiply_parts_count_threshold 500 s3_use_adaptive_timeouts 1 +s3_validate_request_settings 1 +s3queue_allow_experimental_sharded_mode 0 s3queue_default_zookeeper_path /clickhouse/s3queue/ s3queue_enable_logging_to_s3queue_log 0 schema_inference_cache_require_modification_time_for_url 1 @@ -887,6 +1016,8 @@ sleep_after_receiving_query_ms 0 sleep_in_send_data_ms 0 sleep_in_send_tables_status_ms 0 sort_overflow_mode throw +split_intersecting_parts_ranges_into_layers_final 1 +split_parts_ranges_into_intersecting_and_non_intersecting_final 1 splitby_max_substrings_includes_remaining_string 0 stop_refreshable_materialized_views_on_startup 0 storage_file_read_method pread @@ -898,8 +1029,10 @@ stream_poll_timeout_ms 500 system_events_show_zero_values 0 table_function_remote_max_addresses 1000 tcp_keep_alive_timeout 290 +temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds 600000 temporary_files_codec LZ4 temporary_live_view_timeout 1 +throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert 1 throw_if_no_data_to_insert 1 throw_on_error_from_cache_on_write_operations 0 throw_on_max_partitions_per_insert_block 1 @@ -912,8 +1045,10 @@ totals_mode after_having_exclusive trace_profile_events 0 transfer_overflow_mode throw transform_null_in 0 +traverse_shadow_remote_data_paths 0 union_default_mode unknown_packet_in_send_data 0 +update_insert_deduplication_token_in_dependent_materialized_views 0 use_cache_for_count_from_files 1 use_client_time_zone 0 use_compact_format_in_distributed_parts_names 1 @@ -923,12 +1058,15 @@ use_index_for_in_with_subqueries 1 use_index_for_in_with_subqueries_max_values 0 use_local_cache_for_remote_storage 1 use_mysql_types_in_show_columns 0 +use_page_cache_for_disks_without_file_cache 0 use_query_cache 0 use_skip_indexes 1 use_skip_indexes_if_final 0 use_structure_from_insertion_table_in_table_functions 2 use_uncompressed_cache 0 +use_variant_as_common_type 0 use_with_fill_by_sorting_prefix 1 +validate_experimental_and_suspicious_types_inside_nested_types 1 validate_polygons 1 wait_changes_become_visible_after_commit_mode wait_unknown wait_for_async_insert 1 diff --git a/tests/queries/0_stateless/02995_new_settings_history.sh b/tests/queries/0_stateless/02995_new_settings_history.sh index 8de98c55b6a..917dacc04b0 100755 --- a/tests/queries/0_stateless/02995_new_settings_history.sh +++ b/tests/queries/0_stateless/02995_new_settings_history.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # Note that this is a broad check. A per version check is done in the upgrade test -# Baseline generated with 23.12.1 -# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_23_12_1.tsv +# Baseline generated with 24.7.1 +# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_1.tsv $CLICKHOUSE_LOCAL --query " WITH old_settings AS ( - SELECT * FROM file('${CUR_DIR}/02995_baseline_23_12_1.tsv', 'TSV', 'name String, default String') + SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_1.tsv', 'TSV', 'name String, default String') ), new_settings AS ( @@ -21,7 +21,7 @@ $CLICKHOUSE_LOCAL --query " ) SELECT * FROM ( - SELECT 'PLEASE ADD THE NEW SETTING TO SettingsChangesHistory.h: ' || name || ' WAS ADDED', + SELECT 'PLEASE ADD THE NEW SETTING TO SettingsChangesHistory.cpp: ' || name || ' WAS ADDED', FROM new_settings WHERE (name NOT IN ( SELECT name @@ -29,17 +29,17 @@ $CLICKHOUSE_LOCAL --query " )) AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM system.settings_changes - WHERE splitByChar('.', version())[1] >= '24' + WHERE splitByChar('.', version)[1]::UInt64 >= 24 AND splitByChar('.', version)[2]::UInt64 > 7 )) UNION ALL ( - SELECT 'PLEASE ADD THE SETTING VALUE CHANGE TO SettingsChangesHistory.h: ' || name || ' WAS CHANGED FROM ' || old_settings.default || ' TO ' || new_settings.default, + SELECT 'PLEASE ADD THE SETTING VALUE CHANGE TO SettingsChangesHistory.cpp: ' || name || ' WAS CHANGED FROM ' || old_settings.default || ' TO ' || new_settings.default, FROM new_settings LEFT JOIN old_settings ON new_settings.name = old_settings.name WHERE (new_settings.default != old_settings.default) AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM system.settings_changes - WHERE splitByChar('.', version())[1] >= '24' + WHERE splitByChar('.', version)[1]::UInt64 >= 24 AND splitByChar('.', version)[2]::UInt64 > 7 )) ) ) From b178eea09ec80fed40b5043ccf1635d95b9cf19b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 31 Jul 2024 14:59:17 +0200 Subject: [PATCH 0531/1170] Fix broken settings --- src/Core/SettingsChangesHistory.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..ecc558e64d7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,16 @@ 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.12",{ + }}, + {"24.11",{ + }}, + {"24.10",{ + }}, + {"24.9", { + }}, + {"24.8", {{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, + }}, {"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"}, @@ -76,7 +86,6 @@ static std::initializer_list Date: Wed, 31 Jul 2024 15:11:55 +0200 Subject: [PATCH 0532/1170] Try a less conflict prone format --- src/Core/SettingsChangesHistory.cpp | 709 +++++++++++++++++----------- 1 file changed, 441 insertions(+), 268 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ecc558e64d7..21c89b3c5c5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,274 +57,447 @@ 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.12",{ - }}, - {"24.11",{ - }}, - {"24.10",{ - }}, - {"24.9", { - }}, - {"24.8", {{"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - }}, - {"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_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe 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"}, - {"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", + { + } + }, + {"24.11", + { + } + }, + {"24.10", + { + } + }, + {"24.9", + { + } + }, + {"24.8", + { + {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, + } + }, + {"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_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe 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"}, + {"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"} + } + }, }; From bca21bc6c735b710ed15061e41790066b5ffd843 Mon Sep 17 00:00:00 2001 From: sakulali Date: Wed, 31 Jul 2024 21:33:14 +0800 Subject: [PATCH 0533/1170] add test cases --- src/Common/Config/ConfigProcessor.cpp | 11 ++++------- ...89_clickhouse_client_config_format.reference | 4 ++++ .../01889_clickhouse_client_config_format.sh | 17 +++++++++++++++++ 3 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index f0b83c035c8..c4b4a1d5e7e 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -654,22 +654,19 @@ XMLDocumentPtr ConfigProcessor::parseConfig(const std::string & config_path) while (std::getline(file, line)) { const size_t pos = firstNonWhitespacePos(line); - if (pos == std::string::npos) - continue; if (pos < line.size() && '<' == line[pos]) { maybe_xml = true; + break; } - - break; + else if (pos != std::string::npos) + break; } } - if (maybe_xml) return dom_parser.parse(config_path); - else - return YAMLParser::parse(config_path); + return YAMLParser::parse(config_path); } } diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index afe27ddb063..ef0d9ffc538 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -18,3 +18,7 @@ autodetect xml (non leading whitespaces) 2 autodetect yaml 2 +autodetect invalid xml +Code: 1000, e.code() = 0, SAXParseException: Invalid token in '/config_test.badxml', line 2 column 12, Stack trace (when copying this message, always include the lines below): +autodetect invalid yaml +Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 14d67f943f1..9a44ec0d5f5 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -15,6 +15,8 @@ yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml autodetect_xml_with_leading_whitespace_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.config autodetect_xml_non_leading_whitespace_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.cfg autodetect_yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.properties +autodetect_invalid_xml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.badxml +autodetect_invalid_yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.badyaml function cleanup() { @@ -27,6 +29,8 @@ function cleanup() rm "${autodetect_xml_with_leading_whitespace_config:?}" rm "${autodetect_xml_non_leading_whitespace_config:?}" rm "${autodetect_yaml_config:?}" + rm "${autodetect_invalid_xml_config:?}" + rm "${autodetect_invalid_yaml_config:?}" } trap cleanup EXIT @@ -70,6 +74,15 @@ EOL cat > "$autodetect_yaml_config" < "$autodetect_invalid_xml_config" < + +EOL +cat > "$autodetect_invalid_yaml_config" <&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#Poco::Exception. ##" +echo 'autodetect invalid yaml' +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file From 7bd8061979204973d4a1c1ac956ab80fc1c6be38 Mon Sep 17 00:00:00 2001 From: heguangnan Date: Wed, 31 Jul 2024 21:35:55 +0800 Subject: [PATCH 0534/1170] fix test --- .../0_stateless/03214_count_distinct_null_key_memory_leak.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql index d8428ec6b4a..84804e4e016 100644 --- a/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql +++ b/tests/queries/0_stateless/03214_count_distinct_null_key_memory_leak.sql @@ -13,6 +13,6 @@ ORDER BY c SETTINGS index_granularity = 8192, allow_nullable_key=1; INSERT INTO testnull(b,c) SELECT toString(rand64()) AS b, toString(rand64()) AS c FROM numbers(1000000); -SELECT count(distinct b) FROM testnull GROUP BY a SETTINGS max_memory_usage = 54748364; -- {serverError MEMORY_LIMIT_EXCEEDED} +SELECT count(distinct b) FROM testnull GROUP BY a SETTINGS max_memory_usage = 10000000; -- {serverError MEMORY_LIMIT_EXCEEDED} DROP TABLE testnull; \ No newline at end of file From 31c142a96d49fbe1b46b21e4cdad366546dc7864 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:44:54 +0100 Subject: [PATCH 0535/1170] make it possible to rerun test_storage_delta and test_checking_s3_blobs_paranoid --- .../test_checking_s3_blobs_paranoid/test.py | 2 ++ tests/integration/test_storage_delta/test.py | 29 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index dde636b5d29..afe8449b44a 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -61,6 +61,7 @@ def test_upload_after_check_works(cluster, broken_s3): node.query( """ + DROP TABLE IF EXISTS s3_upload_after_check_works; CREATE TABLE s3_upload_after_check_works ( id Int64, data String @@ -631,6 +632,7 @@ def test_no_key_found_disk(cluster, broken_s3): node.query( """ + DROP TABLE IF EXISTS no_key_found_disk; CREATE TABLE no_key_found_disk ( id Int64 ) ENGINE=MergeTree() diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 67cc7cdd6da..698becc18c4 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -52,6 +52,11 @@ def get_spark(): return builder.master("local").getOrCreate() +def remove_local_directory_contents(local_path): + for local_file in glob.glob(local_path + "/**"): + os.unlink(local_file) + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -169,6 +174,9 @@ def test_single_log_file(started_cluster): inserted_data ) + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_partition_by(started_cluster): instance = started_cluster.instances["node1"] @@ -191,6 +199,7 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 + remove_local_directory_contents(f"/{TABLE_NAME}") def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] @@ -266,6 +275,9 @@ def test_checkpoint(started_cluster): ).strip() ) + remove_local_directory_contents(f"/{TABLE_NAME}") + spark.sql(f"DROP TABLE {TABLE_NAME}") + def test_multiple_log_files(started_cluster): instance = started_cluster.instances["node1"] @@ -304,6 +316,8 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_metadata(started_cluster): instance = started_cluster.instances["node1"] @@ -337,6 +351,9 @@ def test_metadata(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_types(started_cluster): TABLE_NAME = "test_types" @@ -409,6 +426,9 @@ def test_types(started_cluster): ] ) + remove_local_directory_contents(f"/{result_file}") + spark.sql(f"DROP TABLE {TABLE_NAME}") + def test_restart_broken(started_cluster): instance = started_cluster.instances["node1"] @@ -470,6 +490,9 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_restart_broken_table_function(started_cluster): instance = started_cluster.instances["node1"] @@ -524,6 +547,9 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + os.unlink(parquet_data_path) + remove_local_directory_contents(f"/{TABLE_NAME}") + def test_partition_columns(started_cluster): instance = started_cluster.instances["node1"] @@ -721,3 +747,6 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) + + remove_local_directory_contents(f"/{TABLE_NAME}") + spark.sql(f"DROP TABLE {TABLE_NAME}") From 2a2dba63cc0182247754a5a4819cb89f21825bfd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:48:04 +0000 Subject: [PATCH 0536/1170] Automatic style fix --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From 7dbd3d75340522195e7d08a725cf5ae116288c8e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:51:38 +0000 Subject: [PATCH 0537/1170] Automatic style fix --- tests/integration/test_storage_delta/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 698becc18c4..e485bc90ee0 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -201,6 +201,7 @@ def test_partition_by(started_cluster): remove_local_directory_contents(f"/{TABLE_NAME}") + def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session From 27f4f468b976e445e8b0dbc198ea9f0a9c62855b Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 31 Jul 2024 14:55:00 +0100 Subject: [PATCH 0538/1170] make it possible to rerun test_recovery_time_metric multiple times --- tests/integration/test_recovery_time_metric/test.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py index 628f2e744e0..6fcf2fad423 100644 --- a/tests/integration/test_recovery_time_metric/test.py +++ b/tests/integration/test_recovery_time_metric/test.py @@ -21,6 +21,7 @@ def start_cluster(): 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') """ @@ -28,6 +29,7 @@ def test_recovery_time_metric(start_cluster): node.query( """ + DROP TABLE IF EXISTS rdb.t; CREATE TABLE rdb.t ( `x` UInt32 @@ -51,3 +53,9 @@ def test_recovery_time_metric(start_cluster): ).strip() ) assert ret > 0 + + node.query( + """ + DROP DATABASE rdb + """ + ) From d6de2be4395e1bcc62ab32ad1d5b02e9db080303 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:08:18 +0200 Subject: [PATCH 0539/1170] Fix build --- src/Planner/findParallelReplicasQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 1140f30ad9c..39edb1e6516 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -144,7 +144,7 @@ public: std::unordered_map replacement_map; }; -QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) +QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const ContextPtr & context) { ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); From 9ffbd8f5073e180592a494742d1dc3af4427b55f Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:13:43 +0000 Subject: [PATCH 0540/1170] Possible fix --- .../03164_s3_settings_for_queries_and_merges.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index ac2070fbd76..e43c9ae7717 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -22,18 +22,18 @@ SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; SELECT - ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE event_date >= yesterday() AND type = 'QueryFinish' +WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; SELECT - ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE event_date >= yesterday() AND type = 'QueryFinish' +WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%OPTIMIZE TABLE t_compact_bytes_s3 FINAL%'; From 67f4792b77f2a2cf0de21ead6e95c3635d26aa88 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:16:40 +0000 Subject: [PATCH 0541/1170] Style check --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From 1f1f0528ce3a1fb20ceee5513523787a14718b80 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 31 Jul 2024 14:32:07 +0000 Subject: [PATCH 0542/1170] Prefer constant to INPUT in PlannerActionsVisitor. --- src/Planner/PlannerActionsVisitor.cpp | 11 +++++++- ...lyzer_materialized_constants_bug.reference | 3 +++ ...15_analyzer_materialized_constants_bug.sql | 26 +++++++++++++++++++ 3 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference create mode 100644 tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 1960855792c..57457493844 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -491,7 +491,16 @@ public: { auto it = node_name_to_node.find(node_name); if (it != node_name_to_node.end()) - return it->second; + { + /// It is possible that ActionsDAG already has an input with the same name as constant. + /// In this case, prefer constant to input. + /// Constatns affect function return type, which should be consistent with QueryTree. + /// Query example: + /// SELECT materialize(toLowCardinality('b')) || 'a' FROM remote('127.0.0.{1,2}', system, one) GROUP BY 'a' + bool materialized_input = it->second->type == ActionsDAG::ActionType::INPUT && !it->second->column; + if (!materialized_input) + return it->second; + } const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference new file mode 100644 index 00000000000..584e34c0cde --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.reference @@ -0,0 +1,3 @@ +ba +\N +1 111111111111111111111111111111111111111 diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql new file mode 100644 index 00000000000..f9ec28d09d8 --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql @@ -0,0 +1,26 @@ +SET allow_experimental_analyzer = 1; + +SELECT concat(materialize(toLowCardinality('b')), 'a') FROM remote('127.0.0.{1,2}', system, one) GROUP BY 'a'; + +SELECT concat(NULLIF(1, materialize(toLowCardinality(1))), concat(NULLIF(1, 1))) FROM remote('127.0.0.{1,2}', system, one) GROUP BY concat(NULLIF(1, 1)); + +DROP TABLE IF EXISTS test__fuzz_21; +CREATE TABLE test__fuzz_21 +( + `x` Decimal(18, 10) +) +ENGINE = MergeTree +ORDER BY x; + +INSERT INTO test__fuzz_21 VALUES (1), (2), (3); + +WITH ( + SELECT CAST(toFixedString(toFixedString(materialize(toFixedString('111111111111111111111111111111111111111', 39)), 39), 39), 'UInt128') + ) AS v +SELECT + coalesce(materialize(toLowCardinality(toNullable(1))), 10, NULL), + max(v) +FROM remote('127.0.0.{1,2}', default, test__fuzz_21) +GROUP BY + coalesce(NULL), + coalesce(1, 10, 10, materialize(NULL)); From e31569a065d4c81cdea671727c39983d7f3a84e5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:32:37 +0200 Subject: [PATCH 0543/1170] Expect an unknown cluster --- .../0_stateless/03215_analyzer_replace_with_dummy_tables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql index 12d2bd627a7..6d084c2ac50 100644 --- a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -12,4 +12,4 @@ FROM ) FROM t ) -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; -- { serverError CLUSTER_DOESNT_EXIST } From 8c36fbf4eddeba9282b53f726976b55f62d3ee19 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:42:38 +0000 Subject: [PATCH 0544/1170] Remove unnecessary change --- access/quotas.list | Bin 0 -> 1 bytes access/roles.list | Bin 0 -> 1 bytes access/row_policies.list | Bin 0 -> 1 bytes access/settings_profiles.list | Bin 0 -> 1 bytes access/users.list | Bin 0 -> 1 bytes .../03164_s3_settings_for_queries_and_merges.sql | 4 ++-- 6 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 access/quotas.list create mode 100644 access/roles.list create mode 100644 access/row_policies.list create mode 100644 access/settings_profiles.list create mode 100644 access/users.list diff --git a/access/quotas.list b/access/quotas.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/roles.list b/access/roles.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/row_policies.list b/access/row_policies.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/settings_profiles.list b/access/settings_profiles.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/access/users.list b/access/users.list new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index e43c9ae7717..94e390537df 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -25,7 +25,7 @@ SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE type = 'QueryFinish' +WHERE event_date >= yesterday() AND type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; @@ -33,7 +33,7 @@ SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log -WHERE type = 'QueryFinish' +WHERE event_date >= yesterday() AND type = 'QueryFinish' AND current_database = currentDatabase() AND query ilike '%OPTIMIZE TABLE t_compact_bytes_s3 FINAL%'; From 20ec27f9dc79d7ee81cd06f1587de83c8ce81441 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:46:53 +0000 Subject: [PATCH 0545/1170] Remove trach dir --- access/quotas.list | Bin 1 -> 0 bytes access/roles.list | Bin 1 -> 0 bytes access/row_policies.list | Bin 1 -> 0 bytes access/settings_profiles.list | Bin 1 -> 0 bytes access/users.list | Bin 1 -> 0 bytes 5 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 access/quotas.list delete mode 100644 access/roles.list delete mode 100644 access/row_policies.list delete mode 100644 access/settings_profiles.list delete mode 100644 access/users.list diff --git a/access/quotas.list b/access/quotas.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/roles.list b/access/roles.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/row_policies.list b/access/row_policies.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/settings_profiles.list b/access/settings_profiles.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/access/users.list b/access/users.list deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 From 650737890299f8cad2c77ad46022ee0a37b284eb Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:49:30 +0000 Subject: [PATCH 0546/1170] Fix erroe with profile event name --- .../0_stateless/03164_s3_settings_for_queries_and_merges.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index 94e390537df..001ef382850 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -22,7 +22,7 @@ SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; SELECT - ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' @@ -30,7 +30,7 @@ WHERE event_date >= yesterday() AND type = 'QueryFinish' AND query ilike '%INSERT INTO t_compact_bytes_s3 SELECT number, number, number%'; SELECT - ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsError'], + ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' From c3c653e7692a755c3467b77e866555734d50ef50 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 Jul 2024 14:58:52 +0000 Subject: [PATCH 0547/1170] Better --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ .../config.d/database_catalog_drop_table_concurrency.xml | 3 +++ tests/config/install.sh | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/config/config.d/database_catalog_drop_table_concurrency.xml diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index a8e5fd7e6aa..48b01a9df43 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,6 +1406,8 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids).contains(uuid) ? "table still exists" : "table dropped successfully"); + /// TSA doesn't support unique_lock if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " diff --git a/tests/config/config.d/database_catalog_drop_table_concurrency.xml b/tests/config/config.d/database_catalog_drop_table_concurrency.xml new file mode 100644 index 00000000000..ac118625f4e --- /dev/null +++ b/tests/config/config.d/database_catalog_drop_table_concurrency.xml @@ -0,0 +1,3 @@ + + 256 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 1b0edc5fc16..7c4b36dc4bd 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -21,6 +21,7 @@ ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ From 5ab8c0357a84d0265bdb43f43795b06317854772 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Jul 2024 16:30:14 +0200 Subject: [PATCH 0548/1170] Fix potential busy loop in keepFreeSpaceRatioFunc --- src/Interpreters/Cache/FileCache.cpp | 30 +++++++++++++------ src/Interpreters/Cache/IFileCachePriority.h | 8 ++++- .../Cache/LRUFileCachePriority.cpp | 21 +++++++++++-- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- .../Cache/SLRUFileCachePriority.cpp | 15 ++++++---- .../Cache/SLRUFileCachePriority.h | 2 +- 6 files changed, 57 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index a88c0de2cfe..bf8dd24a1db 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -998,18 +998,19 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() FileCacheReserveStat stat; EvictionCandidates eviction_candidates; - bool limits_satisfied = true; + IFileCachePriority::DesiredSizeStatus desired_size_status; try { /// Collect at most `keep_up_free_space_remove_batch` elements to evict, /// (we use batches to make sure we do not block cache for too long, /// by default the batch size is quite small). - limits_satisfied = main_priority->collectCandidatesForEviction( + desired_size_status = main_priority->collectCandidatesForEviction( desired_size, desired_elements_num, keep_up_free_space_remove_batch, stat, eviction_candidates, lock); #ifdef DEBUG_OR_SANITIZER_BUILD /// Let's make sure that we correctly processed the limits. - if (limits_satisfied && eviction_candidates.size() < keep_up_free_space_remove_batch) + if (desired_size_status == IFileCachePriority::DesiredSizeStatus::SUCCESS + && eviction_candidates.size() < keep_up_free_space_remove_batch) { const auto current_size = main_priority->getSize(lock); chassert(current_size >= stat.total_stat.releasable_size); @@ -1063,13 +1064,24 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() watch.stop(); ProfileEvents::increment(ProfileEvents::FilesystemCacheFreeSpaceKeepingThreadWorkMilliseconds, watch.elapsedMilliseconds()); - LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms", watch.elapsedMilliseconds()); + LOG_TRACE(log, "Free space ratio keeping thread finished in {} ms (status: {})", + watch.elapsedMilliseconds(), desired_size_status); [[maybe_unused]] bool scheduled = false; - if (limits_satisfied) - scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); - else - scheduled = keep_up_free_space_ratio_task->schedule(); + switch (desired_size_status) + { + case IFileCachePriority::DesiredSizeStatus::SUCCESS: [[fallthrough]]; + case IFileCachePriority::DesiredSizeStatus::CANNOT_EVICT: + { + scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); + break; + } + case IFileCachePriority::DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT: + { + scheduled = keep_up_free_space_ratio_task->schedule(); + break; + } + } chassert(scheduled); } @@ -1546,7 +1558,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheReserveStat stat; if (main_priority->collectCandidatesForEviction( new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, - stat, eviction_candidates, cache_lock)) + stat, eviction_candidates, cache_lock) == IFileCachePriority::DesiredSizeStatus::SUCCESS) { if (eviction_candidates.size() == 0) { diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 5d8eb9dd54a..9885ab00f78 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -151,7 +151,13 @@ public: /// and `desired_elements_num` as current cache state. /// Collect no more than `max_candidates_to_evict` elements. /// Return `true` if the first condition is satisfied. - virtual bool collectCandidatesForEviction( + enum class DesiredSizeStatus + { + SUCCESS, + CANNOT_EVICT, + REACHED_MAX_CANDIDATES_LIMIT, + }; + virtual DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index ec96eb14a8a..7970eaa3e13 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -323,7 +323,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -bool LRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -336,12 +336,24 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock, &desired_size, &desired_elements_count); }; + auto status = DesiredSizeStatus::CANNOT_EVICT; auto stop_condition = [&]() { - return desired_limits_satisfied() || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); + if (desired_limits_satisfied()) + { + status = DesiredSizeStatus::SUCCESS; + return true; + } + if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) + { + status = DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT; + return true; + } + return false; }; iterateForEviction(res, stat, stop_condition, lock); - return desired_limits_satisfied(); + chassert(status != DesiredSizeStatus::SUCCESS || stop_condition()); + return status; } void LRUFileCachePriority::iterateForEviction( @@ -350,6 +362,9 @@ void LRUFileCachePriority::iterateForEviction( StopConditionFunc stop_condition, const CachePriorityGuard::Lock & lock) { + if (stop_condition()) + return; + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); IterateFunc iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index e0691cade43..9bced106727 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -63,7 +63,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - bool collectCandidatesForEviction( + DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 7a3fdf5160e..dc0df223cb0 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -256,7 +256,7 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -bool SLRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -268,7 +268,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); FileCacheReserveStat probationary_stat; - const bool probationary_limit_satisfied = probationary_queue.collectCandidatesForEviction( + const auto probationary_desired_size_status = probationary_queue.collectCandidatesForEviction( desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, probationary_stat, res, lock); @@ -285,14 +285,14 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) - return probationary_limit_satisfied; + if (probationary_desired_size_status == DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT) + return probationary_desired_size_status; const auto desired_protected_size = getRatio(desired_size, size_ratio); const auto desired_protected_elements_num = getRatio(desired_elements_count, size_ratio); FileCacheReserveStat protected_stat; - const bool protected_limit_satisfied = protected_queue.collectCandidatesForEviction( + const auto protected_desired_size_status = protected_queue.collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, max_candidates_to_evict - res.size(), protected_stat, res, lock); @@ -306,7 +306,10 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, protected_queue.getStateInfoForLog(lock)); - return probationary_limit_satisfied && protected_limit_satisfied; + if (probationary_desired_size_status == DesiredSizeStatus::SUCCESS) + return protected_desired_size_status; + else + return probationary_desired_size_status; } void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 2102a0ec558..e6d20e0d0ee 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,7 +58,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - bool collectCandidatesForEviction( + DesiredSizeStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, From 19cd00000373a5707178214744444b4d8c4034a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 31 Jul 2024 17:18:55 +0200 Subject: [PATCH 0549/1170] Update src/Interpreters/DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 48b01a9df43..56d9c323d39 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,7 +1406,7 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); - LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids).contains(uuid) ? "table still exists" : "table dropped successfully"); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids.contains(uuid) ? "table still exists" : "table dropped successfully"); /// TSA doesn't support unique_lock if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) From f032c015ca9ec10b7938bbf3d67bb6181776d24a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 15:40:37 +0000 Subject: [PATCH 0550/1170] Ignore some tests --- tests/queries/0_stateless/00705_drop_create_merge_tree.sh | 4 ++-- .../0_stateless/01019_alter_materialized_view_atomic.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh index fd002668696..ea8b9d02e49 100755 --- a/tests/queries/0_stateless/00705_drop_create_merge_tree.sh +++ b/tests/queries/0_stateless/00705_drop_create_merge_tree.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT & -yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT & +yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & +yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery & wait ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table" diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index eb12a76eb62..4bd21fcee02 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT < Date: Wed, 31 Jul 2024 15:41:37 +0000 Subject: [PATCH 0551/1170] Fix build --- src/Interpreters/DatabaseCatalog.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 56d9c323d39..273e5720679 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1406,10 +1406,11 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) return !tables_marked_dropped_ids.contains(uuid) || is_shutting_down; }); - LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), tables_marked_dropped_ids.contains(uuid) ? "table still exists" : "table dropped successfully"); - /// TSA doesn't support unique_lock - if (TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid)) + const bool has_table = TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid); + LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), has_table ? "table still exists" : "table dropped successfully"); + + if has_table) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " "will finish after restart", uuid); } From 8afe61e04581d0b95ac2d6e927bb9d2427247c7a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 Jul 2024 17:58:41 +0200 Subject: [PATCH 0552/1170] Better --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 273e5720679..fb4fad85f66 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1410,7 +1410,7 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) const bool has_table = TSA_SUPPRESS_WARNING_FOR_READ(tables_marked_dropped_ids).contains(uuid); LOG_DEBUG(log, "Done waiting for the table {} to be dropped. The outcome: {}", toString(uuid), has_table ? "table still exists" : "table dropped successfully"); - if has_table) + if (has_table) throw Exception(ErrorCodes::UNFINISHED, "Did not finish dropping the table with UUID {} because the server is shutting down, " "will finish after restart", uuid); } 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 0553/1170] 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 0554/1170] 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 743d63767a74f41b3628c52ccf166be773baecf2 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 31 Jul 2024 17:06:49 +0000 Subject: [PATCH 0555/1170] fix AsyncLoader destruction race --- src/Common/AsyncLoader.cpp | 33 ++++++++++++++++++++------------- 1 file changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6264eb03106..d40e320e741 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -218,20 +218,27 @@ AsyncLoader::~AsyncLoader() { // All `LoadTask` objects should be destructed before AsyncLoader destruction because they hold a reference. // To make sure we check for all pending jobs to be finished. - std::unique_lock lock{mutex}; - if (scheduled_jobs.empty() && finished_jobs.empty()) - return; + { + std::unique_lock lock{mutex}; + if (!scheduled_jobs.empty() || !finished_jobs.empty()) + { + std::vector scheduled; + std::vector finished; + scheduled.reserve(scheduled_jobs.size()); + finished.reserve(finished_jobs.size()); + for (const auto & [job, _] : scheduled_jobs) + scheduled.push_back(job->name); + for (const auto & job : finished_jobs) + finished.push_back(job->name); + LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); + abort(); + } + } - std::vector scheduled; - std::vector finished; - scheduled.reserve(scheduled_jobs.size()); - finished.reserve(finished_jobs.size()); - for (const auto & [job, _] : scheduled_jobs) - scheduled.push_back(job->name); - for (const auto & job : finished_jobs) - finished.push_back(job->name); - LOG_ERROR(log, "Bug. Destruction with pending ({}) and finished ({}) load jobs.", fmt::join(scheduled, ", "), fmt::join(finished, ", ")); - abort(); + // When all jobs are done we could still have finalizing workers. + // These workers could call updateCurrentPriorityAndSpawn() that scans all pools. + // We need to stop all of them before destructing any of them. + stop(); } void AsyncLoader::start() From 2b79da36c0701bb9ca392fddd9129a7e0e04ef3f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 19:26:45 +0200 Subject: [PATCH 0556/1170] Update 01605_adaptive_granularity_block_borders.sql --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 5f09dc423b2..f9b8bb1c1c6 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache +-- Tags: long, no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher From bfb0133f26f85e64f2d608595b653c20a594abca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0557/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { 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 0558/1170] 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 2c6c5c5c94a2f4131d898b172d6af285eec07c2f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 19:31:22 +0200 Subject: [PATCH 0559/1170] Add an assertion --- src/Common/TimerDescriptor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 9a171ae9487..716dcee2747 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -110,6 +110,9 @@ void TimerDescriptor::drain() const throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot readlink for a timer_fd {}", timer_fd); LOG_TRACE(log, "Received EINTR while trying to drain a TimerDescriptor, fd {}: {}", timer_fd, std::string_view(link_path, link_path_length)); + + /// Check that it's actually a timerfd. + chassert(std::string_view(link_path, link_path_length).contains("timerfd")); continue; } From 26a81e366d722314bbda12d5232fa49117ef4498 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 31 Jul 2024 18:05:53 +0000 Subject: [PATCH 0560/1170] Fix the serialization of parameters for parameterized views Removed the call to convertFieldToString() and added datatype specific serialization code. Parameterized view substitution was broken for multiple datatypes when parameter value was a function or expression returning datatype instance. Testcase added to cover Date/Date32/UUID/IP datatypes. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 10 +- ...zed_view_with_non_literal_params.reference | 31 ++++++ ...meterized_view_with_non_literal_params.sql | 97 +++++++++++++++++++ 3 files changed, 137 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference create mode 100644 tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..6113a38d463 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4546,7 +4546,15 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, resolveExpressionNode(nodes[1], scope, /* allow_lambda_expression */false, /* allow_table_function */false); if (auto * constant = nodes[1]->as()) { - view_params[identifier_node->getIdentifier().getFullName()] = convertFieldToString(constant->getValue()); + /// Serialize the constant value using datatype specific + /// interfaces to match the deserialization in ReplaceQueryParametersVistor. + WriteBufferFromOwnString buf; + auto constval = constant->getValue(); + auto realtype = constant->getResultType(); + auto tempcol = realtype->createColumn(); + tempcol->insert(constval); + realtype->getDefaultSerialization()->serializeTextEscaped(*tempcol, 0, buf, {}); + view_params[identifier_node->getIdentifier().getFullName()] = buf.str(); } } } diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference new file mode 100644 index 00000000000..e4e6c313b85 --- /dev/null +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.reference @@ -0,0 +1,31 @@ +Test with Date parameter +1 +2 +1 +3 +3 +3 +2 +Test with Date32 parameter +1 +2 +1 +3 +5 +3 +4 +Test with UUID parameter +4 +3 +3 +1 +2 +Test with 2 parameters +1 +1 +3 +3 +Test with IPv4 +1 +2 +3 diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql new file mode 100644 index 00000000000..55795c7a785 --- /dev/null +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -0,0 +1,97 @@ + +select 'Test with Date parameter'; + +drop table if exists date_table_pv; +create table date_table_pv (id Int32, dt Date) engine = Memory(); + +insert into date_table_pv values(1, today()); +insert into date_table_pv values(2, yesterday()); +insert into date_table_pv values(3, toDate('1974-04-07')); + +drop view if exists date_pv; +create view date_pv as select * from date_table_pv where dt = {dtparam:Date}; + +select id from date_pv(dtparam=today()); +select id from date_pv(dtparam=yesterday()); +select id from date_pv(dtparam=yesterday()+1); +select id from date_pv(dtparam='1974-04-07'); +select id from date_pv(dtparam=toDate('1974-04-07')); +select id from date_pv(dtparam=toString(toDate('1974-04-07'))); +select id from date_pv(dtparam=toDate('1975-04-07')); +select id from date_pv(dtparam=(select dt from date_table_pv where id = 2)); + +select 'Test with Date32 parameter'; + +drop table if exists date32_table_pv; +create table date32_table_pv (id Int32, dt Date32) engine = Memory(); + +insert into date32_table_pv values(1, today()); +insert into date32_table_pv values(2, yesterday()); +insert into date32_table_pv values(3, toDate32('2199-12-31')); +insert into date32_table_pv values(4, toDate32('1950-12-25')); +insert into date32_table_pv values(5, toDate32('1900-01-01')); + +drop view if exists date32_pv; +create view date32_pv as select * from date32_table_pv where dt = {dtparam:Date32}; + +select id from date32_pv(dtparam=today()); +select id from date32_pv(dtparam=yesterday()); +select id from date32_pv(dtparam=yesterday()+1); +select id from date32_pv(dtparam='2199-12-31'); +select id from date32_pv(dtparam=toDate32('1900-01-01')); +select id from date32_pv(dtparam=(select dt from date32_table_pv where id = 3)); +select id from date32_pv(dtparam=(select dt from date32_table_pv where id = 4)); + + +select 'Test with UUID parameter'; +drop table if exists uuid_table_pv; +create table uuid_table_pv (id Int32, uu UUID) engine = Memory(); + +insert into uuid_table_pv values(1, generateUUIDv4()); +insert into uuid_table_pv values(2, generateUUIDv7()); +insert into uuid_table_pv values(3, toUUID('11111111-2222-3333-4444-555555555555')); +insert into uuid_table_pv select 4, serverUUID(); + + +drop view if exists uuid_pv; +create view uuid_pv as select * from uuid_table_pv where uu = {uuidparam:UUID}; +select id from uuid_pv(uuidparam=serverUUID()); +select id from uuid_pv(uuidparam=toUUID('11111111-2222-3333-4444-555555555555')); +select id from uuid_pv(uuidparam='11111111-2222-3333-4444-555555555555'); +select id from uuid_pv(uuidparam=(select uu from uuid_table_pv where id = 1)); +select id from uuid_pv(uuidparam=(select uu from uuid_table_pv where id = 2)); +-- generateUUIDv4() is not constant foldable, hence cannot be used as parameter value +select id from uuid_pv(uuidparam=generateUUIDv4()); -- { serverError UNKNOWN_QUERY_PARAMETER } +-- But nested "select generateUUIDv4()" works! +select id from uuid_pv(uuidparam=(select generateUUIDv4())); + +select 'Test with 2 parameters'; + +drop view if exists date_pv2; +create view date_pv2 as select * from date_table_pv where dt = {dtparam:Date} and id = {intparam:Int32}; +select id from date_pv2(dtparam=today(),intparam=1); +select id from date_pv2(dtparam=today(),intparam=length('A')); +select id from date_pv2(dtparam='1974-04-07',intparam=length('AAA')); +select id from date_pv2(dtparam=toDate('1974-04-07'),intparam=length('BBB')); + +select 'Test with IPv4'; + +drop table if exists ipv4_table_pv; +create table ipv4_table_pv (id Int32, ipaddr IPv4) ENGINE = Memory(); +insert into ipv4_table_pv values (1, '116.106.34.242'); +insert into ipv4_table_pv values (2, '116.106.34.243'); +insert into ipv4_table_pv values (3, '116.106.34.244'); + +drop view if exists ipv4_pv; +create view ipv4_pv as select * from ipv4_table_pv where ipaddr = {ipv4param:IPv4}; +select id from ipv4_pv(ipv4param='116.106.34.242'); +select id from ipv4_pv(ipv4param=toIPv4('116.106.34.243')); +select id from ipv4_pv(ipv4param=(select ipaddr from ipv4_table_pv where id=3)); + +drop view date_pv; +drop view date_pv2; +drop view uuid_pv; +drop view ipv4_pv; +drop table date_table_pv; +drop table uuid_table_pv; +drop table ipv4_table_pv; 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 0561/1170] 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 0562/1170] 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 dde274f6fad979aa94ea31395b0434c81f72328a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 18:08:14 +0000 Subject: [PATCH 0563/1170] Re-enable ICU on s390/x --- contrib/icu-cmake/CMakeLists.txt | 4 +--- contrib/icudata | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index f9d05f7fe97..adeaa7dcf33 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -4,9 +4,7 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they generated -# the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255 -if (NOT ENABLE_ICU OR ARCH_S390X) +if (NOT ENABLE_ICU) message(STATUS "Not using ICU") return() endif() diff --git a/contrib/icudata b/contrib/icudata index d345d6ac22f..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit d345d6ac22f381c882420de9053d30ae1ff38d75 +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b From e2af1766eb1ea0ee2f6b862f53a0d3c13f53365b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:25:28 +0200 Subject: [PATCH 0564/1170] init --- src/Functions/DateTimeTransforms.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a7bd398cdaa..fe26c5cf353 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -381,11 +381,13 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + return res >= 0 ? res : 0; } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + return res >= 0 ? res : 0; } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { From d0c643180f408d84a5f10a917f413248b9267202 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:29:36 +0200 Subject: [PATCH 0565/1170] add tests --- .../03215_toStartOfWeek_with_dateTime64_fix.reference | 2 ++ .../0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference create mode 100644 tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference new file mode 100644 index 00000000000..fd698107f22 --- /dev/null +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.reference @@ -0,0 +1,2 @@ +1970-01-01 +1970-01-01 diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql new file mode 100644 index 00000000000..0f00a52cb86 --- /dev/null +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql @@ -0,0 +1,2 @@ +SELECT toStartOfWeek(toDateTime64('1970-02-01', 6)); +SELECT toStartOfWeek(toDateTime('1970-01-01')); From 636c3f642340de6e5ca4892481ca156cb236a4cd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:31:22 +0200 Subject: [PATCH 0566/1170] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index fe26c5cf353..1970ec3bdb0 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -382,12 +382,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - return res >= 0 ? res : 0; + return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - return res >= 0 ? res : 0; + return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) { From 34ca3128ed0de60e03a105c43dc0924541f4a2c1 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 20:36:13 +0200 Subject: [PATCH 0567/1170] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 1970ec3bdb0..46fb3bb9f57 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -381,12 +381,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto & res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) 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 0568/1170] 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 9def8cea8121ae8001649629a96e73eb1e10159b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 18:57:08 +0000 Subject: [PATCH 0569/1170] Update version_date.tsv and changelogs after v24.4.4.107-stable --- docs/changelogs/v24.4.4.107-stable.md | 70 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 71 insertions(+) create mode 100644 docs/changelogs/v24.4.4.107-stable.md diff --git a/docs/changelogs/v24.4.4.107-stable.md b/docs/changelogs/v24.4.4.107-stable.md new file mode 100644 index 00000000000..ba7c576715e --- /dev/null +++ b/docs/changelogs/v24.4.4.107-stable.md @@ -0,0 +1,70 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.4.107-stable (af0ed6b197e) FIXME as compared to v24.4.3.25-stable (a915dd4eda4) + +#### Improvement +* Backported in [#65884](https://github.com/ClickHouse/ClickHouse/issues/65884): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65303](https://github.com/ClickHouse/ClickHouse/issues/65303): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65894](https://github.com/ClickHouse/ClickHouse/issues/65894): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65372](https://github.com/ClickHouse/ClickHouse/issues/65372): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#66883](https://github.com/ClickHouse/ClickHouse/issues/66883): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65435](https://github.com/ClickHouse/ClickHouse/issues/65435): Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65448](https://github.com/ClickHouse/ClickHouse/issues/65448): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65710](https://github.com/ClickHouse/ClickHouse/issues/65710): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66689](https://github.com/ClickHouse/ClickHouse/issues/66689): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65353](https://github.com/ClickHouse/ClickHouse/issues/65353): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65060](https://github.com/ClickHouse/ClickHouse/issues/65060): Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.`. [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65329](https://github.com/ClickHouse/ClickHouse/issues/65329): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#64833](https://github.com/ClickHouse/ClickHouse/issues/64833): Fix bug which could lead to non-working TTLs with expressions. [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Backported in [#65086](https://github.com/ClickHouse/ClickHouse/issues/65086): Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65540](https://github.com/ClickHouse/ClickHouse/issues/65540): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65578](https://github.com/ClickHouse/ClickHouse/issues/65578): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65161](https://github.com/ClickHouse/ClickHouse/issues/65161): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65616](https://github.com/ClickHouse/ClickHouse/issues/65616): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65730](https://github.com/ClickHouse/ClickHouse/issues/65730): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65668](https://github.com/ClickHouse/ClickHouse/issues/65668): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65786](https://github.com/ClickHouse/ClickHouse/issues/65786): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65810](https://github.com/ClickHouse/ClickHouse/issues/65810): Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65931](https://github.com/ClickHouse/ClickHouse/issues/65931): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65826](https://github.com/ClickHouse/ClickHouse/issues/65826): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66299](https://github.com/ClickHouse/ClickHouse/issues/66299): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66326](https://github.com/ClickHouse/ClickHouse/issues/66326): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66153](https://github.com/ClickHouse/ClickHouse/issues/66153): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66459](https://github.com/ClickHouse/ClickHouse/issues/66459): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66224](https://github.com/ClickHouse/ClickHouse/issues/66224): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66267](https://github.com/ClickHouse/ClickHouse/issues/66267): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66678](https://github.com/ClickHouse/ClickHouse/issues/66678): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66603](https://github.com/ClickHouse/ClickHouse/issues/66603): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66358](https://github.com/ClickHouse/ClickHouse/issues/66358): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66971](https://github.com/ClickHouse/ClickHouse/issues/66971): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66968](https://github.com/ClickHouse/ClickHouse/issues/66968): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66719](https://github.com/ClickHouse/ClickHouse/issues/66719): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66950](https://github.com/ClickHouse/ClickHouse/issues/66950): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66947](https://github.com/ClickHouse/ClickHouse/issues/66947): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67195](https://github.com/ClickHouse/ClickHouse/issues/67195): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67377](https://github.com/ClickHouse/ClickHouse/issues/67377): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67240](https://github.com/ClickHouse/ClickHouse/issues/67240): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65410](https://github.com/ClickHouse/ClickHouse/issues/65410): Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65903](https://github.com/ClickHouse/ClickHouse/issues/65903): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66385](https://github.com/ClickHouse/ClickHouse/issues/66385): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66424](https://github.com/ClickHouse/ClickHouse/issues/66424): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66542](https://github.com/ClickHouse/ClickHouse/issues/66542): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66857](https://github.com/ClickHouse/ClickHouse/issues/66857): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66873](https://github.com/ClickHouse/ClickHouse/issues/66873): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67057](https://github.com/ClickHouse/ClickHouse/issues/67057): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66944](https://github.com/ClickHouse/ClickHouse/issues/66944): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67250](https://github.com/ClickHouse/ClickHouse/issues/67250): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67410](https://github.com/ClickHouse/ClickHouse/issues/67410): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 027b207d3ad..abd8f84ec74 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -5,6 +5,7 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 +v24.4.4.107-stable 2024-07-31 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From cd3c6c3ae807a321d94079087eb1fb29f4764549 Mon Sep 17 00:00:00 2001 From: sakulali Date: Thu, 1 Aug 2024 03:37:45 +0800 Subject: [PATCH 0570/1170] try to fix flaky test 01889_clickhouse_client_config_format --- .../01889_clickhouse_client_config_format.reference | 2 +- .../0_stateless/01889_clickhouse_client_config_format.sh | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index ef0d9ffc538..2575200e6fa 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -19,6 +19,6 @@ autodetect xml (non leading whitespaces) autodetect yaml 2 autodetect invalid xml -Code: 1000, e.code() = 0, SAXParseException: Invalid token in '/config_test.badxml', line 2 column 12, Stack trace (when copying this message, always include the lines below): +Correct: invalid xml parsed with exception autodetect invalid yaml Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 9a44ec0d5f5..58fd6852116 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -108,7 +108,9 @@ echo 'autodetect xml (non leading whitespaces)' $CLICKHOUSE_CLIENT --config "$autodetect_xml_non_leading_whitespace_config" -q "select getSetting('max_threads')" echo 'autodetect yaml' $CLICKHOUSE_CLIENT --config "$autodetect_yaml_config" -q "select getSetting('max_threads')" + +# Error code is 1000 (Poco::Exception). It is not ignored. echo 'autodetect invalid xml' -$CLICKHOUSE_CLIENT --config "$autodetect_invalid_xml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#Poco::Exception. ##" +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_xml_config" -q "select getSetting('max_threads')" 2>&1 |& grep -q "Code: 1000" && echo "Correct: invalid xml parsed with exception" || echo 'Fail: expected error code 1000 but got other' echo 'autodetect invalid yaml' -$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -n '1p' | sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file +$CLICKHOUSE_CLIENT --config "$autodetect_invalid_yaml_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file From 89ca6aee4ea23ed9cb9b36bb4fa9a6490efe576a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 31 Jul 2024 20:00:37 +0000 Subject: [PATCH 0571/1170] Check for timeout when we stop logs replication --- docker/test/base/setup_export_logs.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 0c869a95db2..db141bcc55e 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -215,7 +215,8 @@ function setup_logs_replication function stop_logs_replication { echo "Detach all logs replication" - clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | { + timeout --preserve-status --signal TERM --kill-after 10m 20m \ + clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | { tee /dev/stderr } | { xargs -n1 -r -i clickhouse-client --query "drop table {}" From cc27c254abd4b6fd8f64b47e0bdf6195041bd5ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0572/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 867784d55c989943d0c79eb9179b01e878fabcbe Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 22:48:16 +0200 Subject: [PATCH 0573/1170] Update DateTimeTransforms.h --- src/Functions/DateTimeTransforms.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 46fb3bb9f57..ce7da406e9a 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include 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 0574/1170] 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 9cb52bd1381ad3e0929062801df7c4b542cf1117 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 23:11:35 +0200 Subject: [PATCH 0575/1170] fix build --- src/Functions/DateTimeTransforms.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index ce7da406e9a..15f1b9580f3 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -25,7 +24,7 @@ namespace DB static constexpr auto millisecond_multiplier = 1'000; static constexpr auto microsecond_multiplier = 1'000'000; -static constexpr auto nanosecond_multiplier = 1'000'000'000; +static constexpr auto nanosecond_multiplier = 1'000'000'000; static constexpr FormatSettings::DateTimeOverflowBehavior default_date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore; @@ -382,12 +381,12 @@ struct ToStartOfWeekImpl static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const int res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) { - const auto res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + const int res = time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); return std::max(res, 0); } static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) From 6403f3f545bee153ffaf4ce5bda6fcde33ef88d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 23:29:54 +0200 Subject: [PATCH 0576/1170] Miscellaneous --- src/Common/Epoll.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 49c86222cf0..ef7c6e143a0 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -19,7 +19,7 @@ Epoll::Epoll() : events_count(0) { epoll_fd = epoll_create1(0); if (epoll_fd == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot open epoll descriptor"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot open epoll descriptor"); } Epoll::Epoll(Epoll && other) noexcept : epoll_fd(other.epoll_fd), events_count(other.events_count.load()) @@ -47,7 +47,7 @@ void Epoll::add(int fd, void * ptr, uint32_t events) ++events_count; if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot add new descriptor to epoll"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot add new descriptor to epoll"); } void Epoll::remove(int fd) @@ -55,7 +55,7 @@ void Epoll::remove(int fd) --events_count; if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1) - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot remove descriptor from epoll"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Cannot remove descriptor from epoll"); } size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout) const @@ -82,7 +82,7 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout continue; } else - throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Error in epoll_wait"); + throw ErrnoException(ErrorCodes::EPOLL_ERROR, "Error in epoll_wait"); } else break; From 6e914ff6da67be1c1381ffed2d04b5758704baf3 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 31 Jul 2024 21:59:37 +0000 Subject: [PATCH 0577/1170] Update settings.md Removing duplicate header "## background_merges_mutations_scheduling_policy" --- docs/en/operations/server-configuration-parameters/settings.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 8278f8c8699..a1e3c292b04 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -103,8 +103,6 @@ Default: 2 The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. -## background_merges_mutations_scheduling_policy - Algorithm used to select next merge or mutation to be executed by background thread pool. Policy may be changed at runtime without server restart. Could be applied from the `default` profile for backward compatibility. From 0e36db543762cb146aa6c233d4536fd62c6101b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 22:09:04 +0000 Subject: [PATCH 0578/1170] Do not check the value of `num_messages_read` because it is not reliable librdkafka usually polls both messages on the first SELECT, but sometimes only one. If it polls only one message first, then it will read two messages before rebalancing at the second SELECT from the table. This means it usually reads a single message twice (thus num_messages_read = 4 is usually fine as 1 discarded message + 3 actually consumed message). But when only one message is read in the first SELECT, then 2 messages are discarded, thus num_messages_read will be 5 as 2 discarded message + 3 actually consumed messages. --- tests/integration/test_storage_kafka/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8393e88db88..37457e00701 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4771,7 +4771,7 @@ def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): assignments.current_offset, if(length(exceptions.time)>0, exceptions.time[1]::String, 'never') as last_exception_time_, if(length(exceptions.text)>0, exceptions.text[1], 'no exception') as last_exception_, - stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, + stable_timestamp(last_poll_time) as last_poll_time_, stable_timestamp(last_commit_time) as last_commit_time_, num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, num_rebalance_revocations, num_rebalance_assignments, is_currently_used FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; @@ -4791,7 +4791,6 @@ assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 4 last_commit_time_: now num_commits: 2 last_rebalance_time_: now @@ -4810,7 +4809,6 @@ assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 1 last_commit_time_: now num_commits: 1 last_rebalance_time_: never From a592ad3624dedd3080b95422151b17bb92f3c10a Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 31 Jul 2024 23:06:53 +0000 Subject: [PATCH 0579/1170] change error code --- src/Storages/MergeTree/MergeTreeData.cpp | 5 +++-- src/Storages/StorageFactory.cpp | 7 ++++--- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/03174_projection_deduplicate.sql | 2 +- .../03206_projection_merge_special_mergetree.sql | 14 +++++++------- 6 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 677c4a92cda..7a0980a0e3b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3216,9 +3216,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { if (auto storage_name = getName(); storage_name != "MergeTree" && storage_name != "ReplicatedMergeTree" && settings_from_storage->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" - " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode."); + " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." + " Current storage name is {}.", storage_name); } commands.apply(new_metadata, local_context); diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index a059d624cd8..7360d351e8a 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; extern const int BAD_ARGUMENTS; extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES; - extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } @@ -223,9 +223,10 @@ StoragePtr StorageFactory::get( } } if (!projection_allowed) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" - " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode."); + " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." + " Current storage name is {}.", engine_name); } } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a5d434796ba..84393a3f1b0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1582,7 +1582,7 @@ bool StorageMergeTree::optimize( if (deduplicate && getInMemoryMetadataPtr()->hasProjections() && getSettings()->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " "User should drop all the projections manually before running the query, " "or consider drop or rebuild option of deduplicate_merge_projection_mode", diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3751883df24..ad578242010 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5796,7 +5796,7 @@ bool StorageReplicatedMergeTree::optimize( if (deduplicate && getInMemoryMetadataPtr()->hasProjections() && getSettings()->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " "User should drop all the projections manually before running the query, " "or consider drop or rebuild option of deduplicate_merge_projection_mode", diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql index 46222b69dc7..f43f0a1f236 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.sql +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -17,7 +17,7 @@ PRIMARY KEY id; INSERT INTO test_projection_deduplicate VALUES (1, 'one'); INSERT INTO test_projection_deduplicate VALUES (1, 'one'); -OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError SUPPORT_IS_DISABLED } SELECT * FROM test_projection_deduplicate; diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index e0a4f4f8cec..d3448138396 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -9,7 +9,7 @@ CREATE TABLE tp ( INSERT INTO tp SELECT number%3, 1 FROM numbers(3); -OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE tp; @@ -26,7 +26,7 @@ OPTIMIZE TABLE tp DEDUPLICATE; ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; -OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE tp; @@ -36,14 +36,14 @@ CREATE TABLE tp ( type Int32, eventcnt UInt64, PROJECTION p (select sum(eventcnt), type group by type) -) engine = ReplacingMergeTree order by type; -- { serverError NOT_IMPLEMENTED } +) engine = ReplacingMergeTree order by type; -- { serverError SUPPORT_IS_DISABLED } CREATE TABLE tp ( type Int32, eventcnt UInt64, PROJECTION p (select sum(eventcnt), type group by type) ) engine = ReplacingMergeTree order by type -SETTINGS deduplicate_merge_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +SETTINGS deduplicate_merge_projection_mode = 'throw'; -- { serverError SUPPORT_IS_DISABLED } CREATE TABLE tp ( type Int32, @@ -65,7 +65,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'tp') AND (active = 1); ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; -OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE tp; @@ -78,7 +78,7 @@ SETTINGS deduplicate_merge_projection_mode = 'rebuild'; ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'throw'; -OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } +OPTIMIZE TABLE tp DEDUPLICATE; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE tp; @@ -88,7 +88,7 @@ CREATE TABLE tp ( eventcnt UInt64 ) engine = ReplacingMergeTree order by type; -ALTER TABLE tp ADD PROJECTION p (SELECT sum(eventcnt), type GROUP BY type); -- { serverError NOT_IMPLEMENTED } +ALTER TABLE tp ADD PROJECTION p (SELECT sum(eventcnt), type GROUP BY type); -- { serverError SUPPORT_IS_DISABLED } ALTER TABLE tp MODIFY SETTING deduplicate_merge_projection_mode = 'drop'; From 587d50380ad3d1cadc56bda0fa700e3441c16ab4 Mon Sep 17 00:00:00 2001 From: sakulali Date: Thu, 1 Aug 2024 09:12:00 +0800 Subject: [PATCH 0580/1170] fix clickhouse-test reference --- .../0_stateless/01889_clickhouse_client_config_format.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference index 2575200e6fa..149315ad9d5 100644 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.reference @@ -21,4 +21,4 @@ autodetect yaml autodetect invalid xml Correct: invalid xml parsed with exception autodetect invalid yaml -Code: 585. Unable to parse YAML configuration file /config_test.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) +Code: 585. Unable to parse YAML configuration file /config_default.badyaml, yaml-cpp: error at line 2, column 12: illegal map value. (CANNOT_PARSE_YAML) From f162d6bd5e03c6f717b4f45cf4c7ba6491aaa5fa Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Aug 2024 06:35:22 +0000 Subject: [PATCH 0581/1170] Update version_date.tsv and changelogs after v24.7.2.13-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.7.2.13-stable.md | 24 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 +- 5 files changed, 28 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.7.2.13-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index e99c86267f9..94603763572 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.1.2915" +ARG VERSION="24.7.2.13" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index fb562b911a3..f40118c7b06 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.1.2915" +ARG VERSION="24.7.2.13" 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 51f4e6a0f40..032aa862e4a 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.1.2915" +ARG VERSION="24.7.2.13" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.7.2.13-stable.md b/docs/changelogs/v24.7.2.13-stable.md new file mode 100644 index 00000000000..4a2fb665116 --- /dev/null +++ b/docs/changelogs/v24.7.2.13-stable.md @@ -0,0 +1,24 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.2.13-stable (6e41f601b2f) FIXME as compared to v24.7.1.2915-stable (a37d2d43da7) + +#### Improvement +* Backported in [#67531](https://github.com/ClickHouse/ClickHouse/issues/67531): In pr : https://github.com/ClickHouse/ClickHouse/pull/66025, we introduce a settings `input_format_orc_read_use_writer_time_zone` to fix when read orc file, make the reader use writer timezone, not always use `GMT`. [#67175](https://github.com/ClickHouse/ClickHouse/pull/67175) ([kevinyhzou](https://github.com/KevinyhZou)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#67505](https://github.com/ClickHouse/ClickHouse/issues/67505): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67580](https://github.com/ClickHouse/ClickHouse/issues/67580): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67551](https://github.com/ClickHouse/ClickHouse/issues/67551): [Green CI] Fix test test_storage_s3_queue/test.py::test_max_set_age. [#67035](https://github.com/ClickHouse/ClickHouse/pull/67035) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67514](https://github.com/ClickHouse/ClickHouse/issues/67514): Split test 02967_parallel_replicas_join_algo_and_analyzer. [#67211](https://github.com/ClickHouse/ClickHouse/pull/67211) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67545](https://github.com/ClickHouse/ClickHouse/issues/67545): [Green CI] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown. [#67474](https://github.com/ClickHouse/ClickHouse/pull/67474) ([Alexey Katsman](https://github.com/alexkats)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index abd8f84ec74..b1391c2d781 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 @@ -5,7 +6,6 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 -v24.4.4.107-stable 2024-07-31 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From c2df527a32d640f52296ea7aefae177e22504082 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 08:42:54 +0200 Subject: [PATCH 0582/1170] Reduce fault rate --- .../test_keeper_map_retries/configs/fault_injection.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml index 145945c7c7c..0933b6b3031 100644 --- a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -1,6 +1,6 @@ - 0.05 - 0.05 + 0.005 + 0.005 From 5564489cca1c14e95e7c543e03c508849abaf079 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 15:31:54 +0800 Subject: [PATCH 0583/1170] change as request --- .../functions/tuple-map-functions.md | 18 +++++++++++--- src/Functions/map.cpp | 24 ++++++++++--------- .../0_stateless/01651_map_functions.reference | 4 ++++ .../0_stateless/01651_map_functions.sql | 7 +++++- 4 files changed, 38 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index db66188b1f5..d670ed42a2a 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -43,7 +43,7 @@ Result: ## mapFromArrays -Creates a map from an array of keys and an array of values. +Creates a map from an array or map of keys and an array or map of values. The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`. For example, instead of writing @@ -62,8 +62,8 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). -- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). +- `keys` — Array or map of keys to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). If `keys` is an array, we accept `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as its type as long as it doesn't contain NULL value. +- `values` - Array or map of values to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** @@ -99,6 +99,18 @@ Result: └───────────────────────────────────────────────────────┘ ``` +```sql +SELECT mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3]) +``` + +Result: + +``` +┌─mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])─┐ +│ {('a',1):1,('b',2):2,('c',3):3} │ +└───────────────────────────────────────────────────────┘ +``` + ## extractKeyValuePairs Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md). diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index a8e5f7ad90e..738c61164a3 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } namespace @@ -157,7 +158,7 @@ private: bool use_variant_as_common_type = false; }; -/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays +/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays or maps class FunctionMapFromArrays : public IFunction { public: @@ -181,13 +182,13 @@ public: getName(), arguments.size()); - auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + auto get_nested_type = [&](const DataTypePtr & type) { DataTypePtr nested; - if (const auto * array_type = checkAndGetDataType(type.get())) - nested = array_type->getNestedType(); - else if (const auto * map_type = checkAndGetDataType(type.get())) - nested = std::make_shared(map_type->getKeyValueTypes()); + if (const auto * type_as_array = checkAndGetDataType(type.get())) + nested = type_as_array->getNestedType(); + else if (const auto * type_as_map = checkAndGetDataType(type.get())) + nested = std::make_shared(type_as_map->getKeyValueTypes()); else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -201,8 +202,9 @@ public: auto key_type = get_nested_type(arguments[0]); auto value_type = get_nested_type(arguments[1]); - /// Remove Nullable from key_type if needed for map key must not be Nullable + /// We accept Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as key types as long as the actual array doesn't contain NULL value(this is checked in executeImpl). key_type = removeNullableOrLowCardinalityNullable(key_type); + DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -210,7 +212,7 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - auto get_array_column = [this](const ColumnPtr & column) -> std::pair + auto get_array_column = [&](const ColumnPtr & column) -> std::pair { bool is_const = isColumnConst(*column); ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; @@ -231,8 +233,9 @@ public: }; auto [col_keys, key_holder] = get_array_column(arguments[0].column); + auto [col_values, values_holder] = get_array_column(arguments[1].column); - /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + /// Nullable(T) or LowCardinality(Nullable(T)) are okay as nested key types but actual NULL values are not okay. ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { @@ -253,10 +256,9 @@ public: if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); + ErrorCodes::BAD_ARGUMENTS, "The nested column of first argument in function {} must not contain NULLs", getName()); } - auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 9114aa419b1..e336a02318d 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -55,3 +55,7 @@ {1:3,2:4} {1:3,2:4} {1:3,2:4} {(1,3):'a',(2,4):'b'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 4604ddd6db1..dc93a38b265 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -68,7 +68,7 @@ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAY select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError BAD_ARGUMENTS } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); @@ -79,3 +79,8 @@ select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, select mapFromArrays([1,2], [3,4]); select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); + +select mapFromArrays(map(1, 'a', 2, 'b'), array('c', 'd')); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), array('c', 'd')); +select mapFromArrays(map(1, 'a', 2, 'b'), materialize(array('c', 'd'))); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), materialize(array('c', 'd'))); From d83c0c1b3b189a78833afec5f87e7004b0f934e3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 1 Aug 2024 07:36:53 +0000 Subject: [PATCH 0584/1170] 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 681441e170202bc3963fb3fa1d7b7785192dbd2e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 16:01:39 +0800 Subject: [PATCH 0585/1170] fix style --- src/Functions/map.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 738c61164a3..6e389f39dec 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -192,7 +192,7 @@ public: else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument types of function {} must be Array or Map, but {} is given", + "Arguments of function {} must be Array or Map, but {} is given", getName(), type->getName()); @@ -275,10 +275,7 @@ public: static constexpr auto name = "mapUpdate"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -287,9 +284,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 2", - getName(), arguments.size()); + getName(), + arguments.size()); const auto * left = checkAndGetDataType(arguments[0].type.get()); const auto * right = checkAndGetDataType(arguments[1].type.get()); @@ -405,7 +404,6 @@ public: return ColumnMap::create(nested_column); } }; - } REGISTER_FUNCTION(Map) From 7db4065898633ace1f909711d4caeda8d135cace Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 10:30:50 +0200 Subject: [PATCH 0586/1170] Add retries to create --- src/Storages/StorageKeeperMap.cpp | 373 +++++++++++------- src/Storages/StorageKeeperMap.h | 6 +- .../configs/keeper_retries.xml | 14 + tests/integration/test_keeper_map/test.py | 3 +- .../configs/fault_injection.xml | 1 + .../configs/keeper_retries.xml | 14 + .../test_keeper_map_retries/test.py | 13 +- 7 files changed, 275 insertions(+), 149 deletions(-) create mode 100644 tests/integration/test_keeper_map/configs/keeper_retries.xml create mode 100644 tests/integration/test_keeper_map_retries/configs/keeper_retries.xml diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 0634c7be6ee..a6be9f8da04 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -408,104 +408,192 @@ StorageKeeperMap::StorageKeeperMap( if (attach) { - checkTable(); + checkTable(context_); return; } - auto client = getClient(); + const auto & settings = context_->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{settings.keeper_max_retries, settings.keeper_retry_initial_backoff_ms, settings.keeper_retry_max_backoff_ms}, + context_->getProcessListElement()}; - if (zk_root_path != "/" && !client->exists(zk_root_path)) - { - LOG_TRACE(log, "Creating root path {}", zk_root_path); - client->createAncestors(zk_root_path); - client->createIfNotExists(zk_root_path, ""); - } + zk_retry.retryLoop( + [&] + { + auto client = getClient(); + if (zk_root_path != "/" && !client->exists(zk_root_path)) + { + LOG_TRACE(log, "Creating root path {}", zk_root_path); + client->createAncestors(zk_root_path); + client->createIfNotExists(zk_root_path, ""); + } + }); + + std::shared_ptr metadata_drop_lock; + int32_t drop_lock_version = -1; for (size_t i = 0; i < 1000; ++i) { - std::string stored_metadata_string; - auto exists = client->tryGet(zk_metadata_path, stored_metadata_string); - - if (exists) - { - // this requires same name for columns - // maybe we can do a smarter comparison for columns and primary key expression - if (stored_metadata_string != metadata_string) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", - zk_root_path, - stored_metadata_string); - - auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); - - /// A table on the same Keeper path already exists, we just appended our table id to subscribe as a new replica - /// We still don't know if the table matches the expected metadata so table_is_valid is not changed - /// It will be checked lazily on the first operation - if (code == Coordination::Error::ZOK) - return; - - if (code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException(code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); - - /// ZNONODE means we dropped zk_tables_path but didn't finish drop completely - } - - if (client->exists(zk_dropped_path)) - { - LOG_INFO(log, "Removing leftover nodes"); - auto code = client->tryCreate(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral); - - if (code == Coordination::Error::ZNONODE) + bool success = false; + zk_retry.retryLoop( + [&] { - LOG_INFO(log, "Someone else removed leftover nodes"); - } - else if (code == Coordination::Error::ZNODEEXISTS) - { - LOG_INFO(log, "Someone else is removing leftover nodes"); - continue; - } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception::fromPath(code, zk_dropped_lock_path); - } - else - { - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); - if (!dropTable(client, metadata_drop_lock)) - continue; - } - } + auto client = getClient(); + std::string stored_metadata_string; + auto exists = client->tryGet(zk_metadata_path, stored_metadata_string); - Coordination::Requests create_requests - { - zkutil::makeCreateRequest(zk_metadata_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_data_path, metadata_string, zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_tables_path, "", zkutil::CreateMode::Persistent), - zkutil::makeCreateRequest(zk_table_path, "", zkutil::CreateMode::Persistent), - }; + if (exists) + { + // this requires same name for columns + // maybe we can do a smarter comparison for columns and primary key expression + if (stored_metadata_string != metadata_string) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", + zk_root_path, + stored_metadata_string); - Coordination::Responses create_responses; - auto code = client->tryMulti(create_requests, create_responses); - if (code == Coordination::Error::ZNODEEXISTS) - { - LOG_INFO(log, "It looks like a table on path {} was created by another server at the same moment, will retry", zk_root_path); - continue; - } - else if (code != Coordination::Error::ZOK) - { - zkutil::KeeperMultiException::check(code, create_requests, create_responses); - } + auto code = client->tryCreate(zk_table_path, "", zkutil::CreateMode::Persistent); + /// A table on the same Keeper path already exists, we just appended our table id to subscribe as a new replica + /// We still don't know if the table matches the expected metadata so table_is_valid is not changed + /// It will be checked lazily on the first operation + if (code == Coordination::Error::ZOK) + { + success = true; + return; + } - table_status = TableStatus::VALID; - /// we are the first table created for the specified Keeper path, i.e. we are the first replica - return; + /// We most likely created the path but got a timeout or disconnect + if (code == Coordination::Error::ZNODEEXISTS && zk_retry.isRetry()) + { + success = true; + return; + } + + if (code != Coordination::Error::ZNONODE) + throw zkutil::KeeperException( + code, "Failed to create table on path {} because a table with same UUID already exists", zk_root_path); + + /// ZNONODE means we dropped zk_tables_path but didn't finish drop completely + } + + if (client->exists(zk_dropped_path)) + { + LOG_INFO(log, "Removing leftover nodes"); + + bool drop_finished = false; + if (zk_retry.isRetry() && metadata_drop_lock != nullptr && drop_lock_version != -1) + { + /// if we have leftover lock from previous try, we need to recreate the ephemeral with our session + Coordination::Requests drop_lock_requests{ + zkutil::makeRemoveRequest(zk_dropped_lock_path, drop_lock_version), + zkutil::makeCreateRequest(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral), + }; + + Coordination::Responses drop_lock_responses; + auto lock_code = client->tryMulti(drop_lock_requests, drop_lock_responses); + if (lock_code == Coordination::Error::ZBADVERSION) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock.reset(); + return; + } + + if (drop_lock_responses[0]->error == Coordination::Error::ZNONODE) + { + /// someone else removed metadata nodes or the previous ephemeral node expired + /// we will try creating dropped lock again to make sure + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock.reset(); + } + else if (lock_code == Coordination::Error::ZOK) + { + metadata_drop_lock->setAlreadyRemoved(); + metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); + drop_lock_version = -1; + Coordination::Stat lock_stat; + client->get(zk_dropped_lock_path, &lock_stat); + drop_lock_version = lock_stat.version; + if (!dropTable(client, metadata_drop_lock)) + { + metadata_drop_lock.reset(); + return; + } + drop_finished = true; + } + } + + if (!drop_finished) + { + auto code = client->tryCreate(zk_dropped_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE) + { + LOG_INFO(log, "Someone else removed leftover nodes"); + } + else if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + return; + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, zk_dropped_lock_path); + } + else + { + metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(zk_dropped_lock_path, *client); + drop_lock_version = -1; + Coordination::Stat lock_stat; + client->get(zk_dropped_lock_path, &lock_stat); + drop_lock_version = lock_stat.version; + if (!dropTable(client, metadata_drop_lock)) + { + metadata_drop_lock.reset(); + return; + } + } + } + } + + Coordination::Requests create_requests{ + zkutil::makeCreateRequest(zk_metadata_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_data_path, metadata_string, zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_tables_path, "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest(zk_table_path, "", zkutil::CreateMode::Persistent), + }; + + Coordination::Responses create_responses; + auto code = client->tryMulti(create_requests, create_responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO( + log, "It looks like a table on path {} was created by another server at the same moment, will retry", zk_root_path); + return; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, create_requests, create_responses); + } + + table_status = TableStatus::VALID; + /// we are the first table created for the specified Keeper path, i.e. we are the first replica + success = true; + }); + + if (success) + return; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot create metadata for table, because it is removed concurrently or because " - "of wrong zk_root_path ({})", zk_root_path); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot create metadata for table, because it is removed concurrently or because " + "of wrong zk_root_path ({})", + zk_root_path); } @@ -518,7 +606,7 @@ Pipe StorageKeeperMap::read( size_t max_block_size, size_t num_streams) { - checkTable(); + checkTable(context_); storage_snapshot->check(column_names); FieldVectorPtr filtered_keys; @@ -592,13 +680,13 @@ Pipe StorageKeeperMap::read( SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { - checkTable(); + checkTable(local_context); return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); } void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { - checkTable(); + checkTable(local_context); const auto & settings = local_context->getSettingsRef(); ZooKeeperRetriesControl zk_retry{ getName(), @@ -657,7 +745,7 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E void StorageKeeperMap::drop() { - auto current_table_status = getTableStatus(); + auto current_table_status = getTableStatus(getContext()); if (current_table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " @@ -666,7 +754,7 @@ void StorageKeeperMap::drop() } /// if only column metadata is wrong we can still drop the table correctly - if (current_table_status == TableStatus::INVALID_KEEPER_STRUCTURE) + if (current_table_status == TableStatus::INVALID_METADATA) return; auto client = getClient(); @@ -1029,7 +1117,7 @@ UInt64 StorageKeeperMap::keysLimit() const return keys_limit; } -StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const +StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus(const ContextPtr & local_context) const { std::lock_guard lock{init_mutex}; if (table_status != TableStatus::UNKNOWN) @@ -1039,57 +1127,70 @@ StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const { try { - auto client = getClient(); + const auto & settings = local_context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings.keeper_max_retries, + settings.keeper_retry_initial_backoff_ms, + settings.keeper_retry_max_backoff_ms}, + local_context->getProcessListElement()}; - Coordination::Stat metadata_stat; - auto stored_metadata_string = client->get(zk_metadata_path, &metadata_stat); - - if (metadata_stat.numChildren == 0) + zk_retry.retryLoop([&] { + auto client = getClient(); + + Coordination::Stat metadata_stat; + auto stored_metadata_string = client->get(zk_metadata_path, &metadata_stat); + + if (metadata_stat.numChildren == 0) + { + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; + return; + } + + if (metadata_string != stored_metadata_string) + { + LOG_ERROR( + log, + "Table definition does not match to the one stored in the path {}. Stored definition: {}", + zk_root_path, + stored_metadata_string); + table_status = TableStatus::INVALID_METADATA; + return; + } + + // validate all metadata and data nodes are present + Coordination::Requests requests; + requests.push_back(zkutil::makeCheckRequest(zk_table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_data_path, -1)); + requests.push_back(zkutil::makeCheckRequest(zk_dropped_path, -1)); + + Coordination::Responses responses; + client->tryMulti(requests, responses); + table_status = TableStatus::INVALID_KEEPER_STRUCTURE; - return; - } + if (responses[0]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); + return; + } - if (metadata_string != stored_metadata_string) - { - LOG_ERROR( - log, - "Table definition does not match to the one stored in the path {}. Stored definition: {}", - zk_root_path, - stored_metadata_string); - table_status = TableStatus::INVALID_METADATA; - return; - } + if (responses[1]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Data node ({}) is missing", zk_data_path); + return; + } - // validate all metadata and data nodes are present - Coordination::Requests requests; - requests.push_back(zkutil::makeCheckRequest(zk_table_path, -1)); - requests.push_back(zkutil::makeCheckRequest(zk_data_path, -1)); - requests.push_back(zkutil::makeCheckRequest(zk_dropped_path, -1)); + if (responses[2]->error == Coordination::Error::ZOK) + { + LOG_ERROR(log, "Tables with root node {} are being dropped", zk_root_path); + return; + } - Coordination::Responses responses; - client->tryMulti(requests, responses); - - table_status = TableStatus::INVALID_KEEPER_STRUCTURE; - if (responses[0]->error != Coordination::Error::ZOK) - { - LOG_ERROR(log, "Table node ({}) is missing", zk_table_path); - return; - } - - if (responses[1]->error != Coordination::Error::ZOK) - { - LOG_ERROR(log, "Data node ({}) is missing", zk_data_path); - return; - } - - if (responses[2]->error == Coordination::Error::ZOK) - { - LOG_ERROR(log, "Tables with root node {} are being dropped", zk_root_path); - return; - } - - table_status = TableStatus::VALID; + table_status = TableStatus::VALID; + }); } catch (const Coordination::Exception & e) { @@ -1227,7 +1328,7 @@ void StorageKeeperMap::checkMutationIsPossible(const MutationCommands & commands void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr local_context) { - checkTable(); + checkTable(local_context); if (commands.empty()) return; diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 8ed348a4f6f..1464eeaabad 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -78,9 +78,9 @@ public: UInt64 keysLimit() const; template - void checkTable() const + void checkTable(const ContextPtr & local_context) const { - auto current_table_status = getTableStatus(); + auto current_table_status = getTableStatus(local_context); if (table_status == TableStatus::UNKNOWN) { static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated " @@ -119,7 +119,7 @@ private: VALID }; - TableStatus getTableStatus() const; + TableStatus getTableStatus(const ContextPtr & context) const; void restoreDataImpl( const BackupPtr & backup, diff --git a/tests/integration/test_keeper_map/configs/keeper_retries.xml b/tests/integration/test_keeper_map/configs/keeper_retries.xml new file mode 100644 index 00000000000..43e5b9a09e8 --- /dev/null +++ b/tests/integration/test_keeper_map/configs/keeper_retries.xml @@ -0,0 +1,14 @@ + + + + 0 + 0 + + + + + + default + + + diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 4b1bcd11cfe..861a7c47687 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -10,6 +10,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/enable_keeper_map.xml"], + user_configs=["configs/keeper_retries.xml"], with_zookeeper=True, stay_alive=True, ) @@ -49,7 +50,6 @@ def assert_keeper_exception_after_partition(query): error = node.query_and_get_error_with_retry( query, sleep_time=1, - settings={"insert_keeper_max_retries": 1, "keeper_max_retries": 1}, ) assert "Coordination::Exception" in error except: @@ -91,7 +91,6 @@ def test_keeper_map_without_zk(started_cluster): error = node.query_and_get_error_with_retry( "SELECT * FROM test_keeper_map_without_zk", sleep_time=1, - settings={"keeper_max_retries": 1}, ) assert "Failed to activate table because of connection issues" in error except: diff --git a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml index 0933b6b3031..8406b7db785 100644 --- a/tests/integration/test_keeper_map_retries/configs/fault_injection.xml +++ b/tests/integration/test_keeper_map_retries/configs/fault_injection.xml @@ -1,5 +1,6 @@ + 1 0.005 0.005 diff --git a/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml b/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml new file mode 100644 index 00000000000..208dd6e47fa --- /dev/null +++ b/tests/integration/test_keeper_map_retries/configs/keeper_retries.xml @@ -0,0 +1,14 @@ + + + + 20 + 10000 + + + + + + default + + + diff --git a/tests/integration/test_keeper_map_retries/test.py b/tests/integration/test_keeper_map_retries/test.py index 352119147cd..c6760e5d1a2 100644 --- a/tests/integration/test_keeper_map_retries/test.py +++ b/tests/integration/test_keeper_map_retries/test.py @@ -11,6 +11,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/enable_keeper_map.xml"], + user_configs=["configs/keeper_retries.xml"], with_zookeeper=True, stay_alive=True, ) @@ -42,10 +43,6 @@ def repeat_query(query, repeat): for _ in range(repeat): node.query( query, - settings={ - "keeper_max_retries": 20, - "keeper_retry_max_backoff_ms": 10000, - }, ) @@ -53,10 +50,6 @@ def test_queries(started_cluster): start_clean_clickhouse() node.query("DROP TABLE IF EXISTS keeper_map_retries SYNC") - node.query( - "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a" - ) - node.stop_clickhouse() node.copy_file_to_container( os.path.join(CONFIG_DIR, "fault_injection.xml"), @@ -66,6 +59,10 @@ def test_queries(started_cluster): repeat_count = 10 + node.query( + "CREATE TABLE keeper_map_retries (a UInt64, b UInt64) Engine=KeeperMap('/keeper_map_retries') PRIMARY KEY a", + ) + repeat_query( "INSERT INTO keeper_map_retries SELECT number, number FROM numbers(500)", repeat_count, From a78b1ddd976f98b1952e7ee350b1fcd1935ede19 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 1 Aug 2024 08:44:49 +0000 Subject: [PATCH 0587/1170] 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 0588/1170] 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 bababe2dd2fe490e877f9c4ac03d979bd739376a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 Jul 2024 15:07:07 +0000 Subject: [PATCH 0589/1170] Fix remainders I --- src/Functions/FunctionsMultiStringSearch.h | 8 +++--- src/Functions/MultiMatchAnyImpl.h | 27 ++++++++++--------- src/Functions/MultiSearchFirstIndexImpl.h | 16 +++++------ src/Functions/MultiSearchFirstPositionImpl.h | 18 ++++++------- src/Functions/MultiSearchImpl.h | 18 ++++++------- .../0_stateless/00927_disable_hyperscan.sql | 2 +- 6 files changed, 45 insertions(+), 44 deletions(-) diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 03db2651fd0..6bcc8581a38 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -81,7 +81,7 @@ public: return Impl::getReturnType(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr & haystack_ptr = arguments[0].column; const ColumnPtr & needles_ptr = arguments[1].column; @@ -110,13 +110,15 @@ public: col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_const->getValue(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 20b2150048b..06d9eded9c9 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -66,9 +66,10 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorConstant( @@ -81,7 +82,8 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -101,7 +103,7 @@ struct MultiMatchAnyImpl throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); } - res.resize(haystack_offsets.size()); + res.resize(input_rows_count); if (needles_arr.empty()) { @@ -133,9 +135,8 @@ struct MultiMatchAnyImpl /// Once we hit the callback, there is no need to search for others. return 1; }; - const size_t haystack_offsets_size = haystack_offsets.size(); UInt64 offset = 0; - for (size_t i = 0; i < haystack_offsets_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; /// vectorscan restriction. @@ -186,9 +187,10 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorVector( @@ -202,12 +204,13 @@ struct MultiMatchAnyImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); - res.resize(haystack_offsets.size()); + res.resize(input_rows_count); #if USE_VECTORSCAN size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -216,7 +219,7 @@ struct MultiMatchAnyImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); @@ -306,7 +309,7 @@ struct MultiMatchAnyImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 36a5fd514d9..5b34dbfe601 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -33,7 +33,8 @@ struct MultiSearchFirstIndexImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -48,14 +49,13 @@ struct MultiSearchFirstIndexImpl auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -80,10 +80,10 @@ struct MultiSearchFirstIndexImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -92,7 +92,7 @@ struct MultiSearchFirstIndexImpl std::vector needles; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index ccdd82a0ee5..06bf7aa94d8 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -33,7 +33,8 @@ struct MultiSearchFirstPositionImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -52,14 +53,13 @@ struct MultiSearchFirstPositionImpl }; auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -89,10 +89,10 @@ struct MultiSearchFirstPositionImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -106,14 +106,12 @@ struct MultiSearchFirstPositionImpl return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); }; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - { needles.emplace_back(needles_data_string.getDataAt(j).toView()); - } auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 467cc96a95f..909425f5a93 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -33,7 +33,8 @@ struct MultiSearchImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -48,14 +49,13 @@ struct MultiSearchImpl auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t iteration = 0; while (searcher.hasMoreToSearch()) { size_t prev_haystack_offset = 0; - for (size_t j = 0; j < haystack_size; ++j) + for (size_t j = 0; j < input_rows_count; ++j) { const auto * haystack = &haystack_data[prev_haystack_offset]; const auto * haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset - 1; @@ -79,10 +79,10 @@ struct MultiSearchImpl bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, size_t /*max_hyperscan_regexp_total_length*/, - bool /*reject_expensive_hyperscan_regexps*/) + bool /*reject_expensive_hyperscan_regexps*/, + size_t input_rows_count) { - const size_t haystack_size = haystack_offsets.size(); - res.resize(haystack_size); + res.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -91,14 +91,12 @@ struct MultiSearchImpl std::vector needles; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - { needles.emplace_back(needles_data_string.getDataAt(j).toView()); - } const auto * const haystack = &haystack_data[prev_haystack_offset]; const size_t haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; diff --git a/tests/queries/0_stateless/00927_disable_hyperscan.sql b/tests/queries/0_stateless/00927_disable_hyperscan.sql index c07848a4fcc..24ec7a35adb 100644 --- a/tests/queries/0_stateless/00927_disable_hyperscan.sql +++ b/tests/queries/0_stateless/00927_disable_hyperscan.sql @@ -1,4 +1,4 @@ --- Tags: no-debug +-- Tags: no-fasttest SET allow_hyperscan = 1; From 0f850952fa304bcc63f7008d4bae54c47ea0564d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 09:52:25 +0000 Subject: [PATCH 0590/1170] Fix remainders II --- .../FunctionsMultiStringFuzzySearch.h | 8 +++--- src/Functions/MultiMatchAllIndicesImpl.h | 27 +++++++++++-------- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index a92a6570279..8346380c35d 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -71,7 +71,7 @@ public: return Impl::getReturnType(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr & haystack_ptr = arguments[0].column; const ColumnPtr & edit_distance_ptr = arguments[1].column; @@ -114,14 +114,16 @@ public: col_needles_const->getValue(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, + input_rows_count); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 3e9c8fba215..3aeac808880 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -52,9 +52,10 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorConstant( @@ -67,7 +68,8 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -87,7 +89,7 @@ struct MultiMatchAllIndicesImpl throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); } - offsets.resize(haystack_offsets.size()); + offsets.resize(input_rows_count); if (needles_arr.empty()) { @@ -114,9 +116,8 @@ struct MultiMatchAllIndicesImpl static_cast*>(context)->push_back(id); return 0; }; - const size_t haystack_offsets_size = haystack_offsets.size(); UInt64 offset = 0; - for (size_t i = 0; i < haystack_offsets_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { UInt64 length = haystack_offsets[i] - offset - 1; /// vectorscan restriction. @@ -146,6 +147,7 @@ struct MultiMatchAllIndicesImpl (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; (void)reject_expensive_hyperscan_regexps; + (void)input_rows_count; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } @@ -160,9 +162,10 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps, input_rows_count); } static void vectorVector( @@ -176,12 +179,13 @@ struct MultiMatchAllIndicesImpl bool allow_hyperscan, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length, - bool reject_expensive_hyperscan_regexps) + bool reject_expensive_hyperscan_regexps, + size_t input_rows_count) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); #if USE_VECTORSCAN - offsets.resize(haystack_offsets.size()); + offsets.resize(input_rows_count); size_t prev_haystack_offset = 0; size_t prev_needles_offset = 0; @@ -189,7 +193,7 @@ struct MultiMatchAllIndicesImpl std::vector needles; - for (size_t i = 0; i < haystack_offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { needles.reserve(needles_offsets[i] - prev_needles_offset); @@ -271,6 +275,7 @@ struct MultiMatchAllIndicesImpl (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; (void)reject_expensive_hyperscan_regexps; + (void)input_rows_count; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } From 99760ad7e4b1b7d4ee86dc2f5aa4df98ead9e260 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 Jul 2024 17:17:40 +0000 Subject: [PATCH 0591/1170] Fix remainders, pt. III --- src/Functions/FunctionsStringSearchToString.h | 4 ++-- src/Functions/FunctionsVisitParam.h | 5 +++-- src/Functions/URL/extractURLParameter.cpp | 7 ++++--- src/Functions/extract.cpp | 7 ++++--- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsStringSearchToString.h b/src/Functions/FunctionsStringSearchToString.h index 978a84de472..c889cf062a3 100644 --- a/src/Functions/FunctionsStringSearchToString.h +++ b/src/Functions/FunctionsStringSearchToString.h @@ -60,7 +60,7 @@ public: return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const ColumnPtr column = arguments[0].column; const ColumnPtr column_needle = arguments[1].column; @@ -75,7 +75,7 @@ public: ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); - Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); + Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res, input_rows_count); return col_res; } diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index 5e13fbbad5c..bcaaf0a1c20 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -168,11 +168,12 @@ struct ExtractParamToStringImpl { static void vector(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, std::string needle, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { /// Constant 5 is taken from a function that performs a similar task FunctionsStringSearch.h::ExtractImpl res_data.reserve(haystack_data.size() / 5); - res_offsets.resize(haystack_offsets.size()); + res_offsets.resize(input_rows_count); /// We are looking for a parameter simply as a substring of the form "name" needle = "\"" + needle + "\":"; diff --git a/src/Functions/URL/extractURLParameter.cpp b/src/Functions/URL/extractURLParameter.cpp index f75875e0200..590c2779d9c 100644 --- a/src/Functions/URL/extractURLParameter.cpp +++ b/src/Functions/URL/extractURLParameter.cpp @@ -10,10 +10,11 @@ struct ExtractURLParameterImpl static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string pattern, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.reserve(data.size() / 5); - res_offsets.resize(offsets.size()); + res_offsets.resize(input_rows_count); pattern += '='; const char * param_str = pattern.c_str(); @@ -22,7 +23,7 @@ struct ExtractURLParameterImpl ColumnString::Offset prev_offset = 0; ColumnString::Offset res_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { ColumnString::Offset cur_offset = offsets[i]; diff --git a/src/Functions/extract.cpp b/src/Functions/extract.cpp index 6bbdaff0e3f..c78ee9898b7 100644 --- a/src/Functions/extract.cpp +++ b/src/Functions/extract.cpp @@ -16,10 +16,11 @@ struct ExtractImpl const ColumnString::Offsets & offsets, const std::string & pattern, ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) + ColumnString::Offsets & res_offsets, + size_t input_rows_count) { res_data.reserve(data.size() / 5); - res_offsets.resize(offsets.size()); + res_offsets.resize(input_rows_count); const OptimizedRegularExpression regexp = Regexps::createRegexp(pattern); @@ -29,7 +30,7 @@ struct ExtractImpl size_t prev_offset = 0; size_t res_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t cur_offset = offsets[i]; From 0452768983f379d201e1871ffce63452430bd0c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 08:10:01 +0000 Subject: [PATCH 0592/1170] Fix remainders, pt. IV --- src/Functions/CountSubstringsImpl.h | 17 ++++----- src/Functions/FunctionsStringSearch.h | 17 ++++++--- src/Functions/FunctionsVisitParam.h | 3 +- src/Functions/HasTokenImpl.h | 5 ++- src/Functions/MatchImpl.h | 54 +++++++++++++-------------- src/Functions/MultiMatchAnyImpl.h | 2 +- src/Functions/PositionImpl.h | 22 +++++------ 7 files changed, 62 insertions(+), 58 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 9ff3e4e1f2a..8e91bc3aeb4 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -37,7 +37,8 @@ struct CountSubstringsImpl const std::string & needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t /*input_rows_count*/) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -150,7 +151,8 @@ struct CountSubstringsImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -158,9 +160,7 @@ struct CountSubstringsImpl ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; @@ -207,7 +207,8 @@ struct CountSubstringsImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -215,9 +216,7 @@ struct CountSubstringsImpl /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { res[i] = 0; auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1); diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index fba6336ebff..7ec0076e395 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -163,7 +163,7 @@ public: return return_type; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const ColumnPtr & column_haystack = (argument_order == ArgumentOrder::HaystackNeedle) ? arguments[0].column : arguments[1].column; const ColumnPtr & column_needle = (argument_order == ArgumentOrder::HaystackNeedle) ? arguments[1].column : arguments[0].column; @@ -236,7 +236,8 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector && col_needle_const) Impl::vectorConstant( col_haystack_vector->getChars(), @@ -244,7 +245,8 @@ public: col_needle_const->getValue(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector_fixed && col_needle_vector) Impl::vectorFixedVector( col_haystack_vector_fixed->getChars(), @@ -253,14 +255,16 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_vector_fixed && col_needle_const) Impl::vectorFixedConstant( col_haystack_vector_fixed->getChars(), col_haystack_vector_fixed->getN(), col_needle_const->getValue(), vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else if (col_haystack_const && col_needle_vector) Impl::constantVector( col_haystack_const->getValue(), @@ -268,7 +272,8 @@ public: col_needle_vector->getOffsets(), column_start_pos, vec_res, - null_map.get()); + null_map.get(), + input_rows_count); else throw Exception( ErrorCodes::ILLEGAL_COLUMN, diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index bcaaf0a1c20..a77fa740f9c 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -93,7 +93,8 @@ struct ExtractParamImpl std::string needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t /*input_rows_count*/) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index a4ff49859cc..4943bf708c5 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -35,12 +35,13 @@ struct HasTokenImpl const std::string & pattern, const ColumnPtr & start_pos, PaddedPODArray & res, - ColumnUInt8 * res_null) + ColumnUInt8 * res_null, + size_t input_rows_count) { if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; const UInt8 * const begin = haystack_data.data(); diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 55b2fee5400..ceac753fe79 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -127,17 +127,17 @@ struct MatchImpl const String & needle, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack_offsets.size(); - - assert(haystack_size == res.size()); + assert(res.size() == haystack_offsets.size()); + assert(res.size() == input_rows_count); assert(start_pos_ == nullptr); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; /// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack: @@ -202,11 +202,11 @@ struct MatchImpl if (required_substring.empty()) { if (!regexp.getRE2()) /// An empty regexp. Always matches. - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + memset(res.data(), !negate, input_rows_count * sizeof(res[0])); else { size_t prev_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, @@ -291,16 +291,16 @@ struct MatchImpl size_t N, const String & needle, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack.size() / N; + assert(res.size() == haystack.size() / N); + assert(res.size() == input_rows_count); - assert(haystack_size == res.size()); - - if (haystack.empty()) + if (input_rows_count == 0) return; /// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack: @@ -370,11 +370,11 @@ struct MatchImpl if (required_substring.empty()) { if (!regexp.getRE2()) /// An empty regexp. Always matches. - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + memset(res.data(), !negate, input_rows_count * sizeof(res[0])); else { size_t offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack[offset]), N}, @@ -464,18 +464,18 @@ struct MatchImpl const ColumnString::Offsets & needle_offset, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack_offsets.size(); - - assert(haystack_size == needle_offset.size()); - assert(haystack_size == res.size()); + assert(haystack_offsets.size() == needle_offset.size()); + assert(res.size() == haystack_offsets.size()); + assert(res.size() == input_rows_count); assert(start_pos_ == nullptr); - if (haystack_offsets.empty()) + if (input_rows_count == 0) return; String required_substr; @@ -488,7 +488,7 @@ struct MatchImpl Regexps::LocalCacheTable cache; Regexps::RegexpPtr regexp; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; const size_t cur_haystack_length = haystack_offsets[i] - prev_haystack_offset - 1; @@ -573,15 +573,15 @@ struct MatchImpl const ColumnString::Offsets & needle_offset, [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); - const size_t haystack_size = haystack.size()/N; - - assert(haystack_size == needle_offset.size()); - assert(haystack_size == res.size()); + assert(res.size() == input_rows_count); + assert(res.size() == haystack.size() / N); + assert(res.size() == needle_offset.size()); assert(start_pos_ == nullptr); if (haystack.empty()) @@ -597,7 +597,7 @@ struct MatchImpl Regexps::LocalCacheTable cache; Regexps::RegexpPtr regexp; - for (size_t i = 0; i < haystack_size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * const cur_haystack_data = &haystack[prev_haystack_offset]; const size_t cur_haystack_length = N; diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 06d9eded9c9..fda752cbacc 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -165,7 +165,7 @@ struct MultiMatchAnyImpl memset(accum.data(), 0, accum.size()); for (size_t j = 0; j < needles.size(); ++j) { - MatchImpl::vectorConstant(haystack_data, haystack_offsets, String(needles[j].data(), needles[j].size()), nullptr, accum, nullptr); + MatchImpl::vectorConstant(haystack_data, haystack_offsets, String(needles[j].data(), needles[j].size()), nullptr, accum, nullptr, input_rows_count); for (size_t i = 0; i < res.size(); ++i) { if constexpr (FindAny) diff --git a/src/Functions/PositionImpl.h b/src/Functions/PositionImpl.h index eeb9d8b6a59..e525b5fab57 100644 --- a/src/Functions/PositionImpl.h +++ b/src/Functions/PositionImpl.h @@ -193,7 +193,8 @@ struct PositionImpl const std::string & needle, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -214,13 +215,12 @@ struct PositionImpl } ColumnString::Offset prev_offset = 0; - size_t rows = haystack_offsets.size(); if (const ColumnConst * start_pos_const = typeid_cast(&*start_pos)) { /// Needle is empty and start_pos is constant UInt64 start = std::max(start_pos_const->getUInt(0), static_cast(1)); - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t haystack_size = Impl::countChars( reinterpret_cast(pos), reinterpret_cast(pos + haystack_offsets[i] - prev_offset - 1)); @@ -234,7 +234,7 @@ struct PositionImpl else { /// Needle is empty and start_pos is not constant - for (size_t i = 0; i < rows; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t haystack_size = Impl::countChars( reinterpret_cast(pos), reinterpret_cast(pos + haystack_offsets[i] - prev_offset - 1)); @@ -359,7 +359,8 @@ struct PositionImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -367,9 +368,7 @@ struct PositionImpl ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; @@ -423,7 +422,8 @@ struct PositionImpl const ColumnString::Offsets & needle_offsets, const ColumnPtr & start_pos, PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * res_null) + [[maybe_unused]] ColumnUInt8 * res_null, + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. assert(!res_null); @@ -431,9 +431,7 @@ struct PositionImpl /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; From 59e51de6f8df35a0df0e08e8b3aacd3ba40c4cbd Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 31 Jul 2024 14:58:01 +0000 Subject: [PATCH 0593/1170] Fix join on tuple with NULLs --- .../Passes/ComparisonTupleEliminationPass.cpp | 10 +++++++++- ...on_tuple_comparison_elimination_bug.reference | 8 ++++++++ ..._join_on_tuple_comparison_elimination_bug.sql | 16 ++++++++++++++++ 3 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference create mode 100644 tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql diff --git a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp index 180470952cd..76dc8ab94b4 100644 --- a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include namespace DB @@ -25,8 +26,15 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - static bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) { + if (parent->getNodeType() == QueryTreeNodeType::JOIN) + { + /// In JOIN ON section comparison of tuples works a bit differently. + /// For example we can join on tuple(NULL) = tuple(NULL), join algorithms consider only NULLs on the top level. + if (parent->as().getJoinExpression().get() == child.get()) + return false; + } return child->getNodeType() != QueryTreeNodeType::TABLE_FUNCTION; } diff --git a/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference new file mode 100644 index 00000000000..c90a64e4a47 --- /dev/null +++ b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.reference @@ -0,0 +1,8 @@ +1 +\N +1 +\N +1 +\N +1 +\N diff --git a/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql new file mode 100644 index 00000000000..7ef98f88cc7 --- /dev/null +++ b/tests/queries/0_stateless/03214_join_on_tuple_comparison_elimination_bug.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; + +CREATE TABLE a (key Nullable(String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO a VALUES (NULL), ('1'); + +CREATE TABLE b (key Nullable(String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO b VALUES (NULL), ('1'); + +SELECT a.key FROM a LEFT SEMI JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; +SELECT a.key FROM a LEFT SEMI JOIN b ON a.key IS NOT DISTINCT FROM b.key ORDER BY a.key; +SELECT a.key FROM a LEFT SEMI JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; +SELECT a.key FROM a LEFT ANY JOIN b ON tuple(a.key) = tuple(b.key) ORDER BY a.key; + +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; From ca01c1c5691e4562ae6fc71af7b1867cf39f7ad1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 11:33:22 +0200 Subject: [PATCH 0594/1170] Fix bad merge --- src/Core/SettingsChangesHistory.cpp | 259 ---------------------------- 1 file changed, 259 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c3b9805700f..8f73e10c44f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -497,265 +497,6 @@ static std::initializer_list 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"}}}, }; From 4768e3878552ae0ce9007c1e4f400943a5712825 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 11:37:42 +0200 Subject: [PATCH 0595/1170] Update ref to 24.7.2 --- ...{02995_baseline_24_7_1.tsv => 02995_baseline_24_7_2.tsv} | 2 +- tests/queries/0_stateless/02995_new_settings_history.sh | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) rename tests/queries/0_stateless/{02995_baseline_24_7_1.tsv => 02995_baseline_24_7_2.tsv} (99%) diff --git a/tests/queries/0_stateless/02995_baseline_24_7_1.tsv b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv similarity index 99% rename from tests/queries/0_stateless/02995_baseline_24_7_1.tsv rename to tests/queries/0_stateless/02995_baseline_24_7_2.tsv index 6c830da8646..10b392f3e04 100644 --- a/tests/queries/0_stateless/02995_baseline_24_7_1.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv @@ -462,7 +462,7 @@ input_format_orc_allow_missing_columns 1 input_format_orc_case_insensitive_column_matching 0 input_format_orc_filter_push_down 1 input_format_orc_import_nested 0 -input_format_orc_read_use_writer_time_zone 0 +input_format_orc_reader_time_zone_name GMT input_format_orc_row_batch_size 100000 input_format_orc_skip_columns_with_unsupported_types_in_schema_inference 0 input_format_orc_use_fast_decoder 1 diff --git a/tests/queries/0_stateless/02995_new_settings_history.sh b/tests/queries/0_stateless/02995_new_settings_history.sh index 917dacc04b0..7fb21f88fae 100755 --- a/tests/queries/0_stateless/02995_new_settings_history.sh +++ b/tests/queries/0_stateless/02995_new_settings_history.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # Note that this is a broad check. A per version check is done in the upgrade test -# Baseline generated with 24.7.1 -# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_1.tsv +# Baseline generated with 24.7.2 +# clickhouse local --query "select name, default from system.settings order by name format TSV" > 02995_baseline_24_7_2.tsv $CLICKHOUSE_LOCAL --query " WITH old_settings AS ( - SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_1.tsv', 'TSV', 'name String, default String') + SELECT * FROM file('${CUR_DIR}/02995_baseline_24_7_2.tsv', 'TSV', 'name String, default String') ), new_settings AS ( From da3a37c561679daaecbcdece74f92ce98380b2b5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 1 Aug 2024 09:38:27 +0000 Subject: [PATCH 0596/1170] Improve regex to take into account the xdist name in the instance --- .../test.py | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 9cdf7db2b08..cc0a9022674 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -71,7 +71,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -99,7 +99,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -127,7 +127,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -161,7 +161,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -189,7 +189,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -217,7 +217,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -251,7 +251,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -279,7 +279,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -307,7 +307,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -341,7 +341,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -369,7 +369,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -397,7 +397,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", From c77f6d78d976430faf4353e350d0205bbecf2837 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Aug 2024 12:09:58 +0200 Subject: [PATCH 0597/1170] Update minio --- tests/integration/compose/docker_compose_minio.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_minio.yml b/tests/integration/compose/docker_compose_minio.yml index 4255a529f6d..40098d05b04 100644 --- a/tests/integration/compose/docker_compose_minio.yml +++ b/tests/integration/compose/docker_compose_minio.yml @@ -2,7 +2,7 @@ version: '2.3' services: minio1: - image: minio/minio:RELEASE.2023-09-30T07-02-29Z + image: minio/minio:RELEASE.2024-07-31T05-46-26Z volumes: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs From 928d5df3f02559fab4fd9bbd258f6e9c6ac4c18a Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 1 Aug 2024 12:32:43 +0200 Subject: [PATCH 0598/1170] added tests for https://github.com/ClickHouse/ClickHouse/pull/65475 Support writting page index into parquet file --- .../test/integration/runner/requirements.txt | 1 + .../test_parquet_page_index/__init__.py | 0 .../test_parquet_page_index/test.py | 77 +++++++++++++++++++ 3 files changed, 78 insertions(+) create mode 100644 tests/integration/test_parquet_page_index/__init__.py create mode 100644 tests/integration/test_parquet_page_index/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8a77d8abf77..428986b5562 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -74,6 +74,7 @@ protobuf==4.25.2 psycopg2-binary==2.9.6 py4j==0.10.9.5 py==1.11.0 +pyarrow==17.0.0 pycparser==2.22 pycryptodome==3.20.0 pymongo==3.11.0 diff --git a/tests/integration/test_parquet_page_index/__init__.py b/tests/integration/test_parquet_page_index/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_parquet_page_index/test.py b/tests/integration/test_parquet_page_index/test.py new file mode 100644 index 00000000000..366216ea2c6 --- /dev/null +++ b/tests/integration/test_parquet_page_index/test.py @@ -0,0 +1,77 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import pyarrow.parquet as pq +import os +import time + +cluster = ClickHouseCluster(__file__) +path_to_userfiles = "/var/lib/clickhouse/user_files/" +path_to_external_dirs = "/ClickHouse/tests/integration/test_parquet_page_index/_instances" +node = cluster.add_instance("node", external_dirs=[path_to_userfiles]) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def check_page_index(file_path): + metadata = pq.read_metadata(file_path) + assert metadata, "pyarrow.parquet library can't read parquet file written by Clickhouse" + return metadata.row_group(0).column(0).has_offset_index + + +def delete_if_exists(file_path): + if os.path.exists(file_path): + os.remove(file_path) + + +@pytest.mark.parametrize("query, expected_result", { + ("SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet " + "SETTINGS output_format_parquet_use_custom_encoder = false, " + "output_format_parquet_write_page_index = true;", True), + ("SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet " + "SETTINGS output_format_parquet_use_custom_encoder = false, " + "output_format_parquet_write_page_index = false;", False), + # # default settings: + # # output_format_parquet_use_custom_encoder = true + ("SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet;", False), +}) +def test_parquet_page_index_select_into_outfile(query, expected_result, start_cluster): + file_name = 'export.parquet' + query = query.format(file_name=file_name) + delete_if_exists(file_name) + assert node.query(query) == '' + assert check_page_index(file_name) == expected_result, "Page offset index have wrong value" + delete_if_exists(file_name) + + +@pytest.mark.parametrize("query, expected_result", { + ("INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SETTINGS output_format_parquet_use_custom_encoder=false, " + "output_format_parquet_write_page_index=true FORMAT Parquet", True), + ("INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SETTINGS output_format_parquet_use_custom_encoder=false, " + "output_format_parquet_write_page_index=false FORMAT Parquet", False), + # # default settings: + # # output_format_parquet_use_custom_encoder = true + ("INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 FORMAT Parquet", False), +}) +def test_parquet_page_index_insert_into_table_function_file(query, expected_result, start_cluster): + file_name = 'export.parquet' + query = query.format(file_name=file_name) + file_path = f"{path_to_external_dirs}{path_to_userfiles}{file_name}" + delete_if_exists(file_path) + assert node.query(query) == '' + assert check_page_index(file_path) == expected_result, "Page offset index have wrong value" + delete_if_exists(file_path) From 572ad2f6fe193d69e9c2c1c64125b3ffdff1de0d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Aug 2024 11:11:52 +0000 Subject: [PATCH 0599/1170] Automatic style fix --- .../test_parquet_page_index/test.py | 108 ++++++++++++------ 1 file changed, 71 insertions(+), 37 deletions(-) diff --git a/tests/integration/test_parquet_page_index/test.py b/tests/integration/test_parquet_page_index/test.py index 366216ea2c6..a820d1b5bc5 100644 --- a/tests/integration/test_parquet_page_index/test.py +++ b/tests/integration/test_parquet_page_index/test.py @@ -6,7 +6,9 @@ import time cluster = ClickHouseCluster(__file__) path_to_userfiles = "/var/lib/clickhouse/user_files/" -path_to_external_dirs = "/ClickHouse/tests/integration/test_parquet_page_index/_instances" +path_to_external_dirs = ( + "/ClickHouse/tests/integration/test_parquet_page_index/_instances" +) node = cluster.add_instance("node", external_dirs=[path_to_userfiles]) @@ -21,7 +23,9 @@ def start_cluster(): def check_page_index(file_path): metadata = pq.read_metadata(file_path) - assert metadata, "pyarrow.parquet library can't read parquet file written by Clickhouse" + assert ( + metadata + ), "pyarrow.parquet library can't read parquet file written by Clickhouse" return metadata.row_group(0).column(0).has_offset_index @@ -30,48 +34,78 @@ def delete_if_exists(file_path): os.remove(file_path) -@pytest.mark.parametrize("query, expected_result", { - ("SElECT number, number+1 FROM system.numbers LIMIT 100 " - "INTO OUTFILE '{file_name}' FORMAT Parquet " - "SETTINGS output_format_parquet_use_custom_encoder = false, " - "output_format_parquet_write_page_index = true;", True), - ("SElECT number, number+1 FROM system.numbers LIMIT 100 " - "INTO OUTFILE '{file_name}' FORMAT Parquet " - "SETTINGS output_format_parquet_use_custom_encoder = false, " - "output_format_parquet_write_page_index = false;", False), - # # default settings: - # # output_format_parquet_use_custom_encoder = true - ("SElECT number, number+1 FROM system.numbers LIMIT 100 " - "INTO OUTFILE '{file_name}' FORMAT Parquet;", False), -}) +@pytest.mark.parametrize( + "query, expected_result", + { + ( + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet " + "SETTINGS output_format_parquet_use_custom_encoder = false, " + "output_format_parquet_write_page_index = true;", + True, + ), + ( + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet " + "SETTINGS output_format_parquet_use_custom_encoder = false, " + "output_format_parquet_write_page_index = false;", + False, + ), + # # default settings: + # # output_format_parquet_use_custom_encoder = true + ( + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "INTO OUTFILE '{file_name}' FORMAT Parquet;", + False, + ), + }, +) def test_parquet_page_index_select_into_outfile(query, expected_result, start_cluster): - file_name = 'export.parquet' + file_name = "export.parquet" query = query.format(file_name=file_name) delete_if_exists(file_name) - assert node.query(query) == '' - assert check_page_index(file_name) == expected_result, "Page offset index have wrong value" + assert node.query(query) == "" + assert ( + check_page_index(file_name) == expected_result + ), "Page offset index have wrong value" delete_if_exists(file_name) -@pytest.mark.parametrize("query, expected_result", { - ("INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 " - "SETTINGS output_format_parquet_use_custom_encoder=false, " - "output_format_parquet_write_page_index=true FORMAT Parquet", True), - ("INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 " - "SETTINGS output_format_parquet_use_custom_encoder=false, " - "output_format_parquet_write_page_index=false FORMAT Parquet", False), - # # default settings: - # # output_format_parquet_use_custom_encoder = true - ("INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 FORMAT Parquet", False), -}) -def test_parquet_page_index_insert_into_table_function_file(query, expected_result, start_cluster): - file_name = 'export.parquet' +@pytest.mark.parametrize( + "query, expected_result", + { + ( + "INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SETTINGS output_format_parquet_use_custom_encoder=false, " + "output_format_parquet_write_page_index=true FORMAT Parquet", + True, + ), + ( + "INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SETTINGS output_format_parquet_use_custom_encoder=false, " + "output_format_parquet_write_page_index=false FORMAT Parquet", + False, + ), + # # default settings: + # # output_format_parquet_use_custom_encoder = true + ( + "INSERT INTO TABLE FUNCTION file('{file_name}') " + "SElECT number, number+1 FROM system.numbers LIMIT 100 FORMAT Parquet", + False, + ), + }, +) +def test_parquet_page_index_insert_into_table_function_file( + query, expected_result, start_cluster +): + file_name = "export.parquet" query = query.format(file_name=file_name) file_path = f"{path_to_external_dirs}{path_to_userfiles}{file_name}" delete_if_exists(file_path) - assert node.query(query) == '' - assert check_page_index(file_path) == expected_result, "Page offset index have wrong value" + assert node.query(query) == "" + assert ( + check_page_index(file_path) == expected_result + ), "Page offset index have wrong value" delete_if_exists(file_path) From 8fc77bec6639c8f0361858d9e031f3cb1175ae30 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:14:14 +0200 Subject: [PATCH 0600/1170] fix tests --- ...403_enable_extended_results_for_datetime_functions.reference | 2 +- .../0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference index c830d790000..147e368b5c9 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference @@ -64,7 +64,7 @@ toStartOfMonth;toDateTime64;false 2099-07-07 type;toStartOfMonth;toDateTime64;false Date toStartOfWeek;toDate32;false 2099-07-07 type;toStartOfWeek;toDate32;false Date -toStartOfWeek;toDateTime64;false 2099-07-07 +toStartOfWeek;toDateTime64;false 1970-01-01 type;toStartOfWeek;toDateTime64;false Date toMonday;toDate32;false 2099-07-08 type;toMonday;toDate32;false Date diff --git a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql index 0f00a52cb86..1769d96aa8d 100644 --- a/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql +++ b/tests/queries/0_stateless/03215_toStartOfWeek_with_dateTime64_fix.sql @@ -1,2 +1,2 @@ -SELECT toStartOfWeek(toDateTime64('1970-02-01', 6)); +SELECT toStartOfWeek(toDateTime64('1970-01-01', 6)); SELECT toStartOfWeek(toDateTime('1970-01-01')); From d0b514f12843fd09f18888666aa98170a52fb9d5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Aug 2024 13:27:09 +0200 Subject: [PATCH 0601/1170] Review fixes --- src/Interpreters/Cache/FileCache.cpp | 12 ++++++------ src/Interpreters/Cache/IFileCachePriority.h | 6 +++--- src/Interpreters/Cache/LRUFileCachePriority.cpp | 10 +++++----- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 6 +++--- src/Interpreters/Cache/SLRUFileCachePriority.h | 2 +- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index bf8dd24a1db..1f79c7d9032 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -998,7 +998,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() FileCacheReserveStat stat; EvictionCandidates eviction_candidates; - IFileCachePriority::DesiredSizeStatus desired_size_status; + IFileCachePriority::CollectStatus desired_size_status; try { /// Collect at most `keep_up_free_space_remove_batch` elements to evict, @@ -1009,7 +1009,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() #ifdef DEBUG_OR_SANITIZER_BUILD /// Let's make sure that we correctly processed the limits. - if (desired_size_status == IFileCachePriority::DesiredSizeStatus::SUCCESS + if (desired_size_status == IFileCachePriority::CollectStatus::SUCCESS && eviction_candidates.size() < keep_up_free_space_remove_batch) { const auto current_size = main_priority->getSize(lock); @@ -1070,13 +1070,13 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() [[maybe_unused]] bool scheduled = false; switch (desired_size_status) { - case IFileCachePriority::DesiredSizeStatus::SUCCESS: [[fallthrough]]; - case IFileCachePriority::DesiredSizeStatus::CANNOT_EVICT: + case IFileCachePriority::CollectStatus::SUCCESS: [[fallthrough]]; + case IFileCachePriority::CollectStatus::CANNOT_EVICT: { scheduled = keep_up_free_space_ratio_task->scheduleAfter(general_reschedule_ms); break; } - case IFileCachePriority::DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT: + case IFileCachePriority::CollectStatus::REACHED_MAX_CANDIDATES_LIMIT: { scheduled = keep_up_free_space_ratio_task->schedule(); break; @@ -1558,7 +1558,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheReserveStat stat; if (main_priority->collectCandidatesForEviction( new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, - stat, eviction_candidates, cache_lock) == IFileCachePriority::DesiredSizeStatus::SUCCESS) + stat, eviction_candidates, cache_lock) == IFileCachePriority::CollectStatus::SUCCESS) { if (eviction_candidates.size() == 0) { diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 9885ab00f78..6970d02473a 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -150,14 +150,14 @@ public: /// Collect eviction candidates sufficient to have `desired_size` /// and `desired_elements_num` as current cache state. /// Collect no more than `max_candidates_to_evict` elements. - /// Return `true` if the first condition is satisfied. - enum class DesiredSizeStatus + /// Return SUCCESS status if the first condition is satisfied. + enum class CollectStatus { SUCCESS, CANNOT_EVICT, REACHED_MAX_CANDIDATES_LIMIT, }; - virtual DesiredSizeStatus collectCandidatesForEviction( + virtual CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 7970eaa3e13..0e0170c76e3 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -323,7 +323,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::CollectStatus LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -336,23 +336,23 @@ IFileCachePriority::DesiredSizeStatus LRUFileCachePriority::collectCandidatesFor return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock, &desired_size, &desired_elements_count); }; - auto status = DesiredSizeStatus::CANNOT_EVICT; + auto status = CollectStatus::CANNOT_EVICT; auto stop_condition = [&]() { if (desired_limits_satisfied()) { - status = DesiredSizeStatus::SUCCESS; + status = CollectStatus::SUCCESS; return true; } if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) { - status = DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT; + status = CollectStatus::REACHED_MAX_CANDIDATES_LIMIT; return true; } return false; }; iterateForEviction(res, stat, stop_condition, lock); - chassert(status != DesiredSizeStatus::SUCCESS || stop_condition()); + chassert(status != CollectStatus::SUCCESS || stop_condition()); return status; } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 9bced106727..0ca62b19d37 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -63,7 +63,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - DesiredSizeStatus collectCandidatesForEviction( + CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index dc0df223cb0..f5ea519d7d4 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -256,7 +256,7 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesForEviction( +IFileCachePriority::CollectStatus SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, @@ -285,7 +285,7 @@ IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesFo chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (probationary_desired_size_status == DesiredSizeStatus::REACHED_MAX_CANDIDATES_LIMIT) + if (probationary_desired_size_status == CollectStatus::REACHED_MAX_CANDIDATES_LIMIT) return probationary_desired_size_status; const auto desired_protected_size = getRatio(desired_size, size_ratio); @@ -306,7 +306,7 @@ IFileCachePriority::DesiredSizeStatus SLRUFileCachePriority::collectCandidatesFo desired_protected_size, desired_protected_elements_num, protected_queue.getStateInfoForLog(lock)); - if (probationary_desired_size_status == DesiredSizeStatus::SUCCESS) + if (probationary_desired_size_status == CollectStatus::SUCCESS) return protected_desired_size_status; else return probationary_desired_size_status; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index e6d20e0d0ee..23bc8c0908b 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,7 +58,7 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - DesiredSizeStatus collectCandidatesForEviction( + CollectStatus collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, From 2c018d6f4d4d6fa1c04e91306b60b6e85d8e468f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:38:41 +0200 Subject: [PATCH 0602/1170] Update cluster.py --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 90b28a4cda3..acf033de46d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -248,7 +248,7 @@ def check_rabbitmq_is_available(rabbitmq_id, cookie): ), stdout=subprocess.PIPE, ) - p.communicate() + p.wait(timeout=60) return p.returncode == 0 From 53d93b177e371b8dcc13e404f48209e3df456fe4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Aug 2024 11:56:07 +0000 Subject: [PATCH 0603/1170] Fix 02434_cancel_insert_when_client_dies --- .../02434_cancel_insert_when_client_dies.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) 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 f85aaed7716..45f4194104e 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 @@ -10,22 +10,26 @@ export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE -$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' +$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A + settings non_replicated_deduplication_window=1000 + , merge_tree_clear_old_temporary_directories_interval_seconds = 1 + ;" $CLICKHOUSE_CLIENT -q "create table dedup_dist(A Int64) Engine = Distributed('test_cluster_one_shard_two_replicas', currentDatabase(), dedup_test)" function insert_data { - SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000" + # send_logs_level: https://github.com/ClickHouse/ClickHouse/issues/67599 + SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000&send_logs_level=fatal" # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" TYPE=$(( RANDOM % 5 )) if [[ "$TYPE" -eq 0 ]]; then # client will send 10000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) - $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" \ + $CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=fatal --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" \ -q 'insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 1 ]]; then - $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" --prefer_localhost_replica="$(( RANDOM % 2))" \ + $CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=fatal --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" --prefer_localhost_replica="$(( RANDOM % 2))" \ -q 'insert into dedup_dist settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 2 ]]; then $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE From 029deaeee8431d0ef6f2a460c2bd8631c8025254 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 14:13:11 +0200 Subject: [PATCH 0604/1170] Fix 02910_bad_logs_level_in_local in fast tests! --- tests/queries/0_stateless/02910_bad_logs_level_in_local.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh index badf7232a95..b5de10bf191 100755 --- a/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh +++ b/tests/queries/0_stateless/02910_bad_logs_level_in_local.sh @@ -1,14 +1,14 @@ #!/usr/bin/expect -f log_user 0 -set timeout 60 +set timeout 30 match_max 100000 spawn bash -c "clickhouse-local" expect ":) " send -- "SET send_logs_level = 't'\r" -expect "Exception on client:" +expect "Unexpected value of LogsLevel:" {} timeout {exit 1} expect ":) " send -- "exit\r" expect eof From bcc75d3681d45b6637211aca0367703b3e957c05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 14:38:41 +0200 Subject: [PATCH 0605/1170] Make 02477_analyzer_function_hints.sh parallelizable --- .../0_stateless/02477_analyzer_function_hints.sh | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.sh b/tests/queries/0_stateless/02477_analyzer_function_hints.sh index d49c20cab75..f83935e47fb 100755 --- a/tests/queries/0_stateless/02477_analyzer_function_hints.sh +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel - set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -14,11 +12,11 @@ $CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS allow_experimental_analyzer = 1 $CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['uniqExact'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function;" -$CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function AS x -> x + 1;" -$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ - | grep "Maybe you meant: \['test_user_defined_function'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function"; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function_$CLICKHOUSE_DATABASE;" +$CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE AS x -> x + 1;" +$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_function_${CLICKHOUSE_DATABASE}A(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep -E "Maybe you meant: \[.*'test_user_defined_function_$CLICKHOUSE_DATABASE'.*\]" &>/dev/null; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE"; $CLICKHOUSE_CLIENT -q "WITH (x -> x + 1) AS lambda_function SELECT lambda_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['lambda_function'\]" &>/dev/null; From 4adc9523e403ab103ed3dec537b02566287b76ee Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 1 Aug 2024 14:38:07 +0200 Subject: [PATCH 0606/1170] added a stateless test for ENGINE=File(Parquet) --- .../0_stateless/03215_parquet_index.reference | 300 ++++++++++++++++++ .../0_stateless/03215_parquet_index.sql | 17 + 2 files changed, 317 insertions(+) create mode 100644 tests/queries/0_stateless/03215_parquet_index.reference create mode 100644 tests/queries/0_stateless/03215_parquet_index.sql diff --git a/tests/queries/0_stateless/03215_parquet_index.reference b/tests/queries/0_stateless/03215_parquet_index.reference new file mode 100644 index 00000000000..334f2f3824e --- /dev/null +++ b/tests/queries/0_stateless/03215_parquet_index.reference @@ -0,0 +1,300 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 +50 50 +51 51 +52 52 +53 53 +54 54 +55 55 +56 56 +57 57 +58 58 +59 59 +60 60 +61 61 +62 62 +63 63 +64 64 +65 65 +66 66 +67 67 +68 68 +69 69 +70 70 +71 71 +72 72 +73 73 +74 74 +75 75 +76 76 +77 77 +78 78 +79 79 +80 80 +81 81 +82 82 +83 83 +84 84 +85 85 +86 86 +87 87 +88 88 +89 89 +90 90 +91 91 +92 92 +93 93 +94 94 +95 95 +96 96 +97 97 +98 98 +99 99 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 +50 50 +51 51 +52 52 +53 53 +54 54 +55 55 +56 56 +57 57 +58 58 +59 59 +60 60 +61 61 +62 62 +63 63 +64 64 +65 65 +66 66 +67 67 +68 68 +69 69 +70 70 +71 71 +72 72 +73 73 +74 74 +75 75 +76 76 +77 77 +78 78 +79 79 +80 80 +81 81 +82 82 +83 83 +84 84 +85 85 +86 86 +87 87 +88 88 +89 89 +90 90 +91 91 +92 92 +93 93 +94 94 +95 95 +96 96 +97 97 +98 98 +99 99 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 +50 50 +51 51 +52 52 +53 53 +54 54 +55 55 +56 56 +57 57 +58 58 +59 59 +60 60 +61 61 +62 62 +63 63 +64 64 +65 65 +66 66 +67 67 +68 68 +69 69 +70 70 +71 71 +72 72 +73 73 +74 74 +75 75 +76 76 +77 77 +78 78 +79 79 +80 80 +81 81 +82 82 +83 83 +84 84 +85 85 +86 86 +87 87 +88 88 +89 89 +90 90 +91 91 +92 92 +93 93 +94 94 +95 95 +96 96 +97 97 +98 98 +99 99 diff --git a/tests/queries/0_stateless/03215_parquet_index.sql b/tests/queries/0_stateless/03215_parquet_index.sql new file mode 100644 index 00000000000..5b176ff70ba --- /dev/null +++ b/tests/queries/0_stateless/03215_parquet_index.sql @@ -0,0 +1,17 @@ +-- default settings. +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet); +INSERT INTO test_parquet SELECT number, toString(number) FROM numbers(100); +SELECT col1, col2 FROM test_parquet; + +-- Parquet will have indexes in columns. We are not checking that indexes exist here, there is an integration test test_parquet_page_index for that. We just check that a setting doesn't break the SELECT +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet) SETTINGS output_format_parquet_use_custom_encoder=false, output_format_parquet_write_page_index=true; +INSERT INTO test_parquet SELECT number, toString(number) FROM numbers(100); +SELECT col1, col2 FROM test_parquet; + +-- Parquet will not have indexes in columns. +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet) SETTINGS output_format_parquet_use_custom_encoder=false, output_format_parquet_write_page_index=false; +INSERT INTO test_parquet SELECT number, toString(number) FROM numbers(100); +SELECT col1, col2 FROM test_parquet; From a8e7c8ae9a5e345fa1725d664fd490891e4fa9a5 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 1 Aug 2024 14:45:46 +0200 Subject: [PATCH 0607/1170] fix for parallel execution --- tests/integration/test_parquet_page_index/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_parquet_page_index/test.py b/tests/integration/test_parquet_page_index/test.py index a820d1b5bc5..db291e20b74 100644 --- a/tests/integration/test_parquet_page_index/test.py +++ b/tests/integration/test_parquet_page_index/test.py @@ -61,7 +61,7 @@ def delete_if_exists(file_path): }, ) def test_parquet_page_index_select_into_outfile(query, expected_result, start_cluster): - file_name = "export.parquet" + file_name = f"export{time.time()}.parquet" query = query.format(file_name=file_name) delete_if_exists(file_name) assert node.query(query) == "" @@ -76,14 +76,14 @@ def test_parquet_page_index_select_into_outfile(query, expected_result, start_cl { ( "INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SELECT number, number+1 FROM system.numbers LIMIT 100 " "SETTINGS output_format_parquet_use_custom_encoder=false, " "output_format_parquet_write_page_index=true FORMAT Parquet", True, ), ( "INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SELECT number, number+1 FROM system.numbers LIMIT 100 " "SETTINGS output_format_parquet_use_custom_encoder=false, " "output_format_parquet_write_page_index=false FORMAT Parquet", False, @@ -92,7 +92,7 @@ def test_parquet_page_index_select_into_outfile(query, expected_result, start_cl # # output_format_parquet_use_custom_encoder = true ( "INSERT INTO TABLE FUNCTION file('{file_name}') " - "SElECT number, number+1 FROM system.numbers LIMIT 100 FORMAT Parquet", + "SELECT number, number+1 FROM system.numbers LIMIT 100 FORMAT Parquet", False, ), }, @@ -100,7 +100,7 @@ def test_parquet_page_index_select_into_outfile(query, expected_result, start_cl def test_parquet_page_index_insert_into_table_function_file( query, expected_result, start_cluster ): - file_name = "export.parquet" + file_name = f"export{time.time()}.parquet" query = query.format(file_name=file_name) file_path = f"{path_to_external_dirs}{path_to_userfiles}{file_name}" delete_if_exists(file_path) From 008408c81f23ed615cb899048418dc46aa3c2a9f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 13:47:34 +0100 Subject: [PATCH 0608/1170] impl --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 9b96ce3e586..aaeee466794 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage +-- Tags: no-random-merge-tree-settings, no-random-settings, no-tsan, no-debug, no-object-storage, long -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher @@ -16,7 +16,7 @@ CREATE TABLE adaptive_table( value String ) ENGINE MergeTree() ORDER BY key -SETTINGS index_granularity_bytes=1048576, +SETTINGS index_granularity_bytes = 1048576, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, enable_vertical_merge_algorithm = 0; From 70228acd7e809230582883a0b6b70c4cd9c04daa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Aug 2024 15:02:29 +0200 Subject: [PATCH 0609/1170] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a4c873ba3f9..5cd4200d9ad 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,7 +22,6 @@ #### New Feature * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). -* Add new window function `percent_rank`. [#62747](https://github.com/ClickHouse/ClickHouse/pull/62747) ([lgbo](https://github.com/lgbo-ustc)). * Support JWT authentication in `clickhouse-client` (will be available only in ClickHouse Cloud). [#62829](https://github.com/ClickHouse/ClickHouse/pull/62829) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Add SQL functions `changeYear`, `changeMonth`, `changeDay`, `changeHour`, `changeMinute`, `changeSecond`. For example, `SELECT changeMonth(toDate('2024-06-14'), 7)` returns date `2024-07-14`. [#63186](https://github.com/ClickHouse/ClickHouse/pull/63186) ([cucumber95](https://github.com/cucumber95)). * Introduce startup scripts, which allow the execution of preconfigured queries at the startup stage. [#64889](https://github.com/ClickHouse/ClickHouse/pull/64889) ([pufit](https://github.com/pufit)). From 48c6e36dfd23f297907575ce4696f761aec49e11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:03:41 +0200 Subject: [PATCH 0610/1170] Make 01062_window_view_event_hop_watch_asc parallelizable --- .../01062_window_view_event_hop_watch_asc.py | 28 ++++++++----------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index d6cc3ee1a88..3c85ff30ba8 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -1,5 +1,4 @@ #!/usr/bin/env python3 -# Tags: no-parallel import os import sys @@ -17,6 +16,7 @@ log = None with client(name="client1>", log=log) as client1, client( name="client2>", log=log ) as client2: + database_name = os.environ["CLICKHOUSE_DATABASE"] client1.expect(prompt) client2.expect(prompt) @@ -31,40 +31,38 @@ with client(name="client1>", log=log) as client1, client( client2.send("SET allow_experimental_analyzer = 0") client2.expect(prompt) - client1.send("CREATE DATABASE IF NOT EXISTS 01062_window_view_event_hop_watch_asc") + client1.send(f"DROP TABLE IF EXISTS {database_name}.mt") client1.expect(prompt) - client1.send("DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.mt") - client1.expect(prompt) - client1.send("DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.wv SYNC") + client1.send(f"DROP TABLE IF EXISTS {database_name}.wv SYNC") client1.expect(prompt) client1.send( - "CREATE TABLE 01062_window_view_event_hop_watch_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" + f"CREATE TABLE {database_name}.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" ) client1.expect(prompt) client1.send( - "CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid" + f"CREATE WINDOW VIEW {database_name}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM {database_name}.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid" ) client1.expect(prompt) - client1.send("WATCH 01062_window_view_event_hop_watch_asc.wv") + client1.send(f"WATCH {database_name}.wv") client1.expect("Query id" + end_of_block) client1.expect("Progress: 0.00 rows.*\\)") client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));" ) client2.expect(prompt) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));" ) client2.expect(prompt) client1.expect("1*" + end_of_block) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" ) client2.expect(prompt) client2.send( - "INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));" + f"INSERT INTO {database_name}.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));" ) client2.expect(prompt) client1.expect("1" + end_of_block) @@ -77,9 +75,7 @@ with client(name="client1>", log=log) as client1, client( if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) - client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.wv SYNC") + client1.send(f"DROP TABLE {database_name}.wv SYNC") client1.expect(prompt) - client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.mt") - client1.expect(prompt) - client1.send("DROP DATABASE IF EXISTS 01062_window_view_event_hop_watch_asc") + client1.send(f"DROP TABLE {database_name}.mt") client1.expect(prompt) From 7b72362e99a093a4f880d333c1b50cd114b590c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:06:27 +0200 Subject: [PATCH 0611/1170] 01493_alter_remove_properties_zookeeper is already parallelizable --- .../0_stateless/01493_alter_remove_properties_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql index 92e6fce2c93..362da3ac364 100644 --- a/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql +++ b/tests/queries/0_stateless/01493_alter_remove_properties_zookeeper.sql @@ -1,4 +1,4 @@ --- Tags: zookeeper, no-parallel +-- Tags: zookeeper DROP TABLE IF EXISTS r_prop_table1; DROP TABLE IF EXISTS r_prop_table2; From 56e48cf43b4d13810e2bb4b4e941954b654a1cb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:09:56 +0200 Subject: [PATCH 0612/1170] Make 01676_range_hashed_dictionary parallelizable --- .../01676_range_hashed_dictionary.sql | 92 +++++++++---------- 1 file changed, 42 insertions(+), 50 deletions(-) diff --git a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql index 430f3a86dc1..ba2a9eba87f 100644 --- a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql +++ b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql @@ -1,10 +1,4 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS database_for_range_dict; - -CREATE DATABASE database_for_range_dict; - -CREATE TABLE database_for_range_dict.date_table +CREATE TABLE date_table ( CountryID UInt64, StartDate Date, @@ -14,11 +8,11 @@ CREATE TABLE database_for_range_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_range_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); +INSERT INTO date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); -CREATE DICTIONARY database_for_range_dict.range_dictionary +CREATE DICTIONARY range_dictionary ( CountryID UInt64, StartDate Date, @@ -26,7 +20,7 @@ CREATE DICTIONARY database_for_range_dict.range_dictionary Tax Float64 DEFAULT 0.2 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_range_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate) @@ -35,30 +29,30 @@ SETTINGS(dictionary_use_async_executor=1, max_threads=8) SELECT 'Dictionary not nullable'; SELECT 'dictGet'; -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31')); -SELECT dictGetOrDefault('database_for_range_dict.range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGet('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetOrDefault('range_dictionary', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); SELECT 'dictHas'; -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(1), toDate('2019-05-15')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(1), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(2), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary', toUInt64(2), toDate('2019-05-31')); +SELECT dictHas('range_dictionary', toUInt64(1), toDate('2019-05-15')); +SELECT dictHas('range_dictionary', toUInt64(1), toDate('2019-05-29')); +SELECT dictHas('range_dictionary', toUInt64(2), toDate('2019-05-29')); +SELECT dictHas('range_dictionary', toUInt64(2), toDate('2019-05-31')); SELECT 'select columns from dictionary'; SELECT 'allColumns'; -SELECT * FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT * FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'noColumns'; -SELECT 1 FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT 1 FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumns'; -SELECT CountryID, StartDate, Tax FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT CountryID, StartDate, Tax FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumn'; -SELECT Tax FROM database_for_range_dict.range_dictionary ORDER BY CountryID, StartDate, EndDate; +SELECT Tax FROM range_dictionary ORDER BY CountryID, StartDate, EndDate; -DROP DICTIONARY database_for_range_dict.range_dictionary; -DROP TABLE database_for_range_dict.date_table; +DROP DICTIONARY range_dictionary; +DROP TABLE date_table; -CREATE TABLE database_for_range_dict.date_table +CREATE TABLE date_table ( CountryID UInt64, StartDate Date, @@ -68,11 +62,11 @@ CREATE TABLE database_for_range_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_range_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_range_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), NULL); +INSERT INTO date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), NULL); -CREATE DICTIONARY database_for_range_dict.range_dictionary_nullable +CREATE DICTIONARY range_dictionary_nullable ( CountryID UInt64, StartDate Date, @@ -80,35 +74,33 @@ CREATE DICTIONARY database_for_range_dict.range_dictionary_nullable Tax Nullable(Float64) DEFAULT 0.2 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_range_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); SELECT 'Dictionary nullable'; SELECT 'dictGet'; -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGet('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31')); -SELECT dictGetOrDefault('database_for_range_dict.range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGet('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetOrDefault('range_dictionary_nullable', 'Tax', toUInt64(2), toDate('2019-05-31'), 0.4); SELECT 'dictHas'; -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(1), toDate('2019-05-15')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(1), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(2), toDate('2019-05-29')); -SELECT dictHas('database_for_range_dict.range_dictionary_nullable', toUInt64(2), toDate('2019-05-31')); +SELECT dictHas('range_dictionary_nullable', toUInt64(1), toDate('2019-05-15')); +SELECT dictHas('range_dictionary_nullable', toUInt64(1), toDate('2019-05-29')); +SELECT dictHas('range_dictionary_nullable', toUInt64(2), toDate('2019-05-29')); +SELECT dictHas('range_dictionary_nullable', toUInt64(2), toDate('2019-05-31')); SELECT 'select columns from dictionary'; SELECT 'allColumns'; -SELECT * FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT * FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'noColumns'; -SELECT 1 FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT 1 FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumns'; -SELECT CountryID, StartDate, Tax FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT CountryID, StartDate, Tax FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; SELECT 'onlySpecificColumn'; -SELECT Tax FROM database_for_range_dict.range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; +SELECT Tax FROM range_dictionary_nullable ORDER BY CountryID, StartDate, EndDate; -DROP DICTIONARY database_for_range_dict.range_dictionary_nullable; -DROP TABLE database_for_range_dict.date_table; - -DROP DATABASE database_for_range_dict; +DROP DICTIONARY range_dictionary_nullable; +DROP TABLE date_table; From bb7039eeec01ce59008103727f8c03ddd26a3d29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 15:17:00 +0200 Subject: [PATCH 0613/1170] Make 01107_atomic_db_detach_attach parallelizable --- .../01107_atomic_db_detach_attach.sh | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index bcaa70abbb5..e9879344259 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -1,29 +1,30 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" +NEW_DATABASE=test_01107_${CLICKHOUSE_DATABASE} +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS ${NEW_DATABASE}" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${NEW_DATABASE} ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${NEW_DATABASE}.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${NEW_DATABASE}.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 -$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 -$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" -$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -q "DETACH TABLE ${NEW_DATABASE}.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "ATTACH TABLE ${NEW_DATABASE}.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK" wait -$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 -$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" +$CLICKHOUSE_CLIENT -q "ATTACH TABLE ${NEW_DATABASE}.mt" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${NEW_DATABASE}.mt" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE ${NEW_DATABASE}" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${NEW_DATABASE}.mt" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${NEW_DATABASE}.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${NEW_DATABASE}" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait From 1d85f9b1cba3c8fe168286a660d3c0a4fd471a95 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 1 Aug 2024 14:42:58 +0100 Subject: [PATCH 0614/1170] fix remove_local_directory_contents --- tests/integration/test_storage_delta/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index e485bc90ee0..384b8296f66 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -52,9 +52,13 @@ def get_spark(): return builder.master("local").getOrCreate() -def remove_local_directory_contents(local_path): - for local_file in glob.glob(local_path + "/**"): - os.unlink(local_file) +def remove_local_directory_contents(full_path): + for path in glob.glob(f"{full_path}/**"): + if os.path.isfile(path): + os.unlink(path) + else: + remove_local_directory_contents(path) + os.rmdir(path) @pytest.fixture(scope="module") From 0913f0189ba350236d32d774265770b654860a80 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Wed, 31 Jul 2024 09:06:30 +0000 Subject: [PATCH 0615/1170] Don't count a search query as a search pattern match --- tests/integration/helpers/cluster.py | 10 +++++++--- .../integration/test_mask_sensitive_info/test.py | 15 +++++++-------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 90b28a4cda3..6bc0ece63ca 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3922,7 +3922,11 @@ class ClickHouseInstance: ) def contains_in_log( - self, substring, from_host=False, filename="clickhouse-server.log" + self, + substring, + from_host=False, + filename="clickhouse-server.log", + exclusion_substring="", ): if from_host: # We check fist file exists but want to look for all rotated logs as well @@ -3930,7 +3934,7 @@ class ClickHouseInstance: [ "bash", "-c", - f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{substring}" {self.logs_dir}/{filename}* || true', + f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{substring}" {self.logs_dir}/{filename}* | ( [ -z "{exclusion_substring}" ] && cat || grep -v "${exclusion_substring}" ) || true', ] ) else: @@ -3938,7 +3942,7 @@ class ClickHouseInstance: [ "bash", "-c", - f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{substring}" /var/log/clickhouse-server/{filename} || true', + f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{substring}" /var/log/clickhouse-server/{filename} | ( [ -z "{exclusion_substring}" ] && cat || grep -v "${exclusion_substring}" ) || true', ] ) return len(result) > 0 diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 902d3800324..6f6dc4d287f 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -13,6 +13,7 @@ node = cluster.add_instance( with_zookeeper=True, with_azurite=True, ) +base_search_query = "SELECT COUNT() FROM system.query_log WHERE query LIKE " @pytest.fixture(scope="module", autouse=True) @@ -35,7 +36,7 @@ def check_logs(must_contain=[], must_not_contain=[]): .replace("]", "\\]") .replace("*", "\\*") ) - assert node.contains_in_log(escaped_str) + assert node.contains_in_log(escaped_str, exclusion_substring=base_search_query) for str in must_not_contain: escaped_str = ( @@ -44,7 +45,9 @@ def check_logs(must_contain=[], must_not_contain=[]): .replace("]", "\\]") .replace("*", "\\*") ) - assert not node.contains_in_log(escaped_str) + assert not node.contains_in_log( + escaped_str, exclusion_substring=base_search_query + ) for str in must_contain: escaped_str = str.replace("'", "\\'") @@ -60,7 +63,7 @@ def system_query_log_contains_search_pattern(search_pattern): return ( int( node.query( - f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{search_pattern}%'" + f"{base_search_query}'%{search_pattern}%' AND query NOT LIKE '{base_search_query}%'" ).strip() ) >= 1 @@ -105,7 +108,6 @@ def test_create_alter_user(): must_not_contain=[ password, "IDENTIFIED BY", - "IDENTIFIED BY", "IDENTIFIED WITH plaintext_password BY", ], ) @@ -366,10 +368,7 @@ def test_table_functions(): f"remoteSecure(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}')", f"s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", f"s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '{password}')", - ( - f"deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", - "DNS_ERROR", - ), + f"deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", f"azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", From 048fbacc40062b05510916134c9d9525e7fab63a Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 1 Aug 2024 16:48:19 +0200 Subject: [PATCH 0616/1170] Update README.md --- README.md | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 98f9108f14c..2120a4d1211 100644 --- a/README.md +++ b/README.md @@ -34,17 +34,13 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.7 Community Call](https://clickhouse.com/company/events/v24-7-community-release-call) - Jul 30 +* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 29 ## Upcoming Events Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9 -* [ClickHouse Cloud - Live Update Call](https://clickhouse.com/company/events/202407-cloud-update-live) - Jul 9 -* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9 -* [AWS Summit in New York](https://clickhouse.com/company/events/2024-07-awssummit-nyc) - Jul 10 -* [ClickHouse Meetup @ Klaviyo - Boston](https://www.meetup.com/clickhouse-boston-user-group/events/300907870) - Jul 11 +* MORE COMING SOON! ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 582bcfdc03e1124500325c3104497719473657cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 16:57:41 +0200 Subject: [PATCH 0617/1170] Add no-parallel back to 01107_atomic_db_detach_attach --- tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e9879344259..a6a99aadac2 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel +# no-parallel: FIXME: Timing issues with INSERT + DETACH (https://github.com/ClickHouse/ClickHouse/pull/67610/files#r1700345054) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0978441a0261c6003c7a9f4661ac87138e909622 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0618/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 53cbb4811047cad2bdf2b882bc89ff9a83ac4577 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:06:05 +0200 Subject: [PATCH 0619/1170] Try fix 03143_asof_join_ddb_long --- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..0b17ade5d1c 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-random-merge-tree-settings +-- no-random-merge-tree-settings - times out in private DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From 9362d1a5668bcd6e4e629ab26ec44d4bc8cb6513 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 15:04:13 +0200 Subject: [PATCH 0620/1170] CI: Create release workflow dry run fix fix --- .github/actions/release/action.yml | 30 +++--- .github/workflows/create_release.yml | 144 +++++++++++++++++++++++++-- tests/ci/create_release.py | 2 +- 3 files changed, 153 insertions(+), 23 deletions(-) diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml index c3897682a33..a287aa8b41d 100644 --- a/.github/actions/release/action.yml +++ b/.github/actions/release/action.yml @@ -16,8 +16,7 @@ inputs: - new dry-run: description: 'Dry run' - required: false - default: true + required: true type: boolean token: required: true @@ -30,8 +29,7 @@ runs: shell: bash run: | python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ - ${{ inputs.dry-run && '--dry-run' || '' }} + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} echo "::group::Release Info" python3 -m json.tool /tmp/release_info.json echo "::endgroup::" @@ -44,20 +42,20 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push Git Tag for the Release shell: bash run: | - python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push New Release Branch if: ${{ inputs.type == 'new' }} shell: bash run: | - python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump CH Version and Update Contributors' List shell: bash run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash @@ -107,37 +105,37 @@ runs: shell: bash if: ${{ inputs.type == 'patch' }} run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export RPM Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test RPM Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export Debian Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test Debian Packages if: ${{ inputs.type == 'patch' }} shell: bash run: | - python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Docker clickhouse/clickhouse-server building if: ${{ inputs.type == 'patch' }} shell: bash @@ -165,4 +163,4 @@ runs: if: ${{ !cancelled() }} shell: bash run: | - python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 55644bdd503..217f27086c5 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -35,10 +35,142 @@ jobs: with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} fetch-depth: 0 - - name: Call Release Action - uses: ./.github/actions/release + - name: Prepare Release Info + shell: bash + run: | + python3 ./tests/ci/create_release.py --prepare-release-info \ + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} + echo "::group::Release Info" + python3 -m json.tool /tmp/release_info.json + echo "::endgroup::" + release_tag=$(jq -r '.release_tag' /tmp/release_info.json) + commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + echo "Release Tag: $release_tag" + echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" + echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + - name: Download All Release Artifacts + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Push Git Tag for the Release + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Push New Release Branch + if: ${{ inputs.type == 'new' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump Docker versions, Changelog, Security + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + git checkout master + python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" + echo "List versions" + ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv + echo "Update docker version" + ./utils/list-versions/update-docker-version.sh + echo "Generate ChangeLog" + export CI=1 + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ + --volume=".:/ClickHouse" clickhouse/style-test \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --gh-user-or-token=${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} --jobs=5 \ + --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md + echo "Generate Security" + python3 ./utils/security-generator/generate_security.py > SECURITY.md + git diff HEAD + - name: Create ChangeLog PR + if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + uses: peter-evans/create-pull-request@v6 with: - ref: ${{ inputs.ref }} - type: ${{ inputs.type }} - dry-run: ${{ inputs.dry-run }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + author: "robot-clickhouse " + token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} + committer: "robot-clickhouse " + commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + branch: auto/${{ env.RELEASE_TAG }} + assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher + delete-branch: true + title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} + labels: do not test + body: | + Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + ### Changelog category (leave one): + - Not for changelog (changelog entry is not required) + - name: Complete previous steps and Restore git state + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --set-progress-completed + git reset --hard HEAD + git checkout "$GITHUB_REF_NAME" + - name: Create GH Release + shell: bash + if: ${{ inputs.type == 'patch' }} + run: | + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Export Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Test Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Docker clickhouse/clickhouse-server building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker server release" + export CHECK_NAME="Docker server image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./create_release.py --set-progress-completed + - name: Docker clickhouse/clickhouse-keeper building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + python3 ./create_release.py --set-progress-started --progress "docker keeper release" + export CHECK_NAME="Docker keeper image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./create_release.py --set-progress-completed + - name: Set current Release progress to Completed with OK + shell: bash + run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" + python3 ./tests/ci/create_release.py --set-progress-completed + - name: Post Slack Message + if: ${{ !cancelled() }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index a0b4083b673..b02a0bb8ed5 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -315,7 +315,7 @@ class ReleaseInfo: cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" - cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file '{body_file} --label 'do not test' --assignee @{actor}" + cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) Shell.run(cmd_push_branch, check=True, dry_run=dry_run) Shell.run(cmd_create_pr, check=True, dry_run=dry_run) From c534cd5bc21b59788750bdfcfb4177ebba0afc85 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 16:22:43 +0200 Subject: [PATCH 0621/1170] changelog.py to retrieve best token s3fs fix changelog.py to use base branch to filter prs --- .github/workflows/create_release.yml | 12 ++++++------ tests/ci/artifactory.py | 4 ++++ tests/ci/changelog.py | 23 +++++++++++++++++++++++ 3 files changed, 33 insertions(+), 6 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 217f27086c5..5e34f50fab5 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -81,7 +81,7 @@ jobs: docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ --volume=".:/ClickHouse" clickhouse/style-test \ /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token=${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} --jobs=5 \ + --jobs=5 \ --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md echo "Generate Security" @@ -111,11 +111,11 @@ jobs: python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - shell: bash - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} +# - name: Create GH Release +# shell: bash +# if: ${{ inputs.type == 'patch' }} +# run: | +# python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 86dcaf79854..4ee326593e6 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -52,6 +52,10 @@ class R2MountPoint: if self.CACHE_ENABLED else "" ) + if not dry_run: + self.aux_mount_options += ( + "-o passwd_file /home/ubuntu/.passwd-s3fs_packages " + ) # without -o nomultipart there are errors like "Error 5 writing to /home/ubuntu/***.deb: Input/output error" self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" elif self.app == MountPointApp.RCLONE: diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 3ba618f3ae5..e23dd8e4c67 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -19,6 +19,8 @@ from env_helper import TEMP_PATH from git_helper import git_runner, is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from ci_utils import Shell from version_helper import ( FILE_WITH_VERSION_PATH, get_abs_path, @@ -171,6 +173,7 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--gh-user-or-token", help="user name or GH token to authenticate", + default=get_best_robot_token(), ) parser.add_argument( "--gh-password", @@ -397,6 +400,15 @@ def get_year(prs: PullRequests) -> int: return max(pr.created_at.year for pr in prs) +def get_branch_by_tag(tag: str) -> Optional[str]: + tag.removeprefix("v") + versions = tag.split(".") + if len(versions) < 3: + print("ERROR: Can't get branch by tag") + return None + return f"{versions[0]}.{versions[1]}" + + def main(): log_levels = [logging.WARN, logging.INFO, logging.DEBUG] args = parse_args() @@ -446,6 +458,17 @@ def main(): gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" + branch = get_branch_by_tag(TO_REF) + if branch and Shell.check(f"git show-ref --quiet {branch}"): + try: + if int(branch.split(".")[-1]) > 1: + query += f" base:{branch}" + print(f"NOTE: will use base branch to filter PRs {branch}") + except ValueError: + print(f"ERROR: cannot get minor version from branch {branch} - pass") + pass + else: + print(f"ERROR: invalid branch {branch} - pass") prs = gh.get_pulls_from_search( query=query, merged=merged, sort="created", progress_func=tqdm.tqdm ) From 8214910cc7bd84f613631c2fada9682820df8003 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 31 Jul 2024 20:14:22 +0200 Subject: [PATCH 0622/1170] add geesfs --- .github/workflows/create_release.yml | 10 +++---- .github/workflows/release.yml | 4 +-- tests/ci/artifactory.py | 35 +++++++++++++++++++++--- tests/ci/changelog.py | 41 ++++++++++++++++++---------- tests/ci/ci_utils.py | 11 ++++---- 5 files changed, 69 insertions(+), 32 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 5e34f50fab5..c3126abe461 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -62,15 +62,14 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - shell: bash - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} +# - name: Bump CH Version and Update Contributors' List +# shell: bash +# run: | +# python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash run: | - git checkout master python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv @@ -96,6 +95,7 @@ jobs: committer: "robot-clickhouse " commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} branch: auto/${{ env.RELEASE_TAG }} + base: master assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher delete-branch: true title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 3bd6dfae6ca..8620d15ec19 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -55,12 +55,12 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --check-name "$CHECK_NAME" --push + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push - name: Check docker clickhouse/clickhouse-keeper building run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --check-name "$CHECK_NAME" --push + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push - name: Cleanup if: always() run: | diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 4ee326593e6..a508374f856 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -15,6 +15,7 @@ from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): RCLONE = "rclone" S3FS = "s3fs" + GEESEFS = "geesefs" class R2MountPoint: @@ -70,6 +71,20 @@ class R2MountPoint: ) # Use --no-modtime to try to avoid: ERROR : rpm/lts/clickhouse-client-24.3.6.5.x86_64.rpm: Failed to apply pending mod time self.mount_cmd = f"rclone mount remote:{self.bucket_name} {self.MOUNT_POINT} --daemon --cache-dir {self.cache_dir} --umask 0000 --log-file {self.LOG_FILE} {self.aux_mount_options}" + elif self.app == MountPointApp.GEESEFS: + self.cache_dir = "/home/ubuntu/geesefs_cache" + self.aux_mount_options += ( + f" --cache={self.cache_dir} " if self.CACHE_ENABLED else "" + ) + if not dry_run: + self.aux_mount_options += f" --shared-config=/home/ubuntu/.r2_auth " + else: + self.aux_mount_options += ( + f" --shared-config=/home/ubuntu/.r2_auth_test " + ) + if self.DEBUG: + self.aux_mount_options += " --debug_s3 --debug_fuse " + self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=2050 --gc-interval=100 --max-flushers=5 --max-parallel-parts=1 --max-parallel-copy=2 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" else: assert False @@ -87,7 +102,7 @@ class R2MountPoint: Shell.run(_UNMOUNT_CMD) Shell.run(_MKDIR_CMD) Shell.run(_MKDIR_FOR_CACHE) - if self.app == MountPointApp.S3FS: + if self.app != MountPointApp.RCLONE: Shell.run(self.mount_cmd, check=True) else: # didn't manage to use simple run() and without blocking or failure @@ -158,7 +173,13 @@ class DebianArtifactory: cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' print("Running test command:") print(f" {cmd}") - Shell.run(cmd, check=True) + assert Shell.check(cmd) + print(f"Test packages installation, version [latest]") + debian_command_2 = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static clickhouse-client" + cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command_2}"' + print("Running test command:") + print(f" {cmd}") + assert Shell.check(cmd) self.release_info.debian_command = debian_command self.release_info.dump() @@ -234,7 +255,13 @@ class RpmArtifactory: cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' print("Running test command:") print(f" {cmd}") - Shell.run(cmd, check=True) + assert Shell.check(cmd) + print(f"Test package installation, version [latest]") + rpm_command_2 = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client" + cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command_2}"' + print("Running test command:") + print(f" {cmd}") + assert Shell.check(cmd) self.release_info.rpm_command = rpm_command self.release_info.dump() @@ -350,7 +377,7 @@ if __name__ == "__main__": ERROR : IO error: NotImplemented: versionId not implemented Failed to copy: NotImplemented: versionId not implemented """ - mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) + mp = R2MountPoint(MountPointApp.GEESEFS, dry_run=args.dry_run) if args.export_debian: with ReleaseContextManager( release_progress=ReleaseProgress.EXPORT_DEB diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index e23dd8e4c67..929f0f3523a 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -7,7 +7,7 @@ import re from datetime import date, timedelta from pathlib import Path from subprocess import DEVNULL -from typing import Any, Dict, List, Optional, TextIO +from typing import Any, Dict, List, Optional, TextIO, Tuple import tqdm # type: ignore from github.GithubException import RateLimitExceededException, UnknownObjectException @@ -400,13 +400,19 @@ def get_year(prs: PullRequests) -> int: return max(pr.created_at.year for pr in prs) -def get_branch_by_tag(tag: str) -> Optional[str]: - tag.removeprefix("v") +def get_branch_and_patch_by_tag(tag: str) -> Tuple[Optional[str], Optional[int]]: + tag = tag.removeprefix("v") versions = tag.split(".") - if len(versions) < 3: + if len(versions) < 4: print("ERROR: Can't get branch by tag") - return None - return f"{versions[0]}.{versions[1]}" + return None, None + try: + patch_version = int(versions[2]) + branch = f"{int(versions[0])}.{int(versions[1])}" + print(f"Branch [{branch}], patch version [{patch_version}]") + except ValueError: + return None, None + return branch, patch_version def main(): @@ -458,17 +464,22 @@ def main(): gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) gh_cache.download() query = f"type:pr repo:{args.repo} is:merged" - branch = get_branch_by_tag(TO_REF) - if branch and Shell.check(f"git show-ref --quiet {branch}"): - try: - if int(branch.split(".")[-1]) > 1: - query += f" base:{branch}" - print(f"NOTE: will use base branch to filter PRs {branch}") - except ValueError: - print(f"ERROR: cannot get minor version from branch {branch} - pass") - pass + + branch, patch = get_branch_and_patch_by_tag(TO_REF) + if branch and patch and Shell.check(f"git show-ref --quiet {branch}"): + if patch > 1: + query += f" base:{branch}" + print( + f"NOTE: It's a patch [{patch}]. will use base branch to filter PRs [{branch}]" + ) + else: + print( + f"NOTE: It's a first patch version. should count PRs merged on master - won't filter PRs by branch" + ) else: print(f"ERROR: invalid branch {branch} - pass") + + print(f"Fetch PRs with query {query}") prs = gh.get_pulls_from_search( query=query, merged=merged, sort="created", progress_func=tqdm.tqdm ) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 447aac74c7f..3182c0bc5d8 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -246,15 +246,14 @@ class Shell: @classmethod def check(cls, command): - result = subprocess.run( + proc = subprocess.Popen( command, shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=False, + stdout=subprocess.STDOUT, + stderr=subprocess.STDOUT, ) - return result.returncode == 0 + proc.wait() + return proc.returncode == 0 class Utils: From 4802ea540a4691c13386b74416352155b93f713d Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 11:57:54 +0200 Subject: [PATCH 0623/1170] improve ci_utils' Shell --- .github/workflows/pull_request.yml | 9 +++- .github/workflows/release.yml | 6 ++- pyproject.toml | 1 + tests/ci/artifactory.py | 84 +++++++++++------------------ tests/ci/auto_release.py | 5 +- tests/ci/ci_buddy.py | 6 ++- tests/ci/ci_definitions.py | 3 +- tests/ci/ci_utils.py | 72 ++++++++++++------------- tests/ci/create_release.py | 86 +++++++++++++++++------------- tests/ci/docker_images_helper.py | 8 +-- 10 files changed, 141 insertions(+), 139 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 854dff530e7..04bef1460a6 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -142,8 +142,13 @@ jobs: # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) Builds_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Builds') }} - needs: [RunConfig, StyleCheck, Builds_1, Builds_2] + if: ${{ !cancelled() + && needs.RunConfig.result == 'success' + && needs.StyleCheck.result != 'failure' + && needs.FastTest.result != 'failure' + && needs.BuildDockers.result != 'failure' + && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Builds') }} + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: Builds diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 8620d15ec19..7dc4e3298a6 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -55,12 +55,14 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push + SHA=$(git rev-list -n 1 "$GITHUB_TAG") + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - name: Check docker clickhouse/clickhouse-keeper building run: | cd "$GITHUB_WORKSPACE/tests/ci" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$(git rev-list -n 1 $GITHUB_TAG)" --check-name "$CHECK_NAME" --push + SHA=$(git rev-list -n 1 "$GITHUB_TAG") + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - name: Cleanup if: always() run: | diff --git a/pyproject.toml b/pyproject.toml index c89d46c0929..9bbeac3ddae 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -39,6 +39,7 @@ disable = ''' no-else-return, global-statement, f-string-without-interpolation, + consider-using-with, ''' [tool.pylint.SIMILARITIES] diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index a508374f856..71deaccf917 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -13,7 +13,6 @@ from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): - RCLONE = "rclone" S3FS = "s3fs" GEESEFS = "geesefs" @@ -31,9 +30,6 @@ class R2MountPoint: DEBUG = True # enable cache for mountpoint CACHE_ENABLED = False - # TODO: which mode is better: minimal/writes/full/off - _RCLONE_CACHE_MODE = "minimal" - UMASK = "0000" def __init__(self, app: str, dry_run: bool) -> None: assert app in MountPointApp @@ -59,18 +55,6 @@ class R2MountPoint: ) # without -o nomultipart there are errors like "Error 5 writing to /home/ubuntu/***.deb: Input/output error" self.mount_cmd = f"s3fs {self.bucket_name} {self.MOUNT_POINT} -o url={self.API_ENDPOINT} -o use_path_request_style -o umask=0000 -o nomultipart -o logfile={self.LOG_FILE} {self.aux_mount_options}" - elif self.app == MountPointApp.RCLONE: - # run rclone mount process asynchronously, otherwise subprocess.run(daemonized command) will not return - self.cache_dir = "/home/ubuntu/rclone_cache" - self.aux_mount_options += "--no-modtime " if self.NOMODTIME else "" - self.aux_mount_options += "-v " if self.DEBUG else "" # -vv too verbose - self.aux_mount_options += ( - f"--vfs-cache-mode {self._RCLONE_CACHE_MODE} --vfs-cache-max-size {self._CACHE_MAX_SIZE_GB}G" - if self.CACHE_ENABLED - else "--vfs-cache-mode off" - ) - # Use --no-modtime to try to avoid: ERROR : rpm/lts/clickhouse-client-24.3.6.5.x86_64.rpm: Failed to apply pending mod time - self.mount_cmd = f"rclone mount remote:{self.bucket_name} {self.MOUNT_POINT} --daemon --cache-dir {self.cache_dir} --umask 0000 --log-file {self.LOG_FILE} {self.aux_mount_options}" elif self.app == MountPointApp.GEESEFS: self.cache_dir = "/home/ubuntu/geesefs_cache" self.aux_mount_options += ( @@ -98,22 +82,17 @@ class R2MountPoint: ) _TEST_MOUNT_CMD = f"mount | grep -q {self.MOUNT_POINT}" - Shell.run(_CLEAN_LOG_FILE_CMD) - Shell.run(_UNMOUNT_CMD) - Shell.run(_MKDIR_CMD) - Shell.run(_MKDIR_FOR_CACHE) - if self.app != MountPointApp.RCLONE: - Shell.run(self.mount_cmd, check=True) - else: - # didn't manage to use simple run() and without blocking or failure - Shell.run_as_daemon(self.mount_cmd) + Shell.check(_CLEAN_LOG_FILE_CMD, verbose=True) + Shell.check(_UNMOUNT_CMD, verbose=True) + Shell.check(_MKDIR_CMD, verbose=True) + Shell.check(_MKDIR_FOR_CACHE, verbose=True) + Shell.check(self.mount_cmd, strict=True, verbose=True) time.sleep(3) - Shell.run(_TEST_MOUNT_CMD, check=True) + Shell.check(_TEST_MOUNT_CMD, strict=True, verbose=True) @classmethod def teardown(cls): - print(f"Unmount [{cls.MOUNT_POINT}]") - Shell.run(f"umount {cls.MOUNT_POINT}") + Shell.check(f"umount {cls.MOUNT_POINT}", verbose=True) class RepoCodenames(metaclass=WithIter): @@ -148,10 +127,9 @@ class DebianArtifactory: ] REPREPRO_CMD_PREFIX = f"reprepro --basedir {R2MountPoint.MOUNT_POINT}/configs/deb --outdir {R2MountPoint.MOUNT_POINT}/deb --verbose" cmd = f"{REPREPRO_CMD_PREFIX} includedeb {self.codename} {' '.join(paths)}" - print("Running export command:") - print(f" {cmd}") - Shell.run(cmd, check=True) - Shell.run("sync") + print("Running export commands:") + Shell.check(cmd, strict=True, verbose=True) + Shell.check("sync") if self.codename == RepoCodenames.LTS: packages_with_version = [ @@ -163,11 +141,11 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} copy {RepoCodenames.STABLE} {RepoCodenames.LTS} {' '.join(packages_with_version)}" print("Running copy command:") print(f" {cmd}") - Shell.run(cmd, check=True) - Shell.run("sync") + Shell.check(cmd, strict=True) + Shell.check("sync") def test_packages(self): - Shell.run("docker pull ubuntu:latest") + Shell.check("docker pull ubuntu:latest", strict=True) print(f"Test packages installation, version [{self.version}]") debian_command = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static={self.version} clickhouse-client={self.version}" cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' @@ -236,20 +214,18 @@ class RpmArtifactory: print(f"Exporting RPM packages into [{codename}]") for command in commands: - print("Running command:") - print(f" {command}") - Shell.run(command, check=True) + Shell.check(command, strict=True, verbose=True) update_public_key = f"gpg --armor --export {self._SIGN_KEY}" pub_key_path = dest_dir / "repodata" / "repomd.xml.key" print("Updating repomd.xml.key") - pub_key_path.write_text(Shell.run(update_public_key, check=True)) + pub_key_path.write_text(Shell.get_output_or_raise(update_public_key)) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - Shell.run("sync") + Shell.check("sync") def test_packages(self): - Shell.run("docker pull fedora:latest") + Shell.check("docker pull fedora:latest", strict=True) print(f"Test package installation, version [{self.version}]") rpm_command = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1" cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' @@ -302,26 +278,30 @@ class TgzArtifactory: if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - Shell.run("sync") + Shell.check("sync") def test_packages(self): tgz_file = "/tmp/tmp.tgz" tgz_sha_file = "/tmp/tmp.tgz.sha512" cmd = f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" - Shell.run( + Shell.check( cmd, - check=True, + strict=True, + verbose=True, ) - Shell.run( + Shell.check( f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512", - check=True, + strict=True, + verbose=True, + ) + expected_checksum = Shell.get_output_or_raise(f"cut -d ' ' -f 1 {tgz_sha_file}") + actual_checksum = Shell.get_output_or_raise( + f"sha512sum {tgz_file} | cut -d ' ' -f 1" ) - expected_checksum = Shell.run(f"cut -d ' ' -f 1 {tgz_sha_file}", check=True) - actual_checksum = Shell.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") assert ( expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" - Shell.run("rm /tmp/tmp.tgz*") + Shell.check("rm /tmp/tmp.tgz*", verbose=True) self.release_info.tgz_command = cmd self.release_info.dump() @@ -373,9 +353,9 @@ if __name__ == "__main__": args = parse_args() """ - Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: - ERROR : IO error: NotImplemented: versionId not implemented - Failed to copy: NotImplemented: versionId not implemented + S3FS - very slow with a big repo + RCLONE - fuse had many different errors with r2 remote and completely removed + GEESEFS ? """ mp = R2MountPoint(MountPointApp.GEESEFS, dry_run=args.dry_run) if args.export_debian: diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index f2386fe207f..6c17b4c74ad 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -85,7 +85,7 @@ class AutoReleaseInfo: def _prepare(token): assert len(token) > 10 os.environ["GH_TOKEN"] = token - Shell.run("gh auth status", check=True) + Shell.check("gh auth status") gh = GitHub(token) prs = gh.get_release_pulls(GITHUB_REPOSITORY) @@ -106,9 +106,8 @@ def _prepare(token): latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - commits = Shell.run( + commits = Shell.get_output_or_raise( f"git rev-list --first-parent {latest_release_tag.tag}..origin/{pr.head.ref}", - check=True, ).split("\n") commit_num = len(commits) print( diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index dfb5885270a..138909c1db0 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -120,8 +120,10 @@ class CIBuddy: ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: - instance_id = Shell.run("ec2metadata --instance-id") or instance_id - instance_type = Shell.run("ec2metadata --instance-type") or instance_type + instance_id = Shell.get_output("ec2metadata --instance-id") or instance_id + instance_type = ( + Shell.get_output("ec2metadata --instance-type") or instance_type + ) if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") sign = ":red_circle:" if not critical else ":black_circle:" diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 054b554b8fa..51de8c63509 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -554,7 +554,7 @@ class CommonJobConfigs: run_command="sqllogic_test.py", timeout=10800, release_only=True, - runner_type=Runners.STYLE_CHECKER, + runner_type=Runners.FUNC_TESTER, ) SQL_TEST = JobConfig( job_name_keyword="sqltest", @@ -582,6 +582,7 @@ class CommonJobConfigs: digest=DigestConfig( include_paths=[ "tests/ci/docker_server.py", + "tests/ci/docker_images_helper.py", "./docker/server", ] ), diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 3182c0bc5d8..cd21554788c 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -2,6 +2,7 @@ import json import os import re import subprocess +import sys import time from contextlib import contextmanager from pathlib import Path @@ -192,7 +193,7 @@ class GHActions: get_url_cmd = ( f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" ) - url = Shell.run(get_url_cmd) + url = Shell.get_output(get_url_cmd) if not url: print(f"ERROR: PR nor found, branch [{branch}]") return url @@ -200,59 +201,56 @@ class GHActions: class Shell: @classmethod - def run_strict(cls, command): + def get_output_or_raise(cls, command): + return cls.get_output(command, strict=True) + + @classmethod + def get_output(cls, command, strict=False): res = subprocess.run( command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE, text=True, - check=True, + check=strict, ) return res.stdout.strip() @classmethod - def run(cls, command, check=False, dry_run=False, **kwargs): + def check( + cls, + command, + strict=False, + verbose=False, + dry_run=False, + stdin_str=None, + **kwargs, + ): if dry_run: print(f"Dry-ryn. Would run command [{command}]") - return "" - print(f"Run command [{command}]") - res = "" - result = subprocess.run( - command, - shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=False, - **kwargs, - ) - if result.returncode == 0: - print(f"stdout: {result.stdout.strip()}") - res = result.stdout - else: - print( - f"ERROR: stdout: {result.stdout.strip()}, stderr: {result.stderr.strip()}" - ) - if check: - assert result.returncode == 0 - return res.strip() - - @classmethod - def run_as_daemon(cls, command): - print(f"Run daemon command [{command}]") - subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with - return 0, "" - - @classmethod - def check(cls, command): + return 0 + if verbose: + print(f"Run command [{command}]") proc = subprocess.Popen( command, shell=True, - stdout=subprocess.STDOUT, stderr=subprocess.STDOUT, + stdout=subprocess.PIPE, + stdin=subprocess.PIPE if stdin_str else None, + universal_newlines=True, + start_new_session=True, + bufsize=1, + errors="backslashreplace", + **kwargs, ) + if stdin_str: + proc.communicate(input=stdin_str) + elif proc.stdout: + for line in proc.stdout: + sys.stdout.write(line) proc.wait() + if strict: + assert proc.returncode == 0 return proc.returncode == 0 @@ -277,7 +275,7 @@ class Utils: @staticmethod def clear_dmesg(): - Shell.run("sudo dmesg --clear ||:") + Shell.check("sudo dmesg --clear", verbose=True) @staticmethod def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index b02a0bb8ed5..0d505d6ccc7 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -137,12 +137,13 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - Shell.run( + Shell.check( f"git merge-base --is-ancestor {commit_ref} origin/master", - check=True, + strict=True, + verbose=True, ) with checkout(commit_ref): - commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) + commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -154,13 +155,13 @@ class ReleaseInfo: ), f"BUG: latest tag [{git.latest_tag}], expected [{expected_prev_tag}]" release_tag = version.describe previous_release_tag = expected_prev_tag - previous_release_sha = Shell.run_strict( + previous_release_sha = Shell.get_output_or_raise( f"git rev-parse {previous_release_tag}" ) assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) + commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -168,11 +169,16 @@ class ReleaseInfo: version.with_description(codename) release_branch = f"{version.major}.{version.minor}" release_tag = version.describe - Shell.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags", check=True) + Shell.check( + f"{GIT_PREFIX} fetch origin {release_branch} --tags", + strict=True, + verbose=True, + ) # check commit is right and on a right branch - Shell.run( + Shell.check( f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}", - check=True, + strict=True, + verbose=True, ) if version.patch == 1: expected_version = copy(version) @@ -197,7 +203,7 @@ class ReleaseInfo: False ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]" - previous_release_sha = Shell.run_strict( + previous_release_sha = Shell.get_output_or_raise( f"git rev-parse {previous_release_tag}" ) assert previous_release_sha @@ -226,25 +232,26 @@ class ReleaseInfo: def push_release_tag(self, dry_run: bool) -> None: if dry_run: # remove locally created tag from prev run - Shell.run( - f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag} ||:" + Shell.check( + f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag}" ) # Create release tag print( f"Create and push release tag [{self.release_tag}], commit [{self.commit_sha}]" ) tag_message = f"Release {self.release_tag}" - Shell.run( + Shell.check( f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}", - check=True, + strict=True, + verbose=True, ) cmd_push_tag = f"{GIT_PREFIX} push origin {self.release_tag}:{self.release_tag}" - Shell.run(cmd_push_tag, dry_run=dry_run, check=True) + Shell.check(cmd_push_tag, dry_run=dry_run, strict=True, verbose=True) @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - Shell.run(cmd, dry_run=dry_run, check=True) + Shell.check(cmd, dry_run=dry_run, strict=True) def push_new_release_branch(self, dry_run: bool) -> None: assert ( @@ -261,7 +268,7 @@ class ReleaseInfo: ), f"Unexpected current version in git, must precede [{self.version}] by one step, actual [{version.string}]" if dry_run: # remove locally created branch from prev run - Shell.run( + Shell.check( f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch}" ) print( @@ -275,7 +282,7 @@ class ReleaseInfo: cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) - Shell.run(cmd_push_branch, dry_run=dry_run, check=True) + Shell.check(cmd_push_branch, dry_run=dry_run, strict=True, verbose=True) print("Create and push backport tags for new release branch") ReleaseInfo._create_gh_label( @@ -284,13 +291,14 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - Shell.run( + Shell.check( f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, dry_run=dry_run, - check=True, + strict=True, + verbose=True, ) def update_version_and_contributors_list(self, dry_run: bool) -> None: @@ -316,13 +324,19 @@ class ReleaseInfo: body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" - Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) - Shell.run(cmd_push_branch, check=True, dry_run=dry_run) - Shell.run(cmd_create_pr, check=True, dry_run=dry_run) + Shell.check( + cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) + Shell.check(cmd_create_pr, strict=True, dry_run=dry_run, verbose=True) if dry_run: - Shell.run(f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'") - Shell.run( - f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, ) self.version_bump_pr = "dry-run" else: @@ -358,7 +372,7 @@ class ReleaseInfo: cmds.append(f"gh release upload {self.release_tag} {file}") if not dry_run: for cmd in cmds: - Shell.run(cmd, check=True) + Shell.check(cmd, strict=True, verbose=True) self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" else: print("Dry-run, would run commands:") @@ -424,7 +438,7 @@ class PackageDownloader: self.macos_package_files = ["clickhouse-macos", "clickhouse-macos-aarch64"] self.file_to_type = {} - Shell.run(f"mkdir -p {self.LOCAL_DIR}") + Shell.check(f"mkdir -p {self.LOCAL_DIR}") for package_type in self.PACKAGE_TYPES: for package in self.package_names: @@ -474,7 +488,7 @@ class PackageDownloader: return res def run(self): - Shell.run(f"rm -rf {self.LOCAL_DIR}/*") + Shell.check(f"rm -rf {self.LOCAL_DIR}/*") for package_file in ( self.deb_package_files + self.rpm_package_files + self.tgz_package_files ): @@ -549,33 +563,33 @@ class PackageDownloader: @contextmanager def checkout(ref: str) -> Iterator[None]: - orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + orig_ref = Shell.get_output_or_raise(f"{GIT_PREFIX} symbolic-ref --short HEAD") rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref if ref not in (orig_ref,): - Shell.run(f"{GIT_PREFIX} checkout {ref}") + Shell.check(f"{GIT_PREFIX} checkout {ref}", strict=True, verbose=True) try: yield except (Exception, KeyboardInterrupt) as e: print(f"ERROR: Exception [{e}]") - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) raise - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) @contextmanager def checkout_new(ref: str) -> Iterator[None]: - orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + orig_ref = Shell.get_output_or_raise(f"{GIT_PREFIX} symbolic-ref --short HEAD") rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref - Shell.run(f"{GIT_PREFIX} checkout -b {ref}", check=True) + Shell.check(f"{GIT_PREFIX} checkout -b {ref}", strict=True, verbose=True) try: yield except (Exception, KeyboardInterrupt) as e: print(f"ERROR: Exception [{e}]") - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) raise - Shell.run(rollback_cmd) + Shell.check(rollback_cmd, verbose=True) def parse_args() -> argparse.Namespace: diff --git a/tests/ci/docker_images_helper.py b/tests/ci/docker_images_helper.py index e6869852c4e..f0323145cfa 100644 --- a/tests/ci/docker_images_helper.py +++ b/tests/ci/docker_images_helper.py @@ -19,11 +19,11 @@ def docker_login(relogin: bool = True) -> None: if relogin or not Shell.check( "docker system info | grep --quiet -E 'Username|Registry'" ): - Shell.run( # pylint: disable=unexpected-keyword-arg + Shell.check( # pylint: disable=unexpected-keyword-arg "docker login --username 'robotclickhouse' --password-stdin", - input=get_parameter_from_ssm("dockerhub_robot_password"), + strict=True, + stdin_str=get_parameter_from_ssm("dockerhub_robot_password"), encoding="utf-8", - check=True, ) @@ -42,7 +42,7 @@ class DockerImage: def pull_image(image: DockerImage) -> DockerImage: try: logging.info("Pulling image %s - start", image) - Shell.run(f"docker pull {image}", check=True) + Shell.check(f"docker pull {image}", strict=True) logging.info("Pulling image %s - done", image) except Exception as ex: logging.info("Got exception pulling docker %s", ex) From a6d0b7afbb8299eb8cf056368e93267ef51359ba Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 18:05:19 +0200 Subject: [PATCH 0624/1170] recovery option --- .github/workflows/create_release.yml | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index c3126abe461..424dfe60be4 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -16,6 +16,11 @@ concurrency: options: - patch - new + only-repo: + description: 'Run only repos updates including docker (recovery)' + required: false + default: false + type: boolean dry-run: description: 'Dry run' required: false @@ -54,11 +59,12 @@ jobs: run: | python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push Git Tag for the Release + if: ${{ ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} + if: ${{ inputs.type == 'new' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} @@ -67,7 +73,7 @@ jobs: # run: | # python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" @@ -87,7 +93,7 @@ jobs: python3 ./utils/security-generator/generate_security.py > SECURITY.md git diff HEAD - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + if: ${{ inputs.type == 'patch' && ! inputs.dry-run && ! inputs.only-repo }} uses: peter-evans/create-pull-request@v6 with: author: "robot-clickhouse " @@ -105,7 +111,7 @@ jobs: ### Changelog category (leave one): - Not for changelog (changelog entry is not required) - name: Complete previous steps and Restore git state - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-completed From dab5eb9c24cc2f43a0ad8ee65ecac613896cff10 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Aug 2024 16:16:34 +0000 Subject: [PATCH 0625/1170] Fix INTERPOLATE by constant. Fix other tests. --- src/Analyzer/InterpolateNode.cpp | 2 +- src/Analyzer/InterpolateNode.h | 2 ++ src/Analyzer/Resolve/QueryAnalyzer.cpp | 13 ++++------- src/Planner/CollectTableExpressionData.cpp | 2 +- src/Planner/Planner.cpp | 22 +++++++++++++++++++ src/Planner/PlannerExpressionAnalysis.cpp | 3 +++ src/Processors/QueryPlan/FillingStep.cpp | 14 +++++++++++- ..._no_aggregates_and_constant_keys.reference | 4 ++-- ...15_analyzer_materialized_constants_bug.sql | 2 +- 9 files changed, 49 insertions(+), 15 deletions(-) diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 97dc79f565b..17c734cf386 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -24,7 +24,7 @@ void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_st { buffer << std::string(indent, ' ') << "INTERPOLATE id: " << format_state.getNodeId(this); - buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << expression_name << " \n"; getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); buffer << '\n' << std::string(indent + 2, ' ') << "INTERPOLATE_EXPRESSION\n"; diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index ec493ed8bdd..eb3d64d7170 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -50,6 +50,8 @@ public: return QueryTreeNodeType::INTERPOLATE; } + const std::string & getExpressionName() const { return expression_name; } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; protected: diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..e973bd8fb34 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -64,6 +64,8 @@ #include #include +#include + #include namespace ProfileEvents @@ -4122,11 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo { auto & interpolate_node_typed = interpolate_node->as(); - auto * column_to_interpolate = interpolate_node_typed.getExpression()->as(); - if (!column_to_interpolate) - throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found", - interpolate_node_typed.getExpression()->formatASTForErrorMessage()); - auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName(); + auto column_to_interpolate_name = interpolate_node_typed.getExpressionName(); resolveExpressionNode(interpolate_node_typed.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -4135,14 +4133,11 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo auto & interpolation_to_resolve = interpolate_node_typed.getInterpolateExpression(); IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/); - auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node_typed.getExpression()); + auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node); if (is_column_constant) interpolate_scope.expression_argument_name_to_node.emplace(column_to_interpolate_name, fake_column_node); resolveExpressionNode(interpolation_to_resolve, interpolate_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - if (is_column_constant) - interpolation_to_resolve = interpolation_to_resolve->cloneAndReplace(fake_column_node, interpolate_node_typed.getExpression()); } } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 2fe62aa9be0..c48813a4ed4 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -46,7 +46,7 @@ public: auto column_source_node = column_node->getColumnSource(); auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::LAMBDA) + if (column_source_node_type == QueryTreeNodeType::LAMBDA || column_source_node_type == QueryTreeNodeType::INTERPOLATE) return; /// JOIN using expression diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 968642dc9de..b837d9428a1 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -744,6 +744,8 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, } else { + ActionsDAG rename_dag; + for (auto & interpolate_node : interpolate_list_nodes) { auto & interpolate_node_typed = interpolate_node->as(); @@ -772,8 +774,28 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name); interpolate_actions_dag.getOutputs().push_back(alias_node); + + /// Here we fix INTERPOLATE by constant expression. + /// Example from 02336_sort_optimization_with_fill: + /// + /// SELECT 5 AS x, 'Hello' AS s ORDER BY x WITH FILL FROM 1 TO 10 INTERPOLATE (s AS s||'A') + /// + /// For this query, INTERPOLATE_EXPRESSION would be : s AS concat(s, 'A'), + /// so that interpolate_actions_dag would have INPUT `s`. + /// + /// However, INPUT `s` does not exist. Instead, we have a constant with execution name 'Hello'_String. + /// To fix this, we prepend a rename : 'Hello'_String -> s + if (const auto * constant_node = interpolate_node_typed.getExpression()->as()) + { + const auto * node = &rename_dag.addInput(alias_node->result_name, alias_node->result_type); + node = &rename_dag.addAlias(*node, interpolate_node_typed.getExpressionName()); + rename_dag.getOutputs().push_back(node); + } } + if (!rename_dag.getOutputs().empty()) + interpolate_actions_dag = ActionsDAG::merge(std::move(rename_dag), std::move(interpolate_actions_dag)); + interpolate_actions_dag.removeUnusedActions(); } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 2b67c96d843..ed3f78193ee 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -462,6 +462,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); + if (interpolate_node_typed.getExpression()->getNodeType() == QueryTreeNodeType::CONSTANT) + continue; + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 81622389ada..8687886447a 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include namespace DB @@ -58,14 +59,25 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build void FillingStep::describeActions(FormatSettings & settings) const { - settings.out << String(settings.offset, ' '); + String prefix(settings.offset, settings.indent_char); + settings.out << prefix; dumpSortDescription(sort_description, settings.out); settings.out << '\n'; + if (interpolate_description) + { + auto expression = std::make_shared(interpolate_description->actions.clone()); + expression->describeActions(settings.out, prefix); + } } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description)); + if (interpolate_description) + { + auto expression = std::make_shared(interpolate_description->actions.clone()); + map.add("Expression", expression->toTree()); + } } void FillingStep::updateOutputStream() diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index 63b8a9d14fc..fc77ed8a241 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -8,13 +8,13 @@ 40 41 -0 +41 2 42 2 42 43 -0 +43 11 11 diff --git a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql index f9ec28d09d8..b2fd69d75d0 100644 --- a/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql +++ b/tests/queries/0_stateless/03215_analyzer_materialized_constants_bug.sql @@ -20,7 +20,7 @@ WITH ( SELECT coalesce(materialize(toLowCardinality(toNullable(1))), 10, NULL), max(v) -FROM remote('127.0.0.{1,2}', default, test__fuzz_21) +FROM remote('127.0.0.{1,2}', currentDatabase(), test__fuzz_21) GROUP BY coalesce(NULL), coalesce(1, 10, 10, materialize(NULL)); From 4e9761acf93a58a93186f59d3ad083fd438329dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 18:59:17 +0200 Subject: [PATCH 0626/1170] Don't run ASAN unit tests under gdb --- docker/test/unit/Dockerfile | 2 +- docker/test/unit/run.sh | 21 ++++++++++++++++++++- tests/ci/unit_tests_check.py | 5 ++++- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index af44dc930b2..5f907d94d39 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -4,4 +4,4 @@ ARG FROM_TAG=latest FROM clickhouse/test-base:$FROM_TAG COPY run.sh / -CMD ["/bin/bash", "/run.sh"] +ENTRYPOINT ["/run.sh"] diff --git a/docker/test/unit/run.sh b/docker/test/unit/run.sh index 7323c384d9c..ba11f568218 100644 --- a/docker/test/unit/run.sh +++ b/docker/test/unit/run.sh @@ -2,4 +2,23 @@ set -x -timeout 40m gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms --gtest_output='json:test_output/test_result.json' | tee test_output/test_result.txt +if [ "$#" -ne 1 ]; then + echo "Expected exactly one argument" + exit 1 +fi + +if [ "$1" = "GDB" ]; +then + timeout 40m \ + gdb -q -ex "set print inferior-events off" -ex "set confirm off" -ex "set print thread-events off" -ex run -ex bt -ex quit --args \ + ./unit_tests_dbms --gtest_output='json:test_output/test_result.json' \ + | tee test_output/test_result.txt +elif [ "$1" = "NO_GDB" ]; +then + timeout 40m \ + ./unit_tests_dbms --gtest_output='json:test_output/test_result.json' \ + | tee test_output/test_result.txt +else + echo "Unknown argument: $1" + exit 1 +fi diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index b66a4312657..716625d7077 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -174,10 +174,13 @@ def main(): test_output = temp_path / "test_output" test_output.mkdir(parents=True, exist_ok=True) + # Don't run ASAN under gdb since that breaks leak detection + gdb_enabled = "NO_GDB" if "asan" in check_name else "GDB" + run_command = ( f"docker run --cap-add=SYS_PTRACE --volume={tests_binary}:/unit_tests_dbms " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--volume={test_output}:/test_output {docker_image}" + f"--volume={test_output}:/test_output {docker_image} ${gdb_enabled}" ) run_log_path = test_output / "run.log" From d683fb05a009ed3f58c0e11fc329c3783f934369 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Aug 2024 19:05:30 +0200 Subject: [PATCH 0627/1170] Fix --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 ++++++--- src/Interpreters/Cache/FileCache.cpp | 35 ++++++++++++------- src/Interpreters/Cache/FileCache.h | 4 ++- src/Interpreters/Cache/FileSegment.cpp | 9 ++++- src/Interpreters/Cache/FileSegment.h | 4 ++- tests/config/config.d/storage_conf.xml | 3 +- 6 files changed, 51 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index c928d25c7b8..b471f3fc58f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -135,8 +135,11 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet( + cache_key, file_offset_of_buffer_end, size, file_size.value(), + create_settings, settings.filesystem_cache_segments_batch_size, user); } + return !file_segments->empty(); } @@ -158,8 +161,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current offset: {}", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + "Having {} file segments to read: {}, current read range: [{}, {})", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); initialized = true; } @@ -1043,6 +1046,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; + + LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", + remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); + size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1055,8 +1062,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {})", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); + fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); } swap(*implementation_buffer); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index a88c0de2cfe..0a03f5dcc7d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -316,14 +316,14 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: return result; } -std::vector FileCache::splitRange(size_t offset, size_t size) +std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { assert(size > 0); std::vector ranges; size_t current_pos = offset; size_t end_pos_non_included = offset + size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -343,17 +343,20 @@ FileSegments FileCache::splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + /// We take `size` as a soft limit and `aligned_size` as a hard limit. auto current_pos = offset; auto end_pos_non_included = offset + size; size_t current_file_segment_size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -369,6 +372,8 @@ FileSegments FileCache::splitRangeIntoFileSegments( current_pos += current_file_segment_size; } + chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), + fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); return file_segments; } @@ -376,6 +381,7 @@ void FileCache::fillHolesWithEmptyFileSegments( LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & create_settings) @@ -442,7 +448,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, hole_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -479,7 +485,7 @@ void FileCache::fillHolesWithEmptyFileSegments( chassert(!file_segments_limit || file_segments.size() < file_segments_limit); - if (current_pos <= range.right) + if (current_pos <= non_aligned_right_offset) { /// ________] -- requested range /// _____] @@ -487,6 +493,7 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; + auto non_aligned_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { @@ -497,7 +504,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, non_aligned_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -542,7 +549,7 @@ FileSegmentsHolderPtr FileCache::set( else { file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -659,9 +666,13 @@ FileCache::getOrSet( } } + chassert(range.left >= aligned_offset); + if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, create_settings); + file_segments = splitRangeIntoFileSegments( + *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), + FileSegment::State::EMPTY, file_segments_limit, create_settings); } else { @@ -669,9 +680,9 @@ FileCache::getOrSet( chassert(file_segments.back()->range().left <= range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(offset)) + if (!file_segments.front()->range().contains(range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", @@ -713,7 +724,7 @@ FileSegmentsHolderPtr FileCache::get( } fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 527fd9d5edf..3f7eec73b56 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -263,7 +263,7 @@ private: /// Split range into subranges by max_file_segment_size, /// each subrange size must be less or equal to max_file_segment_size. - std::vector splitRange(size_t offset, size_t size); + std::vector splitRange(size_t offset, size_t size, size_t aligned_size); /// Split range into subranges by max_file_segment_size (same as in splitRange()) /// and create a new file segment for each subrange. @@ -273,6 +273,7 @@ private: LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); @@ -281,6 +282,7 @@ private: LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 1664a91b694..c46fb978ae4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1008,7 +1008,12 @@ FileSegment & FileSegmentsHolder::add(FileSegmentPtr && file_segment) return *file_segments.back(); } -String FileSegmentsHolder::toString() +String FileSegmentsHolder::toString(bool with_state) +{ + return DB::toString(file_segments, with_state); +} + +String toString(const FileSegments & file_segments, bool with_state) { String ranges; for (const auto & file_segment : file_segments) @@ -1018,6 +1023,8 @@ String FileSegmentsHolder::toString() ranges += file_segment->range().toString(); if (file_segment->isUnbound()) ranges += "(unbound)"; + if (with_state) + ranges += "(" + FileSegment::stateToString(file_segment->state()) + ")"; } return ranges; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index d6b37b60dc1..25ffb880b45 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -291,7 +291,7 @@ struct FileSegmentsHolder : private boost::noncopyable size_t size() const { return file_segments.size(); } - String toString(); + String toString(bool with_state = false); void popFront() { completeAndPopFrontImpl(); } @@ -317,4 +317,6 @@ private: using FileSegmentsHolderPtr = std::unique_ptr; +String toString(const FileSegments & file_segments, bool with_state = false); + } diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 7a9b579c00a..4daa64b520d 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,8 @@ cache s3_disk s3_cache/ - 104857600 + 100Mi + 5Mi 1 100 LRU From 1e8d0d4a5e8d83a1d123a4b5b6c5a91b41caac1c Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 31 Jul 2024 06:09:14 +0000 Subject: [PATCH 0628/1170] disable parallel run for network_receive_time_metric_insert If run in parallel, several tests may affect the value of the `NetworkReceiveElapsedMicroseconds` profile event. This may contribute to test flakiness. --- .../0_stateless/01923_network_receive_time_metric_insert.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index 97835d97965..77b909ed89e 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel # Tag no-fasttest: needs pv +# Tag no-parallel: reads from a system table CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c5a8653daf7dc9cb1031c2ac4b2be3623117848d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 19:27:12 +0200 Subject: [PATCH 0629/1170] Playing with Docker and the CI --- docker/test/unit/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index 5f907d94d39..9f4b86aa0ca 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -4,4 +4,5 @@ ARG FROM_TAG=latest FROM clickhouse/test-base:$FROM_TAG COPY run.sh / +RUN chmod +x run.sh ENTRYPOINT ["/run.sh"] From 4709222dd1f3a37c5f97e638526c21ade6b5218f Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 31 Jul 2024 23:16:43 +0000 Subject: [PATCH 0630/1170] print debug info if the test fails --- ...1923_network_receive_time_metric_insert.sh | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index 77b909ed89e..adf4fd96a00 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -13,9 +13,23 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) seq 1 1000 | pv --quiet --rate-limit 400 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" # We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client. -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; - WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS time - SELECT time >= 1000000 ? 1 : time FROM system.query_log - WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" +result=$(${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS; + WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS elapsed_us + SELECT elapsed_us FROM system.query_log + WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 'QueryFinish' + ORDER BY event_time DESC LIMIT 1;") + +elapsed_us=$(echo $result | sed 's/ .*//') + +min_elapsed_us=1000000 +if [[ "$elapsed_us" -ge "$min_elapsed_us" ]]; then + echo 1 +else + # Print debug info + ${CLICKHOUSE_CLIENT} --query " + WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS elapsed_us + SELECT query_start_time_microseconds, event_time_microseconds, query_duration_ms, elapsed_us, query FROM system.query_log + WHERE current_database = currentDatabase() and event_date >= yesterday() AND type = 'QueryFinish' ORDER BY query_start_time;" +fi ${CLICKHOUSE_CLIENT} --query "DROP TABLE t" From 35b6112b7bb06f18cd5b07860ec1b9c6ce38014f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Aug 2024 19:50:13 +0200 Subject: [PATCH 0631/1170] Try to stop on leaks --- docker/test/base/Dockerfile | 2 ++ docker/test/stateless/run.sh | 2 +- tests/queries/shell_config.sh | 3 --- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index a81826ed6b5..38c4df459ae 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -28,12 +28,14 @@ RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 abort_on_error=1 history_ RUN echo "UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768'" >> /etc/environment RUN echo "MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768'" >> /etc/environment RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt max_allocation_size_mb=32768'" >> /etc/environment +RUN echo "ASAN_OPTIONS='halt_on_error=1 abort_on_error=1'" >> /etc/environment # Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") # (but w/o verbosity for TSAN, otherwise test.reference will not match) ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1 max_allocation_size_mb=32768' ENV UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768' ENV LSAN_OPTIONS='max_allocation_size_mb=32768' +ENV ASAN_OPTIONS='halt_on_error=1 abort_on_error=1' # for external_symbolizer_path RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index f9f96c76d59..c359d8a1847 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -164,7 +164,7 @@ do done setup_logs_replication -attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +attach_gdb_to_clickhouse function fn_exists() { declare -F "$1" > /dev/null; diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index f7017958635..9e5bf75d335 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -1,9 +1,6 @@ #!/usr/bin/env bash # shellcheck disable=SC2120 -# Don't check for ODR violation, since we may test shared build with ASAN -export ASAN_OPTIONS=detect_odr_violation=0 - # If ClickHouse was built with coverage - dump the coverage information at exit # (in other cases this environment variable has no effect) export CLICKHOUSE_WRITE_COVERAGE="coverage" From fc72742e37ae78f47b3e55a969e088f5c372ee36 Mon Sep 17 00:00:00 2001 From: sakulali Date: Fri, 2 Aug 2024 02:00:20 +0800 Subject: [PATCH 0632/1170] ping CI From 30e0c1a1b8479e9b6be0701ba21e6050906a7e43 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 18:46:09 +0200 Subject: [PATCH 0633/1170] try less mem for geesefs --- .github/workflows/create_release.yml | 21 +++++++++++---------- tests/ci/artifactory.py | 8 +++++--- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 424dfe60be4..3c61fa4cfe1 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -17,7 +17,7 @@ concurrency: - patch - new only-repo: - description: 'Run only repos updates including docker (recovery)' + description: 'Run only repos updates including docker (repo-recovery, tests)' required: false default: false type: boolean @@ -68,10 +68,11 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} -# - name: Bump CH Version and Update Contributors' List -# shell: bash -# run: | -# python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + if: ${{ ! inputs.only-repo }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash @@ -117,11 +118,11 @@ jobs: python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" -# - name: Create GH Release -# shell: bash -# if: ${{ inputs.type == 'patch' }} -# run: | -# python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} + - name: Create GH Release + if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 71deaccf917..8bba7bca30e 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -67,8 +67,8 @@ class R2MountPoint: f" --shared-config=/home/ubuntu/.r2_auth_test " ) if self.DEBUG: - self.aux_mount_options += " --debug_s3 --debug_fuse " - self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=2050 --gc-interval=100 --max-flushers=5 --max-parallel-parts=1 --max-parallel-copy=2 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" + self.aux_mount_options += " --debug_s3 " + self.mount_cmd = f"geesefs --endpoint={self.API_ENDPOINT} --cheap --memory-limit=1000 --gc-interval=100 --max-flushers=10 --max-parallel-parts=1 --max-parallel-copy=10 --log-file={self.LOG_FILE} {self.aux_mount_options} {self.bucket_name} {self.MOUNT_POINT}" else: assert False @@ -207,8 +207,10 @@ class RpmArtifactory: for package in paths: _copy_if_not_exists(Path(package), dest_dir) + # switching between different fuse providers invalidates --update option (apparently some fuse(s) can mess around with mtime) + # add --skip-stat to skip mtime check commands = ( - f"createrepo_c --local-sqlite --workers=2 --update --verbose {dest_dir}", + f"createrepo_c --local-sqlite --workers=2 --update --skip-stat --verbose {dest_dir}", f"gpg --sign-with {self._SIGN_KEY} --detach-sign --batch --yes --armor {dest_dir / 'repodata' / 'repomd.xml'}", ) print(f"Exporting RPM packages into [{codename}]") From e034558f74a1cd46bb8fbdfac3b7dc6d25165f4e Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 1 Aug 2024 20:51:36 +0200 Subject: [PATCH 0634/1170] add automerge prs step --- .github/workflows/create_release.yml | 4 +++ pyproject.toml | 1 + tests/ci/ci_utils.py | 2 +- tests/ci/create_release.py | 47 +++++++++++++++++++++++++++- tests/ci/release.py | 1 + 5 files changed, 53 insertions(+), 2 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 3c61fa4cfe1..e27db1b09a4 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -171,6 +171,10 @@ jobs: export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed + - name: Update release info. Merge created PRs + shell: bash + run: | + python3 ./tests/ci/create_release.py --merge-prs ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Set current Release progress to Completed with OK shell: bash run: | diff --git a/pyproject.toml b/pyproject.toml index 9bbeac3ddae..4268901e7f9 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -40,6 +40,7 @@ disable = ''' global-statement, f-string-without-interpolation, consider-using-with, + use-maxsplit-arg, ''' [tool.pylint.SIMILARITIES] diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index cd21554788c..4f696a2c55a 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -228,7 +228,7 @@ class Shell: ): if dry_run: print(f"Dry-ryn. Would run command [{command}]") - return 0 + return True if verbose: print(f"Run command [{command}]") proc = subprocess.Popen( diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 0d505d6ccc7..c407a74fbf0 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -43,6 +43,7 @@ class ReleaseProgress: TEST_TGZ = "test TGZ packages" TEST_RPM = "test RPM packages" TEST_DEB = "test DEB packages" + MERGE_CREATED_PRS = "merge created PRs" COMPLETED = "completed" @@ -101,6 +102,7 @@ class ReleaseInfo: previous_release_sha: str changelog_pr: str = "" version_bump_pr: str = "" + prs_merged: bool = False release_url: str = "" debian_command: str = "" rpm_command: str = "" @@ -380,6 +382,38 @@ class ReleaseInfo: self.release_url = f"dry-run" self.dump() + def merge_prs(self, dry_run: bool) -> None: + repo = CI.Envs.GITHUB_REPOSITORY + assert self.version_bump_pr + if dry_run: + version_bump_pr_num = 12345 + else: + version_bump_pr_num = int(self.version_bump_pr.split("/")[-1]) + print("Merging Version bump PR") + res_1 = Shell.check( + f"gh pr merge {version_bump_pr_num} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + + res_2 = True + if not self.release_tag.endswith("-new"): + assert self.changelog_pr + print("Merging ChangeLog PR") + if dry_run: + changelog_pr_num = 23456 + else: + changelog_pr_num = int(self.changelog_pr.split("/")[-1]) + res_2 = Shell.check( + f"gh pr merge {changelog_pr_num} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + else: + assert not self.changelog_pr + + self.prs_merged = res_1 and res_2 + class RepoTypes: RPM = "rpm" @@ -627,6 +661,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Create GH Release object and attach all packages", ) + parser.add_argument( + "--merge-prs", + action="store_true", + help="Merge PRs with version, changelog updates", + ) parser.add_argument( "--post-status", action="store_true", @@ -732,7 +771,6 @@ if __name__ == "__main__": if args.post_status: release_info = ReleaseInfo.from_file() - release_info.update_release_info(dry_run=args.dry_run) if release_info.is_new_release_branch(): title = "New release branch" else: @@ -766,6 +804,13 @@ if __name__ == "__main__": ri.progress_description = ReleaseProgressDescription.OK ri.dump() + if args.merge_prs: + with ReleaseContextManager( + release_progress=ReleaseProgress.MERGE_CREATED_PRS + ) as release_info: + release_info.update_release_info(dry_run=args.dry_run) + release_info.merge_prs(dry_run=args.dry_run) + # tear down ssh if _ssh_agent and _key_pub: _ssh_agent.remove(_key_pub) diff --git a/tests/ci/release.py b/tests/ci/release.py index 2de20d00a00..b26d6205f3b 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -689,4 +689,5 @@ def main(): if __name__ == "__main__": + assert False, "Script Deprecated, ask ci team for help" main() From 67b11300e45f6e24c3515a978d23a9bc998a666e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 1 Aug 2024 21:05:49 +0000 Subject: [PATCH 0635/1170] Fix crash in KeyCondition::cloneASTWithInversionPushDown() caused by type change --- src/Storages/MergeTree/KeyCondition.cpp | 82 +++++++++---------- .../03215_key_condition_bug.reference | 1 + .../0_stateless/03215_key_condition_bug.sql | 3 + 3 files changed, 44 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/03215_key_condition_bug.reference create mode 100644 tests/queries/0_stateless/03215_key_condition_bug.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 69bffac9160..eaf9f0af623 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -566,6 +566,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } const ActionsDAG::Node * res = nullptr; + bool handled_inversion = false; switch (node.type) { @@ -582,7 +583,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( /// Re-generate column name for constant. /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. /// DAG from PK does not use it. This breaks matching by column name sometimes. - /// Ideally, we should not compare manes, but DAG subtrees instead. + /// Ideally, we should not compare names, but DAG subtrees instead. name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); else name = node.result_name; @@ -593,9 +594,9 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( case (ActionsDAG::ActionType::ALIAS): { /// Ignore aliases - const auto & alias = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); - to_inverted[&node] = &alias; - return alias; + res = &cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); + handled_inversion = true; + break; } case (ActionsDAG::ActionType::ARRAY_JOIN): { @@ -608,20 +609,10 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( auto name = node.function_base->getName(); if (name == "not") { - const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion); - to_inverted[&node] = &arg; - return arg; + res = &cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion); + handled_inversion = true; } - - if (name == "materialize") - { - /// Ignore materialize - const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); - to_inverted[&node] = &arg; - return arg; - } - - if (name == "indexHint") + else if (name == "indexHint") { ActionsDAG::NodeRawConstPtrs children; if (const auto * adaptor = typeid_cast(node.function_base.get())) @@ -636,12 +627,10 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - const auto & func = inverted_dag.addFunction(node.function_base, children, ""); - to_inverted[&node] = &func; - return func; + res = &inverted_dag.addFunction(node.function_base, children, ""); + handled_inversion = true; } - - if (need_inversion && (name == "and" || name == "or")) + else if (need_inversion && (name == "and" || name == "or")) { ActionsDAG::NodeRawConstPtrs children(node.children); @@ -659,34 +648,43 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( /// We match columns by name, so it is important to fill name correctly. /// So, use empty string to make it automatically. - const auto & func = inverted_dag.addFunction(function_builder, children, ""); - to_inverted[&node] = &func; - return func; + res = &inverted_dag.addFunction(function_builder, children, ""); + handled_inversion = true; } - - ActionsDAG::NodeRawConstPtrs children(node.children); - - for (auto & arg : children) - arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false); - - auto it = inverse_relations.find(name); - if (it != inverse_relations.end()) + else { - const auto & func_name = need_inversion ? it->second : it->first; - auto function_builder = FunctionFactory::instance().get(func_name, context); - const auto & func = inverted_dag.addFunction(function_builder, children, ""); - to_inverted[&node] = &func; - return func; - } + ActionsDAG::NodeRawConstPtrs children(node.children); - res = &inverted_dag.addFunction(node.function_base, children, ""); - chassert(res->result_type == node.result_type); + for (auto & arg : children) + arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false); + + auto it = inverse_relations.find(name); + if (it != inverse_relations.end()) + { + const auto & func_name = need_inversion ? it->second : it->first; + auto function_builder = FunctionFactory::instance().get(func_name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); + handled_inversion = true; + } + else + { + res = &inverted_dag.addFunction(node.function_base, children, ""); + chassert(res->result_type == node.result_type); + } + } } } - if (need_inversion) + if (!handled_inversion && need_inversion) res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, ""); + /// Make sure we don't change any data types (e.g. remove LowCardinality). + /// If it turns out that we actually want to change data types sometimes, it's ok to remove this + /// check *and* replace all `addFunction(node.function_base, ...)` calls above with + /// `addFunction(FunctionFactory::instance().get(name, context), ...)` to re-resolve overloads. + if (!node.result_type->equals(*res->result_type)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", node.result_type->getName(), res->result_type->getName(), node.result_name); + to_inverted[&node] = res; return *res; } diff --git a/tests/queries/0_stateless/03215_key_condition_bug.reference b/tests/queries/0_stateless/03215_key_condition_bug.reference new file mode 100644 index 00000000000..84ab67a85e0 --- /dev/null +++ b/tests/queries/0_stateless/03215_key_condition_bug.reference @@ -0,0 +1 @@ +(0) diff --git a/tests/queries/0_stateless/03215_key_condition_bug.sql b/tests/queries/0_stateless/03215_key_condition_bug.sql new file mode 100644 index 00000000000..ef2113e81f8 --- /dev/null +++ b/tests/queries/0_stateless/03215_key_condition_bug.sql @@ -0,0 +1,3 @@ +CREATE TABLE t (x Int8) ENGINE MergeTree ORDER BY x; +INSERT INTO t VALUES (1); +SELECT arrayJoin([tuple((toNullable(10) * toLowCardinality(20)) < materialize(30))]) AS row FROM t WHERE row.1 = 0; \ No newline at end of file From 2e7a15df89a7a3d28445095cb4392b056c3f19a9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 1 Aug 2024 23:23:06 +0200 Subject: [PATCH 0636/1170] Update setup_export_logs.sh --- docker/test/base/setup_export_logs.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index db141bcc55e..e544397dd0c 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -215,10 +215,9 @@ function setup_logs_replication function stop_logs_replication { echo "Detach all logs replication" - timeout --preserve-status --signal TERM --kill-after 10m 20m \ - clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | { + clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | { tee /dev/stderr } | { - xargs -n1 -r -i clickhouse-client --query "drop table {}" + timeout --preserve-status --signal TERM --kill-after 5m 15m xargs -n1 -r -i clickhouse-client --query "drop table {}" } } From eac2c9fc3d8a88c1033e0f23e048421ecf4db850 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0637/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 69bd306a445a6bc8a55be14bb0080864921f8b69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 00:48:25 +0200 Subject: [PATCH 0638/1170] Fix race condition in system.processes and Settings --- src/Backups/RestoreCoordinationRemote.cpp | 2 +- src/Backups/RestoreCoordinationRemote.h | 2 -- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 2 +- src/Interpreters/ProcessList.cpp | 2 +- 5 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 44214d00be5..0a69bc0eafb 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -323,7 +323,7 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic return false; bool result = false; - std::string path = zookeeper_path +"/stage"; + std::string path = zookeeper_path + "/stage"; auto holder = with_retries.createRetriesControlHolder("createRootNodes"); holder.retries_ctl.retryLoop( diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 9c299865cfa..a3d57e9a4d0 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -61,8 +61,6 @@ private: void createRootNodes(); void removeAllNodes(); - class ReplicatedDatabasesMetadataSync; - /// get_zookeeper will provide a zookeeper client without any fault injection const zkutil::GetZooKeeper get_zookeeper; const String root_zookeeper_path; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index ca30ee6db15..3fb6d30fcb8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -44,7 +44,7 @@ namespace ErrorCodes DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_) - : DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_) + : DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { } diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 4347649117d..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 in memory only cache. + * 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/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 271e23a7288..6cb50b310ad 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -657,7 +657,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even { if (auto ctx = context.lock()) { - res.query_settings = std::make_shared(ctx->getSettingsRef()); + res.query_settings = std::make_shared(ctx->getSettingsCopy()); res.current_database = ctx->getCurrentDatabase(); } } From a80c7c080cdbb3bec4662501686133298b7a4a2d Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 1 Aug 2024 18:23:11 -0600 Subject: [PATCH 0639/1170] 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 572831f865d66e046d3e507d214e0f5aeae49ad4 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 2 Aug 2024 00:09:26 +0000 Subject: [PATCH 0640/1170] async_insert_race_long flakiness fixes 1. Make the test truly asynchronous. The setting `--async_insert_max_data_size 1` leads to data being flushed synchronously for all inserts in this test. This triggers part creation and extra resource consumption. 2. Do not run the `--wait_for_async_insert` query as a background process with a fixed (50ms) sleep time. If the actual execution time is longer than the anticipated delay time, it may lead to excessive process creation. --- tests/queries/0_stateless/02481_async_insert_race_long.sh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_race_long.sh b/tests/queries/0_stateless/02481_async_insert_race_long.sh index b0088017d32..91e6c4960e0 100755 --- a/tests/queries/0_stateless/02481_async_insert_race_long.sh +++ b/tests/queries/0_stateless/02481_async_insert_race_long.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -export MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert_busy_timeout_ms 10 --async_insert_max_data_size 1 --async_insert 1" +export MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert_busy_timeout_min_ms 50 --async_insert_busy_timeout_max_ms 50 --async_insert 1" function insert1() { @@ -29,11 +29,8 @@ function insert3() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" & - sleep 0.05 + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" done - - wait } function select1() From c0d298781e72aaa1f34fc7fd610f5dbcaa9acf2d Mon Sep 17 00:00:00 2001 From: morning-color Date: Fri, 2 Aug 2024 10:36:15 +0800 Subject: [PATCH 0641/1170] 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 0642/1170] 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 0772ed7f6ac92bb2e016e5db00e85deafeecb127 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Fri, 2 Aug 2024 04:02:43 +0000 Subject: [PATCH 0643/1170] Code style feedback incorporated --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 10 +++++----- ...3209_parameterized_view_with_non_literal_params.sql | 2 ++ 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 6113a38d463..bffdba2f58a 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4549,11 +4549,11 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// Serialize the constant value using datatype specific /// interfaces to match the deserialization in ReplaceQueryParametersVistor. WriteBufferFromOwnString buf; - auto constval = constant->getValue(); - auto realtype = constant->getResultType(); - auto tempcol = realtype->createColumn(); - tempcol->insert(constval); - realtype->getDefaultSerialization()->serializeTextEscaped(*tempcol, 0, buf, {}); + const auto & value = constant->getValue(); + auto real_type = constant->getResultType(); + auto temporary_column = real_type->createColumn(); + temporary_column->insert(value); + real_type->getDefaultSerialization()->serializeTextEscaped(*temporary_column, 0, buf, {}); view_params[identifier_node->getIdentifier().getFullName()] = buf.str(); } } diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql index 55795c7a785..f2c61e5cb1d 100644 --- a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -90,8 +90,10 @@ select id from ipv4_pv(ipv4param=(select ipaddr from ipv4_table_pv where id=3)); drop view date_pv; drop view date_pv2; +drop view date32_pv; drop view uuid_pv; drop view ipv4_pv; drop table date_table_pv; +drop table date32_table_pv; drop table uuid_table_pv; drop table ipv4_table_pv; From d6da86dad282e6ad176b115d4344944daa8b9756 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 26 Jun 2024 01:27:47 +0000 Subject: [PATCH 0644/1170] Store plain_rewritable metadata in a separate layout --- .../CommonPathPrefixKeyGenerator.cpp | 6 +- .../MetadataStorageFromPlainObjectStorage.cpp | 36 ++++-- .../MetadataStorageFromPlainObjectStorage.h | 16 ++- ...torageFromPlainObjectStorageOperations.cpp | 71 +++++++---- ...aStorageFromPlainObjectStorageOperations.h | 14 ++- ...torageFromPlainRewritableObjectStorage.cpp | 119 +++++++++++++++--- ...aStorageFromPlainRewritableObjectStorage.h | 15 ++- .../test_s3_plain_rewritable/test.py | 13 ++ 8 files changed, 225 insertions(+), 65 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index e321c8a3c5a..2a06d56e5c7 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -19,11 +19,11 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); - auto key = std::filesystem::path(object_key_prefix.empty() ? storage_key_prefix : object_key_prefix); + auto key = std::filesystem::path(object_key_prefix.empty() ? std::string() : object_key_prefix); /// The longest prefix is the same as path, meaning that the path is already mapped. if (suffix_parts.empty()) - return ObjectStorageKey::createAsRelative(std::move(key)); + return ObjectStorageKey::createAsRelative(storage_key_prefix, std::move(key)); /// File and top-level directory paths are mapped as is. if (!is_directory || object_key_prefix.empty()) @@ -39,7 +39,7 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo key /= getRandomASCIIString(part_size); } - return ObjectStorageKey::createAsRelative(key); + return ObjectStorageKey::createAsRelative(storage_key_prefix, key); } std::tuple> CommonPathPrefixKeyGenerator::getLongestObjectKeyPrefix(const std::string & path) const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 30111d04d20..3da190c7256 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -7,6 +7,7 @@ #include #include +#include namespace DB { @@ -79,14 +80,16 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co object_storage->listObjects(abs_key, files, 0); - return getDirectChildrenOnDisk(abs_key, files, path); + std::unordered_set directories; + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const { /// Required for MergeTree auto paths = listDirectory(path); - // Prepend path, since iterateDirectory() includes path, unlike listDirectory() + /// Prepend path, since iterateDirectory() includes path, unlike listDirectory() std::for_each(paths.begin(), paths.end(), [&](auto & child) { child = fs::path(path) / child; }); std::vector fs_paths(paths.begin(), paths.end()); return std::make_unique(std::move(fs_paths)); @@ -99,10 +102,13 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std return {StoredObject(object_key.serialize(), path, object_size)}; } -std::vector MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & /* local_path */) const +void MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & /* storage_key_perfix */, + const RelativePathsWithMetadata & remote_paths, + const std::string & /* local_path */, + std::unordered_set & result) const { - std::unordered_set duplicates_filter; for (const auto & elem : remote_paths) { const auto & path = elem->relative_path; @@ -111,11 +117,10 @@ std::vector MetadataStorageFromPlainObjectStorage::getDirectChildre /// string::npos is ok. const auto slash_pos = path.find('/', child_pos); if (slash_pos == std::string::npos) - duplicates_filter.emplace(path.substr(child_pos)); + result.emplace(path.substr(child_pos)); else - duplicates_filter.emplace(path.substr(child_pos, slash_pos - child_pos)); + result.emplace(path.substr(child_pos, slash_pos - child_pos)); } - return std::vector(std::make_move_iterator(duplicates_filter.begin()), std::make_move_iterator(duplicates_filter.end())); } const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getStorageForNonTransactionalReads() const @@ -140,7 +145,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std else { addOperation(std::make_unique( - normalizeDirectoryPath(path), *metadata_storage.getPathMap(), object_storage)); + normalizeDirectoryPath(path), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix())); } } @@ -151,8 +156,13 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std auto normalized_path = normalizeDirectoryPath(path); auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path).serialize(); + chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto op = std::make_unique( - std::move(normalized_path), std::move(key_prefix), *metadata_storage.getPathMap(), object_storage); + std::move(normalized_path), + key_prefix.substr(object_storage->getCommonKeyPrefix().size()), + *metadata_storage.getPathMap(), + object_storage, + metadata_storage.getMetadataKeyPrefix()); addOperation(std::move(op)); } @@ -167,7 +177,11 @@ void MetadataStorageFromPlainObjectStorageTransaction::moveDirectory(const std:: throwNotImplemented(); addOperation(std::make_unique( - normalizeDirectoryPath(path_from), normalizeDirectoryPath(path_to), *metadata_storage.getPathMap(), object_storage)); + normalizeDirectoryPath(path_from), + normalizeDirectoryPath(path_to), + *metadata_storage.getPathMap(), + object_storage, + metadata_storage.getMetadataKeyPrefix())); } void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 66da0f2431e..97c5715a937 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace DB { @@ -78,10 +80,20 @@ public: bool supportsStat() const override { return false; } protected: + /// Get the object storage prefix for storing metadata files. If stored behind a separate endpoint, + /// the metadata keys reflect the layout of the regular files. + virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } + + /// Returns a map of local paths to paths in object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } - virtual std::vector getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const; + /// Retrieves the immediate files and directories within a given directory on a disk. + virtual void getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const; }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 7e4b1f69962..0a6086bd39d 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -20,14 +20,24 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; +ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const std::string & metadata_key_prefix) +{ + auto prefix = std::filesystem::path(metadata_key_prefix) / key_prefix; + return ObjectStorageKey::createAsRelative(prefix.string(), PREFIX_PATH_FILE_NAME); +} } MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_) - : path(std::move(path_)), key_prefix(key_prefix_), path_map(path_map_), object_storage(object_storage_) + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path(std::move(path_)) + , key_prefix(key_prefix_) + , path_map(path_map_) + , object_storage(object_storage_) + , metadata_key_prefix(metadata_key_prefix_) { } @@ -36,13 +46,17 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: if (path_map.contains(path)) return; - LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Creating metadata for directory '{}'", path); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); + LOG_TRACE( + getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), + "Creating metadata for directory '{}' with remote path='{}'", + path, + metadata_object_key.serialize()); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, @@ -66,25 +80,31 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + if (write_finalized) { path_map.erase(path); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); - object_storage->removeObject(StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); + object_storage->removeObject(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } else if (write_created) - object_storage->removeObjectIfExists(StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); + object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_) - : path_from(std::move(path_from_)), path_to(std::move(path_to_)), path_map(path_map_), object_storage(object_storage_) + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path_from(std::move(path_from_)) + , path_to(std::move(path_to_)) + , path_map(path_map_) + , object_storage(object_storage_) + , metadata_key_prefix(metadata_key_prefix_) { } @@ -98,26 +118,26 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo if (path_map.contains(new_path)) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); - auto object_key = ObjectStorageKey::createAsRelative(expected_it->second, PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); - auto object = StoredObject(object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); if (validate_content) { std::string data; - auto read_buf = object_storage->readObject(object); + auto read_buf = object_storage->readObject(metadata_object); readStringUntilEOF(data, *read_buf); if (data != path_from) throw Exception( ErrorCodes::INCORRECT_DATA, "Incorrect data for object key {}, expected {}, got {}", - object_key.serialize(), + metadata_object_key.serialize(), expected_path, data); } auto write_buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /*buf_size*/ DBMS_DEFAULT_BUFFER_SIZE, @@ -156,8 +176,11 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq } MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, ObjectStoragePtr object_storage_) - : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_) + std::filesystem::path && path_, + MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_) + : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } @@ -170,9 +193,9 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); key_prefix = path_it->second; - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); - object_storage->removeObject(object); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + object_storage->removeObject(metadata_object); path_map.erase(path_it); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -189,10 +212,10 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un if (!removed) return; - auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME); - auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( - object, + metadata_object, WriteMode::Rewrite, /* object_attributes */ std::nullopt, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 4b196f787fd..e31e3cbb262 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -16,6 +16,7 @@ private: std::string key_prefix; MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; bool write_created = false; bool write_finalized = false; @@ -26,7 +27,8 @@ public: std::filesystem::path && path_, std::string && key_prefix_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_); + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; void undo(std::unique_lock & metadata_lock) override; @@ -39,6 +41,7 @@ private: std::filesystem::path path_to; MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; bool write_created = false; bool write_finalized = false; @@ -51,7 +54,8 @@ public: std::filesystem::path && path_from_, std::filesystem::path && path_to_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_); + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; @@ -65,13 +69,17 @@ private: MetadataStorageFromPlainObjectStorage::PathMap & path_map; ObjectStoragePtr object_storage; + const std::string metadata_key_prefix; std::string key_prefix; bool removed = false; public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, MetadataStorageFromPlainObjectStorage::PathMap & path_map_, ObjectStoragePtr object_storage_); + std::filesystem::path && path_, + MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + ObjectStoragePtr object_storage_, + const std::string & metadata_key_prefix_); void execute(std::unique_lock & metadata_lock) override; void undo(std::unique_lock & metadata_lock) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 7718fba9c28..f3d00a928e3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -21,8 +22,22 @@ namespace { constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; +constexpr auto METADATA_PATH_TOKEN = "__meta/"; -MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & root, ObjectStoragePtr object_storage) +/// Use a separate layout for metadata iff: +/// 1. The disk endpoint does not contain objects, OR +/// 2. The metadata is already stored behind a separate endpoint. +/// Otherwise, store metadata along with regular data for backward compatibility. +std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) +{ + const auto common_key_prefix = std::filesystem::path(object_storage->getCommonKeyPrefix()); + const auto metadata_key_prefix = std::filesystem::path(common_key_prefix) / METADATA_PATH_TOKEN; + return !object_storage->existsOrHasAnyChild(metadata_key_prefix / "") && object_storage->existsOrHasAnyChild(common_key_prefix / "") + ? common_key_prefix + : metadata_key_prefix; +} + +MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { MetadataStorageFromPlainObjectStorage::PathMap result; @@ -39,16 +54,16 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri LOG_DEBUG(log, "Loading metadata"); size_t num_files = 0; - for (auto iterator = object_storage->iterate(root, 0); iterator->isValid(); iterator->next()) + for (auto iterator = object_storage->iterate(metadata_key_prefix, 0); iterator->isValid(); iterator->next()) { ++num_files; auto file = iterator->current(); String path = file->getPath(); - auto remote_path = std::filesystem::path(path); - if (remote_path.filename() != PREFIX_PATH_FILE_NAME) + auto remote_metadata_path = std::filesystem::path(path); + if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_path, path, &object_storage, &result, &mutex, &log, &settings] + runner([remote_metadata_path, path, &object_storage, &result, &mutex, &log, &settings, &metadata_key_prefix] { setThreadName("PlainRWMetaLoad"); @@ -75,7 +90,10 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri throw; } - chassert(remote_path.has_parent_path()); + chassert(remote_metadata_path.has_parent_path()); + chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); + auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); + auto remote_path = std::filesystem::path(std::move(suffix)); std::pair res; { std::lock_guard lock(mutex); @@ -103,17 +121,17 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri return result; } -std::vector getDirectChildrenOnRewritableDisk( +void getDirectChildrenOnRewritableDisk( const std::string & storage_key, + const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, const MetadataStorageFromPlainObjectStorage::PathMap & local_path_prefixes, - SharedMutex & shared_mutex) + SharedMutex & shared_mutex, + std::unordered_set & result) { using PathMap = MetadataStorageFromPlainObjectStorage::PathMap; - std::unordered_set duplicates_filter; - /// Map remote paths into local subdirectories. std::unordered_map remote_to_local_subdir; @@ -149,22 +167,21 @@ std::vector getDirectChildrenOnRewritableDisk( /// File names. auto filename = path.substr(child_pos); if (!skip_list.contains(filename)) - duplicates_filter.emplace(std::move(filename)); + result.emplace(std::move(filename)); } else { /// Subdirectories. - auto it = remote_to_local_subdir.find(path.substr(0, slash_pos)); + chassert(path.find(storage_key_perfix) == 0); + auto it = remote_to_local_subdir.find(path.substr(storage_key_perfix.size(), slash_pos - storage_key_perfix.size())); /// Mapped subdirectories. if (it != remote_to_local_subdir.end()) - duplicates_filter.emplace(it->second); + result.emplace(it->second); /// The remote subdirectory name is the same as the local subdirectory. else - duplicates_filter.emplace(path.substr(child_pos, slash_pos - child_pos)); + result.emplace(path.substr(child_pos, slash_pos - child_pos)); } } - - return std::vector(std::make_move_iterator(duplicates_filter.begin()), std::make_move_iterator(duplicates_filter.end())); } } @@ -172,7 +189,8 @@ std::vector getDirectChildrenOnRewritableDisk( MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewritableObjectStorage( ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) - , path_map(std::make_shared(loadPathPrefixMap(object_storage->getCommonKeyPrefix(), object_storage))) + , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) + , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) { if (object_storage->isWriteOnce()) throw Exception( @@ -190,10 +208,71 @@ MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewrit CurrentMetrics::sub(metric, path_map->size()); } -std::vector MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const +bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & path) const { - return getDirectChildrenOnRewritableDisk(storage_key, remote_paths, local_path, *getPathMap(), metadata_mutex); + if (MetadataStorageFromPlainObjectStorage::exists(path)) + return true; + + if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + { + auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); + return object_storage->existsOrHasAnyChild(metadata_key); + } + + return false; +} + +bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::string & path) const +{ + if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + { + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path).serialize()) / ""; + chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key + = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); + return object_storage->existsOrHasAnyChild(metadata_key); + } + else + return MetadataStorageFromPlainObjectStorage::isDirectory(path); +} + +std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const +{ + auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + + RelativePathsWithMetadata files; + std::string abs_key = key_prefix; + if (!abs_key.ends_with('/')) + abs_key += '/'; + + object_storage->listObjects(abs_key, files, 0); + + std::unordered_set directories; + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove + /// metadata along with regular files. + if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) + { + chassert(abs_key.starts_with(object_storage->getCommonKeyPrefix())); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / abs_key.substr(object_storage->getCommonKeyPrefix().size()); + RelativePathsWithMetadata metadata_files; + object_storage->listObjects(metadata_key, metadata_files, 0); + getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); + } + + return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); +} + +void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const +{ + getDirectChildrenOnRewritableDisk(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index a5394b9428d..71153cbdc25 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -11,6 +12,7 @@ namespace DB class MetadataStorageFromPlainRewritableObjectStorage final : public MetadataStorageFromPlainObjectStorage { private: + const std::string metadata_key_prefix; std::shared_ptr path_map; public: @@ -18,11 +20,20 @@ public: ~MetadataStorageFromPlainRewritableObjectStorage() override; MetadataStorageType getType() const override { return MetadataStorageType::PlainRewritable; } + bool exists(const std::string & path) const override; + bool isDirectory(const std::string & path) const override; + std::vector listDirectory(const std::string & path) const override; + protected: + std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } - std::vector getDirectChildrenOnDisk( - const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const override; + void getDirectChildrenOnDisk( + const std::string & storage_key, + const std::string & storage_key_perfix, + const RelativePathsWithMetadata & remote_paths, + const std::string & local_path, + std::unordered_set & result) const override; }; } diff --git a/tests/integration/test_s3_plain_rewritable/test.py b/tests/integration/test_s3_plain_rewritable/test.py index 4b1aaafc814..020e170eb48 100644 --- a/tests/integration/test_s3_plain_rewritable/test.py +++ b/tests/integration/test_s3_plain_rewritable/test.py @@ -139,6 +139,19 @@ def test(storage_policy): == insert_values_arr[i] ) + metadata_it = cluster.minio_client.list_objects( + cluster.minio_bucket, "data/", recursive=True + ) + metadata_count = 0 + for obj in list(metadata_it): + if "/__meta/" in obj.object_name: + assert obj.object_name.endswith("/prefix.path") + metadata_count += 1 + else: + assert not obj.object_name.endswith("/prefix.path") + + assert metadata_count > 0 + for i in range(NUM_WORKERS): node = cluster.instances[f"node{i + 1}"] node.query("DROP TABLE IF EXISTS test SYNC") From 98ad45ba960de4cc29ee794e2eeccf9fe6f8e0a8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 00:58:20 +0000 Subject: [PATCH 0645/1170] Add prefix_path parameter to object key generator --- src/Common/ObjectStorageKeyGenerator.cpp | 11 ++++++++--- src/Common/ObjectStorageKeyGenerator.h | 3 ++- .../AzureBlobStorage/AzureObjectStorage.cpp | 4 +++- .../AzureBlobStorage/AzureObjectStorage.h | 2 +- .../Cached/CachedObjectStorage.cpp | 10 ++++++---- .../ObjectStorages/Cached/CachedObjectStorage.h | 5 +++-- .../CommonPathPrefixKeyGenerator.cpp | 3 ++- .../CommonPathPrefixKeyGenerator.h | 3 ++- .../DiskObjectStorageTransaction.cpp | 6 +++--- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 ++++-- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 5 +++-- .../ObjectStorages/Local/LocalObjectStorage.cpp | 14 ++++++++------ .../ObjectStorages/Local/LocalObjectStorage.h | 2 +- .../MetadataStorageFromPlainObjectStorage.cpp | 14 +++++++------- ...aStorageFromPlainRewritableObjectStorage.cpp | 6 +++--- src/Disks/ObjectStorages/PlainObjectStorage.h | 2 +- .../PlainRewritableObjectStorage.h | 17 +++++++++++------ src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 4 ++-- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 +- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 2 +- 22 files changed, 74 insertions(+), 51 deletions(-) diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index e9212c3f04d..3e7bf3116bd 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -14,7 +15,10 @@ public: , re_gen(key_template) { } - DB::ObjectStorageKey generate(const String &, bool) const override { return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); } + DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional & /* key_prefix */) const override + { + return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); + } private: String key_template; @@ -29,7 +33,7 @@ public: : key_prefix(std::move(key_prefix_)) {} - DB::ObjectStorageKey generate(const String &, bool) const override + DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional & /* key_prefix */) const override { /// Path to store the new S3 object. @@ -60,7 +64,8 @@ public: : key_prefix(std::move(key_prefix_)) {} - DB::ObjectStorageKey generate(const String & path, bool) const override + DB::ObjectStorageKey + generate(const String & path, bool /* is_directory */, const std::optional & /* key_prefix */) const override { return DB::ObjectStorageKey::createAsRelative(key_prefix, path); } diff --git a/src/Common/ObjectStorageKeyGenerator.h b/src/Common/ObjectStorageKeyGenerator.h index 11da039b33b..12aeec1714d 100644 --- a/src/Common/ObjectStorageKeyGenerator.h +++ b/src/Common/ObjectStorageKeyGenerator.h @@ -11,7 +11,8 @@ class IObjectStorageKeysGenerator public: virtual ~IObjectStorageKeysGenerator() = default; - virtual ObjectStorageKey generate(const String & path, bool is_directory) const = 0; + /// Generates an object storage key based on a path in the virtual filesystem. + virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const = 0; }; using ObjectStorageKeysGeneratorPtr = std::shared_ptr; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index bc16955143b..0d92561d142 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -1,3 +1,4 @@ +#include #include #include "Common/Exception.h" @@ -117,7 +118,8 @@ AzureObjectStorage::AzureObjectStorage( { } -ObjectStorageKey AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { return ObjectStorageKey::createAsRelative(getRandomASCIIString(32)); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 2c7ce5e18dc..bc90b05e64d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -101,7 +101,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return true; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index a3b6e25e8ea..fb817005399 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -34,14 +34,16 @@ FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const return cache->createKeyForPath(path); } -ObjectStorageKey CachedObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey +CachedObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { - return object_storage->generateObjectKeyForPath(path); + return object_storage->generateObjectKeyForPath(path, key_prefix); } -ObjectStorageKey CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const +ObjectStorageKey +CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const { - return object_storage->generateObjectKeyPrefixForDirectoryPath(path); + return object_storage->generateObjectKeyPrefixForDirectoryPath(path, key_prefix); } ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settings) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 93ef2659cbb..efcdbfebabf 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -98,9 +98,10 @@ public: const std::string & getCacheName() const override { return cache_config_name; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; - ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; + ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const override; void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { object_storage->setKeysGenerator(gen); } diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 2a06d56e5c7..0a4426e8e66 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -15,7 +15,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( { } -ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory) const +ObjectStorageKey +CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & /* key_prefix */) const { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index fb1140de908..08495738505 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -26,7 +27,7 @@ public: explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); - ObjectStorageKey generate(const String & path, bool is_directory) const override; + ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; private: /// Longest key prefix and unresolved parts of the source path. diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index b5805f6d23a..880911b9958 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -537,7 +537,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation for (const auto & object_from : source_blobs) { - auto object_key = destination_object_storage.generateObjectKeyForPath(to_path); + auto object_key = destination_object_storage.generateObjectKeyForPath(to_path, std::nullopt /* key_prefix */); auto object_to = StoredObject(object_key.serialize()); object_storage.copyObjectToAnotherObjectStorage(object_from, object_to,read_settings,write_settings, destination_object_storage); @@ -738,7 +738,7 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile const WriteSettings & settings, bool autocommit) { - auto object_key = object_storage.generateObjectKeyForPath(path); + auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; if (metadata_helper) @@ -835,7 +835,7 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) { /// This function is a simplified and adapted version of DiskObjectStorageTransaction::writeFile(). - auto object_key = object_storage.generateObjectKeyForPath(path); + auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; if (metadata_helper) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index dcb2af9d4d3..3ce2a0f4903 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -4,8 +4,9 @@ #include #include -#include +#include #include +#include #include #include @@ -53,7 +54,8 @@ std::string HDFSObjectStorage::extractObjectKeyFromURL(const StoredObject & obje return path; } -ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { initializeHDFSFS(); /// what ever data_source_description.description value is, consider that key as relative key diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 8aae90d0721..0cb31eb8b8b 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -111,7 +111,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return true; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index ceea4d5a2bb..529c79790fd 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -232,10 +232,11 @@ public: /// Generate blob name for passed absolute local path. /// Path can be generated either independently or based on `path`. - virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path) const = 0; + virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const = 0; /// Object key prefix for local paths in the directory 'path'. - virtual ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */) const + virtual ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'generateObjectKeyPrefixForDirectoryPath' is not implemented"); } diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index a247d86ddce..20ef135cdf7 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,15 +1,16 @@ #include -#include -#include -#include +#include +#include +#include #include #include -#include #include #include +#include +#include #include -#include +#include namespace fs = std::filesystem; @@ -222,7 +223,8 @@ std::unique_ptr LocalObjectStorage::cloneObjectStorage( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage() is not implemented for LocalObjectStorage"); } -ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const +ObjectStorageKey +LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { constexpr size_t key_name_total_size = 32; return ObjectStorageKey::createAsRelative(key_prefix, getRandomASCIIString(key_name_total_size)); diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 371cd37f8b2..564d49bf876 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -81,7 +81,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isRemote() const override { return false; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 3da190c7256..589b18abca8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -42,7 +42,7 @@ bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) con { /// NOTE: exists() cannot be used here since it works only for existing /// key, and does not work for some intermediate path. - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); return object_storage->existsOrHasAnyChild(object_key.serialize()); } @@ -54,7 +54,7 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); auto directory = std::filesystem::path(std::move(key_prefix)) / ""; return object_storage->existsOrHasAnyChild(directory); @@ -62,7 +62,7 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); auto metadata = object_storage->tryGetObjectMetadata(object_key.serialize()); if (metadata) return metadata->size_bytes; @@ -71,7 +71,7 @@ uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); RelativePathsWithMetadata files; std::string abs_key = key_prefix; @@ -98,7 +98,7 @@ DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(con StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std::string & path) const { size_t object_size = getFileSize(path); - auto object_key = object_storage->generateObjectKeyForPath(path); + auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); return {StoredObject(object_key.serialize(), path, object_size)}; } @@ -130,7 +130,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { - auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path); + auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */); auto object = StoredObject(object_key.serialize()); metadata_storage.object_storage->removeObject(object); } @@ -155,7 +155,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path).serialize(); + auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, std::nullopt /* key_prefix */).serialize(); chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto op = std::make_unique( std::move(normalized_path), diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index f3d00a928e3..de65cd5c233 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -215,7 +215,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); return object_storage->existsOrHasAnyChild(metadata_key); @@ -228,7 +228,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str { if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path).serialize()) / ""; + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize()) / ""; chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); @@ -240,7 +240,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); RelativePathsWithMetadata files; std::string abs_key = key_prefix; diff --git a/src/Disks/ObjectStorages/PlainObjectStorage.h b/src/Disks/ObjectStorages/PlainObjectStorage.h index e0907d0b4d8..805b3436fce 100644 --- a/src/Disks/ObjectStorages/PlainObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainObjectStorage.h @@ -26,7 +26,7 @@ public: bool isPlain() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & /* key_prefix */) const override { return ObjectStorageKey::createAsRelative(BaseObjectStorage::getCommonKeyPrefix(), path); } diff --git a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h index 5f000afe625..dcea5964fc5 100644 --- a/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainRewritableObjectStorage.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include #include "CommonPathPrefixKeyGenerator.h" @@ -33,9 +35,10 @@ public: bool isPlain() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; - ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override; + ObjectStorageKey + generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional & key_prefix) const override; void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; } @@ -46,20 +49,22 @@ private: template -ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyForPath( + const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ false); + return key_generator->generate(path, /* is_directory */ false, key_prefix); } template -ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const +ObjectStorageKey PlainRewritableObjectStorage::generateObjectKeyPrefixForDirectoryPath( + const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ true); + return key_generator->generate(path, /* is_directory */ true, key_prefix); } } diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index 63e7ebb00c5..b20a2940e47 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -79,7 +79,7 @@ bool checkBatchRemove(S3ObjectStorage & storage) /// We are using generateObjectKeyForPath() which returns random object key. /// That generated key is placed in a right directory where we should have write access. const String path = fmt::format("clickhouse_remove_objects_capability_{}", getServerUUID()); - const auto key = storage.generateObjectKeyForPath(path); + const auto key = storage.generateObjectKeyForPath(path, {} /* key_prefix */); StoredObject object(key.serialize(), path); try { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a6672e14e10..3c4b4d76bf5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -624,12 +624,12 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( std::move(new_client), std::move(new_s3_settings), new_uri, s3_capabilities, key_generator, disk_name); } -ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const +ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { if (!key_generator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); - return key_generator->generate(path, /* is_directory */ false); + return key_generator->generate(path, /* is_directory */ false, key_prefix); } std::shared_ptr S3ObjectStorage::getS3StorageClient() diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index cbe004bc298..d786a6b37f3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -164,7 +164,7 @@ public: bool supportParallelWrite() const override { return true; } - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool isReadOnly() const override { return s3_settings.get()->read_only; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 9ca2950dae0..ab357d6f50d 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -82,7 +82,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & /* key_prefix */) const override { return ObjectStorageKey::createAsRelative(path); } From 27392fee6eec22c7f2dac3d17f73ab9a528f1fc8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 01:30:49 +0000 Subject: [PATCH 0646/1170] Minor refactor --- .../CommonPathPrefixKeyGenerator.cpp | 8 ++-- .../MetadataStorageFromPlainObjectStorage.cpp | 38 +++++++------------ .../MetadataStorageFromPlainObjectStorage.h | 8 ---- ...torageFromPlainRewritableObjectStorage.cpp | 26 +++++-------- ...aStorageFromPlainRewritableObjectStorage.h | 2 +- 5 files changed, 28 insertions(+), 54 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 0a4426e8e66..ef599a2f366 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -16,15 +16,15 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( } ObjectStorageKey -CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & /* key_prefix */) const +CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); - auto key = std::filesystem::path(object_key_prefix.empty() ? std::string() : object_key_prefix); + auto key = std::filesystem::path(object_key_prefix); /// The longest prefix is the same as path, meaning that the path is already mapped. if (suffix_parts.empty()) - return ObjectStorageKey::createAsRelative(storage_key_prefix, std::move(key)); + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, std::move(key)); /// File and top-level directory paths are mapped as is. if (!is_directory || object_key_prefix.empty()) @@ -40,7 +40,7 @@ CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, c key /= getRandomASCIIString(part_size); } - return ObjectStorageKey::createAsRelative(storage_key_prefix, key); + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, key); } std::tuple> CommonPathPrefixKeyGenerator::getLongestObjectKeyPrefix(const std::string & path) const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 589b18abca8..02048c07a57 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -80,9 +80,20 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co object_storage->listObjects(abs_key, files, 0); - std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); - return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); + std::unordered_set result; + for (const auto & elem : files) + { + const auto & p = elem->relative_path; + chassert(p.find(abs_key) == 0); + const auto child_pos = abs_key.size(); + /// string::npos is ok. + const auto slash_pos = p.find('/', child_pos); + if (slash_pos == std::string::npos) + result.emplace(p.substr(child_pos)); + else + result.emplace(p.substr(child_pos, slash_pos - child_pos)); + } + return std::vector(std::make_move_iterator(result.begin()), std::make_move_iterator(result.end())); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const @@ -102,27 +113,6 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std return {StoredObject(object_key.serialize(), path, object_size)}; } -void MetadataStorageFromPlainObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, - const std::string & /* storage_key_perfix */, - const RelativePathsWithMetadata & remote_paths, - const std::string & /* local_path */, - std::unordered_set & result) const -{ - for (const auto & elem : remote_paths) - { - const auto & path = elem->relative_path; - chassert(path.find(storage_key) == 0); - const auto child_pos = storage_key.size(); - /// string::npos is ok. - const auto slash_pos = path.find('/', child_pos); - if (slash_pos == std::string::npos) - result.emplace(path.substr(child_pos)); - else - result.emplace(path.substr(child_pos, slash_pos - child_pos)); - } -} - const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getStorageForNonTransactionalReads() const { return metadata_storage; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 97c5715a937..237327cd1f4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -86,14 +86,6 @@ protected: /// Returns a map of local paths to paths in object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } - - /// Retrieves the immediate files and directories within a given directory on a disk. - virtual void getDirectChildrenOnDisk( - const std::string & storage_key, - const std::string & storage_key_perfix, - const RelativePathsWithMetadata & remote_paths, - const std::string & local_path, - std::unordered_set & result) const; }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index de65cd5c233..b904c0d92b9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -121,7 +121,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri return result; } -void getDirectChildrenOnRewritableDisk( +void getDirectChildrenOnDiskImpl( const std::string & storage_key, const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, @@ -215,10 +215,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); - chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size()); - return object_storage->existsOrHasAnyChild(metadata_key); + auto key_prefix = object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize(); + return object_storage->existsOrHasAnyChild(key_prefix); } return false; @@ -228,11 +226,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str { if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) { - auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize()) / ""; - chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key - = std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size()); - return object_storage->existsOrHasAnyChild(metadata_key); + auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize()) / ""; + return object_storage->existsOrHasAnyChild(directory); } else return MetadataStorageFromPlainObjectStorage::isDirectory(path); @@ -240,12 +235,10 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize(); + auto key_prefix = object_storage->generateObjectKeyForPath(path, "" /* key_prefix */).serialize(); RelativePathsWithMetadata files; - std::string abs_key = key_prefix; - if (!abs_key.ends_with('/')) - abs_key += '/'; + auto abs_key = std::filesystem::path(object_storage->getCommonKeyPrefix()) / key_prefix / ""; object_storage->listObjects(abs_key, files, 0); @@ -255,8 +248,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) { - chassert(abs_key.starts_with(object_storage->getCommonKeyPrefix())); - auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / abs_key.substr(object_storage->getCommonKeyPrefix().size()); + auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; object_storage->listObjects(metadata_key, metadata_files, 0); getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); @@ -272,7 +264,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnRewritableDisk(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); + getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index 71153cbdc25..b067b391878 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -33,7 +33,7 @@ protected: const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - std::unordered_set & result) const override; + std::unordered_set & result) const; }; } From ecca720f9e076e49f280e48c6ff4046a19894b2a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 28 Jun 2024 06:42:38 +0000 Subject: [PATCH 0647/1170] minor --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 02048c07a57..7553c7733b5 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -145,11 +145,10 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, std::nullopt /* key_prefix */).serialize(); - chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix())); + auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, "" /* key_prefix */).serialize(); auto op = std::make_unique( std::move(normalized_path), - key_prefix.substr(object_storage->getCommonKeyPrefix().size()), + std::move(key_prefix), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix()); From 97519ae800b9a26942973d888354a2b013d53cc6 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 4 Jul 2024 07:02:13 +0000 Subject: [PATCH 0648/1170] in-memory map path comparator --- .../CommonPathPrefixKeyGenerator.cpp | 5 +++-- .../CommonPathPrefixKeyGenerator.h | 4 +++- .../MetadataStorageFromPlainObjectStorage.h | 3 ++- ...torageFromPlainObjectStorageOperations.cpp | 18 ++++++++--------- ...torageFromPlainRewritableObjectStorage.cpp | 9 ++++----- src/Disks/ObjectStorages/PathComparator.h | 20 +++++++++++++++++++ 6 files changed, 41 insertions(+), 18 deletions(-) create mode 100644 src/Disks/ObjectStorages/PathComparator.h diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index ef599a2f366..062a2542654 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -18,7 +18,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { - const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); + const auto & [object_key_prefix, suffix_parts] + = getLongestObjectKeyPrefix(is_directory ? std::filesystem::path(path).parent_path().string() : path); auto key = std::filesystem::path(object_key_prefix); @@ -54,7 +55,7 @@ std::tuple> CommonPathPrefixKeyGenerator:: while (p != p.root_path()) { - auto it = ptr->find(p / ""); + auto it = ptr->find(p); if (it != ptr->end()) { std::vector vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end())); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index 08495738505..bca4f7060c4 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include @@ -23,7 +25,7 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: /// Local to remote path map. Leverages filesystem::path comparator for paths. - using PathMap = std::map; + using PathMap = std::map; explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 237327cd1f4..9ea1c475821 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -29,7 +30,7 @@ class MetadataStorageFromPlainObjectStorage : public IMetadataStorage { public: /// Local path prefixes mapped to storage key prefixes. - using PathMap = std::map; + using PathMap = std::map; private: friend class MetadataStorageFromPlainObjectStorageTransaction; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 0a6086bd39d..b0b384f62c7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -43,7 +43,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - if (path_map.contains(path)) + if (path_map.contains(path.parent_path())) return; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -64,7 +64,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; - [[maybe_unused]] auto result = path_map.emplace(path, std::move(key_prefix)); + [[maybe_unused]] auto result = path_map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); @@ -84,7 +84,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un if (write_finalized) { - path_map.erase(path); + path_map.erase(path.parent_path()); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -111,11 +111,11 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto expected_it = path_map.find(expected_path); + auto expected_it = path_map.find(expected_path.parent_path()); if (expected_it == path_map.end()) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); - if (path_map.contains(new_path)) + if (path_map.contains(new_path.parent_path())) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); @@ -156,7 +156,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - [[maybe_unused]] auto result = path_map.emplace(path_to, path_map.extract(path_from).mapped()); + [[maybe_unused]] auto result = path_map.emplace(path_to.parent_path(), path_map.extract(path_from.parent_path()).mapped()); chassert(result.second); write_finalized = true; @@ -165,7 +165,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock &) { if (write_finalized) - path_map.emplace(path_from, path_map.extract(path_to).mapped()); + path_map.emplace(path_from.parent_path(), path_map.extract(path_to.parent_path()).mapped()); if (write_created) { @@ -186,7 +186,7 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto path_it = path_map.find(path); + auto path_it = path_map.find(path.parent_path()); if (path_it == path_map.end()) return; @@ -223,7 +223,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - path_map.emplace(path, std::move(key_prefix)); + path_map.emplace(path.parent_path(), std::move(key_prefix)); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index b904c0d92b9..ba8dfc891dd 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -97,7 +97,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri std::pair res; { std::lock_guard lock(mutex); - res = result.emplace(local_path, remote_path.parent_path()); + res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); } /// This can happen if table replication is enabled, then the same local path is written @@ -145,11 +145,10 @@ void getDirectChildrenOnDiskImpl( break; auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); - if (slash_num != 1) + if (slash_num != 0) continue; - chassert(k.back() == '/'); - remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end() - 1)); + remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/"); } } @@ -243,7 +242,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi object_storage->listObjects(abs_key, files, 0); std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); + getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/PathComparator.h new file mode 100644 index 00000000000..fe97a465937 --- /dev/null +++ b/src/Disks/ObjectStorages/PathComparator.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ +// TODO: rename +struct PathComparator +{ + bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + { + auto d1 = std::distance(path1.begin(), path1.end()); + auto d2 = std::distance(path2.begin(), path2.end()); + if (d1 != d2) + return d1 < d2; + return path1 < path2; + } +}; + +} From aa290b6398a5affa8405d3584795bce6bf7450d4 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 10 Jul 2024 01:15:57 +0000 Subject: [PATCH 0649/1170] use a designated mutex for path_map --- .../CommonPathPrefixKeyGenerator.cpp | 13 ++- .../CommonPathPrefixKeyGenerator.h | 6 +- .../MetadataStorageFromPlainObjectStorage.h | 6 +- ...torageFromPlainObjectStorageOperations.cpp | 99 ++++++++++++++----- ...aStorageFromPlainObjectStorageOperations.h | 13 +-- ...torageFromPlainRewritableObjectStorage.cpp | 25 ++--- ...aStorageFromPlainRewritableObjectStorage.h | 4 +- src/Disks/ObjectStorages/PathComparator.h | 27 +++-- 8 files changed, 122 insertions(+), 71 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 062a2542654..19dd819fc17 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,4 +1,5 @@ #include "CommonPathPrefixKeyGenerator.h" +#include "Disks/ObjectStorages/PathComparator.h" #include @@ -9,9 +10,8 @@ namespace DB { -CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator( - String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_) - : storage_key_prefix(key_prefix_), shared_mutex(shared_mutex_), path_map(std::move(path_map_)) +CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator(String key_prefix_, std::weak_ptr path_map_) + : storage_key_prefix(key_prefix_), path_map(std::move(path_map_)) { } @@ -49,14 +49,13 @@ std::tuple> CommonPathPrefixKeyGenerator:: std::filesystem::path p(path); std::deque dq; - std::shared_lock lock(shared_mutex); - auto ptr = path_map.lock(); + std::shared_lock lock(ptr->mutex); while (p != p.root_path()) { - auto it = ptr->find(p); - if (it != ptr->end()) + auto it = ptr->map.find(p); + if (it != ptr->map.end()) { std::vector vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end())); return std::make_tuple(it->second, std::move(vec)); diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index bca4f7060c4..e337745b627 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -25,9 +25,8 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: /// Local to remote path map. Leverages filesystem::path comparator for paths. - using PathMap = std::map; - explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr path_map_); + explicit CommonPathPrefixKeyGenerator(String key_prefix_, std::weak_ptr path_map_); ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; @@ -37,8 +36,7 @@ private: const String storage_key_prefix; - SharedMutex & shared_mutex; - std::weak_ptr path_map; + std::weak_ptr path_map; }; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 9ea1c475821..dfb9632666c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -28,10 +28,6 @@ using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; - private: friend class MetadataStorageFromPlainObjectStorageTransaction; @@ -86,7 +82,7 @@ protected: virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } /// Returns a map of local paths to paths in object storage. - virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } + virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index b0b384f62c7..b4a85efbaab 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,4 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" +#include "Disks/ObjectStorages/PathComparator.h" #include #include @@ -30,7 +31,7 @@ ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const s MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)) @@ -43,8 +44,13 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - if (path_map.contains(path.parent_path())) - return; + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::shared_lock lock(mutex); + if (map.contains(path.parent_path())) + return; + } auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -64,8 +70,11 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; - [[maybe_unused]] auto result = path_map.emplace(path.parent_path(), std::move(key_prefix)); - chassert(result.second); + { + std::unique_lock lock(mutex); + [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); + chassert(result.second); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); @@ -80,11 +89,17 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { + auto & map = path_map.map; + auto & mutex = path_map.mutex; + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); if (write_finalized) { - path_map.erase(path.parent_path()); + { + std::unique_lock lock(mutex); + map.erase(path.parent_path()); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -97,7 +112,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path_from(std::move(path_from_)) @@ -111,14 +126,25 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto expected_it = path_map.find(expected_path.parent_path()); - if (expected_it == path_map.end()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); + auto & map = path_map.map; + auto & mutex = path_map.mutex; - if (path_map.contains(new_path.parent_path())) - throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); + std::filesystem::path remote_path; + { + std::shared_lock lock(mutex); + auto expected_it = map.find(expected_path.parent_path()); + if (expected_it == map.end()) + throw Exception( + ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); - auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); + if (map.contains(new_path.parent_path())) + throw Exception( + ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); + + remote_path = expected_it->second; + } + + auto metadata_object_key = createMetadataObjectKey(remote_path, metadata_key_prefix); auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); @@ -156,8 +182,13 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - [[maybe_unused]] auto result = path_map.emplace(path_to.parent_path(), path_map.extract(path_from.parent_path()).mapped()); - chassert(result.second); + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::unique_lock lock(mutex); + [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); + chassert(result.second); + } write_finalized = true; } @@ -165,7 +196,12 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock &) { if (write_finalized) - path_map.emplace(path_from.parent_path(), path_map.extract(path_to.parent_path()).mapped()); + { + auto & map = path_map.map; + auto & mutex = path_map.mutex; + std::unique_lock lock(mutex); + map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); + } if (write_created) { @@ -176,28 +212,34 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq } MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( - std::filesystem::path && path_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, - ObjectStoragePtr object_storage_, - const std::string & metadata_key_prefix_) + std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto path_it = path_map.find(path.parent_path()); - if (path_it == path_map.end()) - return; + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::shared_lock lock(mutex); + auto path_it = map.find(path.parent_path()); + if (path_it == map.end()) + return; + key_prefix = path_it->second; + } LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); - key_prefix = path_it->second; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); object_storage->removeObject(metadata_object); - path_map.erase(path_it); + { + std::unique_lock lock(mutex); + map.erase(path.parent_path()); + } + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -223,7 +265,12 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - path_map.emplace(path.parent_path(), std::move(key_prefix)); + auto & map = path_map.map; + auto & mutex = path_map.mutex; + { + std::unique_lock lock(mutex); + map.emplace(path.parent_path(), std::move(key_prefix)); + } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index e31e3cbb262..1b2471dd316 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -2,6 +2,7 @@ #include #include +#include "Disks/ObjectStorages/PathComparator.h" #include #include @@ -14,7 +15,7 @@ class MetadataStorageFromPlainObjectStorageCreateDirectoryOperation final : publ private: std::filesystem::path path; std::string key_prefix; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -26,7 +27,7 @@ public: MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, std::string && key_prefix_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); @@ -39,7 +40,7 @@ class MetadataStorageFromPlainObjectStorageMoveDirectoryOperation final : public private: std::filesystem::path path_from; std::filesystem::path path_to; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -53,7 +54,7 @@ public: MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( std::filesystem::path && path_from_, std::filesystem::path && path_to_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); @@ -67,7 +68,7 @@ class MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation final : publ private: std::filesystem::path path; - MetadataStorageFromPlainObjectStorage::PathMap & path_map; + InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; @@ -77,7 +78,7 @@ private: public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, - MetadataStorageFromPlainObjectStorage::PathMap & path_map_, + InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index ba8dfc891dd..cf51a6a5314 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -3,11 +3,12 @@ #include #include -#include #include +#include #include #include #include "CommonPathPrefixKeyGenerator.h" +#include "Disks/ObjectStorages/PathComparator.h" namespace DB @@ -37,9 +38,10 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) +InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { - MetadataStorageFromPlainObjectStorage::PathMap result; + using Map = InMemoryPathMap::Map; + Map result; ThreadPool & pool = getIOThreadPool().get(); ThreadPoolCallbackRunnerLocal runner(pool, "PlainRWMetaLoad"); @@ -94,7 +96,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); auto remote_path = std::filesystem::path(std::move(suffix)); - std::pair res; + std::pair res; { std::lock_guard lock(mutex); res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); @@ -126,14 +128,13 @@ void getDirectChildrenOnDiskImpl( const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - const MetadataStorageFromPlainObjectStorage::PathMap & local_path_prefixes, + const InMemoryPathMap::Map & local_path_prefixes, SharedMutex & shared_mutex, std::unordered_set & result) { - using PathMap = MetadataStorageFromPlainObjectStorage::PathMap; - /// Map remote paths into local subdirectories. - std::unordered_map remote_to_local_subdir; + using Map = InMemoryPathMap::Map; + std::unordered_map remote_to_local_subdir; { std::shared_lock lock(shared_mutex); @@ -189,7 +190,7 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) - , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) + , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) { if (object_storage->isWriteOnce()) throw Exception( @@ -197,14 +198,14 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), metadata_mutex, path_map); + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage() { auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, path_map->size()); + CurrentMetrics::sub(metric, path_map->map.size()); } bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & path) const @@ -263,7 +264,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, *getPathMap(), metadata_mutex, result); + getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index b067b391878..fea461abab8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -13,7 +13,7 @@ class MetadataStorageFromPlainRewritableObjectStorage final : public MetadataSto { private: const std::string metadata_key_prefix; - std::shared_ptr path_map; + std::shared_ptr path_map; public: MetadataStorageFromPlainRewritableObjectStorage(ObjectStoragePtr object_storage_, String storage_path_prefix_); @@ -27,7 +27,7 @@ public: protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } - std::shared_ptr getPathMap() const override { return path_map; } + std::shared_ptr getPathMap() const override { return path_map; } void getDirectChildrenOnDisk( const std::string & storage_key, const std::string & storage_key_perfix, diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/PathComparator.h index fe97a465937..fae82108789 100644 --- a/src/Disks/ObjectStorages/PathComparator.h +++ b/src/Disks/ObjectStorages/PathComparator.h @@ -1,20 +1,29 @@ #pragma once #include +#include +#include "Common/SharedMutex.h" namespace DB { -// TODO: rename -struct PathComparator + + +struct InMemoryPathMap { - bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + struct PathComparator { - auto d1 = std::distance(path1.begin(), path1.end()); - auto d2 = std::distance(path2.begin(), path2.end()); - if (d1 != d2) - return d1 < d2; - return path1 < path2; - } + bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const + { + auto d1 = std::distance(path1.begin(), path1.end()); + auto d2 = std::distance(path2.begin(), path2.end()); + if (d1 != d2) + return d1 < d2; + return path1 < path2; + } + }; + using Map = std::map; + Map map; + SharedMutex mutex; }; } From 0e78ed6b580646cc08721eef415ffb3fe2f697cb Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 10 Jul 2024 23:04:21 +0000 Subject: [PATCH 0650/1170] simplify listDirectory --- ...torageFromPlainRewritableObjectStorage.cpp | 28 ++++--------------- ...aStorageFromPlainRewritableObjectStorage.h | 1 - 2 files changed, 5 insertions(+), 24 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index cf51a6a5314..6a0eff0a136 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -125,17 +125,12 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, void getDirectChildrenOnDiskImpl( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, const InMemoryPathMap::Map & local_path_prefixes, SharedMutex & shared_mutex, std::unordered_set & result) { - /// Map remote paths into local subdirectories. - using Map = InMemoryPathMap::Map; - std::unordered_map remote_to_local_subdir; - { std::shared_lock lock(shared_mutex); auto end_it = local_path_prefixes.end(); @@ -147,9 +142,9 @@ void getDirectChildrenOnDiskImpl( auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); if (slash_num != 0) - continue; + break; - remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/"); + result.emplace(std::string(k.begin() + local_path.size(), k.end()) + "/"); } } @@ -169,18 +164,6 @@ void getDirectChildrenOnDiskImpl( if (!skip_list.contains(filename)) result.emplace(std::move(filename)); } - else - { - /// Subdirectories. - chassert(path.find(storage_key_perfix) == 0); - auto it = remote_to_local_subdir.find(path.substr(storage_key_perfix.size(), slash_pos - storage_key_perfix.size())); - /// Mapped subdirectories. - if (it != remote_to_local_subdir.end()) - result.emplace(it->second); - /// The remote subdirectory name is the same as the local subdirectory. - else - result.emplace(path.substr(child_pos, slash_pos - child_pos)); - } } } @@ -243,7 +226,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi object_storage->listObjects(abs_key, files, 0); std::unordered_set directories; - getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, std::filesystem::path(path) / "", directories); + getDirectChildrenOnDisk(abs_key, files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) @@ -251,7 +234,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; object_storage->listObjects(metadata_key, metadata_files, 0); - getDirectChildrenOnDisk(metadata_key, getMetadataKeyPrefix(), metadata_files, path, directories); + getDirectChildrenOnDisk(metadata_key, metadata_files, std::filesystem::path(path) / "", directories); } return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); @@ -259,12 +242,11 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, storage_key_perfix, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); + getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index fea461abab8..8fd147e15b9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -30,7 +30,6 @@ protected: std::shared_ptr getPathMap() const override { return path_map; } void getDirectChildrenOnDisk( const std::string & storage_key, - const std::string & storage_key_perfix, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const; From 82f5aceb484a322960065f973bec2b61c31219aa Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 11 Jul 2024 06:22:26 +0000 Subject: [PATCH 0651/1170] introduce flat structure --- .../FlatStructureKeyGenerator.cpp | 51 +++++++++++++++++++ .../FlatStructureKeyGenerator.h | 23 +++++++++ ...torageFromPlainRewritableObjectStorage.cpp | 15 ++++-- 3 files changed, 86 insertions(+), 3 deletions(-) create mode 100644 src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp create mode 100644 src/Disks/ObjectStorages/FlatStructureKeyGenerator.h diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp new file mode 100644 index 00000000000..d6fb32b65d4 --- /dev/null +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp @@ -0,0 +1,51 @@ +#include "FlatStructureKeyGenerator.h" +#include "Common/ObjectStorageKey.h" +#include "Common/SharedMutex.h" +#include "Disks/ObjectStorages/PathComparator.h" +#include + +#include +#include +#include + +namespace DB +{ + +FlatStructureKeyGenerator::FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) + : storage_key_prefix(storage_key_prefix_), path_map(std::move(path_map_)) +{ +} + +ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const +{ + if (is_directory) + chassert(path.ends_with('/')); + + const auto p = std::filesystem::path(path); + auto directory = p.parent_path(); + + constexpr size_t part_size = 32; + + std::optional remote_path; + { + auto ptr = path_map.lock(); + std::shared_lock lock(ptr->mutex); + auto it = ptr->map.find(p); + if (it != ptr->map.end()) + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, it->second); + + it = ptr->map.find(directory); + if (it != ptr->map.end()) + remote_path = it->second; + } + std::filesystem::path key = remote_path.has_value() ? *remote_path + : is_directory ? std::filesystem::path(getRandomASCIIString(part_size)) + : directory; + + if (!is_directory) + key /= p.filename(); + + return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, key); +} + +} diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h new file mode 100644 index 00000000000..2c585dffb81 --- /dev/null +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +#include +namespace DB +{ + +class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator +{ +public: + explicit FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); + + ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; + +private: + const String storage_key_prefix; + + std::weak_ptr path_map; +}; + +} diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 6a0eff0a136..afaa7bf06ff 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include #include @@ -8,7 +10,6 @@ #include #include #include "CommonPathPrefixKeyGenerator.h" -#include "Disks/ObjectStorages/PathComparator.h" namespace DB @@ -181,8 +182,16 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); - object_storage->setKeysGenerator(keys_gen); + if (getMetadataKeyPrefix() == object_storage->getCommonKeyPrefix()) + { + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + object_storage->setKeysGenerator(keys_gen); + } + else + { + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + object_storage->setKeysGenerator(keys_gen); + } } MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage() From c0e6780dfe5977316e50e587877f3fe6ef11d048 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 11 Jul 2024 22:48:07 +0000 Subject: [PATCH 0652/1170] rename PathComparator.h -> InMemoryPathMap.h --- src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp | 4 ++-- src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h | 5 ++--- src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp | 2 +- src/Disks/ObjectStorages/FlatStructureKeyGenerator.h | 2 +- .../ObjectStorages/{PathComparator.h => InMemoryPathMap.h} | 0 .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 1 + .../ObjectStorages/MetadataStorageFromPlainObjectStorage.h | 3 ++- .../MetadataStorageFromPlainObjectStorageOperations.cpp | 2 +- .../MetadataStorageFromPlainObjectStorageOperations.h | 2 +- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 4 ++-- 10 files changed, 13 insertions(+), 12 deletions(-) rename src/Disks/ObjectStorages/{PathComparator.h => InMemoryPathMap.h} (100%) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 19dd819fc17..1fa06823bae 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,5 +1,5 @@ -#include "CommonPathPrefixKeyGenerator.h" -#include "Disks/ObjectStorages/PathComparator.h" +#include +#include #include diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index e337745b627..8b5037e3804 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -1,9 +1,6 @@ #pragma once #include -#include - -#include #include #include @@ -21,6 +18,8 @@ namespace DB /// /// The key generator ensures that the original directory hierarchy is /// preserved, which is required for the MergeTree family. + +struct InMemoryPathMap; class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator { public: diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp index d6fb32b65d4..414aea2b08b 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp @@ -1,7 +1,7 @@ #include "FlatStructureKeyGenerator.h" +#include #include "Common/ObjectStorageKey.h" #include "Common/SharedMutex.h" -#include "Disks/ObjectStorages/PathComparator.h" #include #include diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h index 2c585dffb81..6b5b2203bed 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h +++ b/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h @@ -1,12 +1,12 @@ #pragma once -#include #include #include namespace DB { +struct InMemoryPathMap; class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator { public: diff --git a/src/Disks/ObjectStorages/PathComparator.h b/src/Disks/ObjectStorages/InMemoryPathMap.h similarity index 100% rename from src/Disks/ObjectStorages/PathComparator.h rename to src/Disks/ObjectStorages/InMemoryPathMap.h diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 7553c7733b5..364d04e2b52 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -1,5 +1,6 @@ #include "MetadataStorageFromPlainObjectStorage.h" #include +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index dfb9632666c..a9a1a648f96 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -2,9 +2,9 @@ #include #include +#include #include #include -#include #include #include @@ -13,6 +13,7 @@ namespace DB { +struct InMemoryPathMap; struct UnlinkMetadataFileOperationOutcome; using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index b4a85efbaab..6f5109faec4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,5 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" -#include "Disks/ObjectStorages/PathComparator.h" +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 1b2471dd316..778585fa758 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -1,8 +1,8 @@ #pragma once #include +#include #include -#include "Disks/ObjectStorages/PathComparator.h" #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index afaa7bf06ff..c312eae4077 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,7 +1,7 @@ -#include #include +#include +#include #include -#include #include #include From 3f066018fb0c74783ee486f54fc472ffd9cd7cc1 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 12 Jul 2024 02:27:45 +0000 Subject: [PATCH 0653/1170] style and doc --- src/Common/ObjectStorageKeyGenerator.cpp | 1 - src/Common/ObjectStorageKeyGenerator.h | 4 ++++ .../CommonPathPrefixKeyGenerator.h | 1 + ...=> FlatDirectoryStructureKeyGenerator.cpp} | 9 ++++--- ...h => FlatDirectoryStructureKeyGenerator.h} | 4 ++-- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 - src/Disks/ObjectStorages/InMemoryPathMap.h | 2 +- .../Local/LocalObjectStorage.cpp | 1 - .../MetadataStorageFromPlainObjectStorage.cpp | 2 -- .../MetadataStorageFromPlainObjectStorage.h | 5 ++-- ...torageFromPlainObjectStorageOperations.cpp | 24 +++++++++---------- ...aStorageFromPlainObjectStorageOperations.h | 3 +-- ...torageFromPlainRewritableObjectStorage.cpp | 12 +++++++--- 13 files changed, 35 insertions(+), 34 deletions(-) rename src/Disks/ObjectStorages/{FlatStructureKeyGenerator.cpp => FlatDirectoryStructureKeyGenerator.cpp} (80%) rename src/Disks/ObjectStorages/{FlatStructureKeyGenerator.h => FlatDirectoryStructureKeyGenerator.h} (64%) diff --git a/src/Common/ObjectStorageKeyGenerator.cpp b/src/Common/ObjectStorageKeyGenerator.cpp index 3e7bf3116bd..3bdc0004198 100644 --- a/src/Common/ObjectStorageKeyGenerator.cpp +++ b/src/Common/ObjectStorageKeyGenerator.cpp @@ -3,7 +3,6 @@ #include #include -#include #include diff --git a/src/Common/ObjectStorageKeyGenerator.h b/src/Common/ObjectStorageKeyGenerator.h index 12aeec1714d..008e3c88fac 100644 --- a/src/Common/ObjectStorageKeyGenerator.h +++ b/src/Common/ObjectStorageKeyGenerator.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "ObjectStorageKey.h" namespace DB @@ -12,6 +13,9 @@ public: virtual ~IObjectStorageKeysGenerator() = default; /// Generates an object storage key based on a path in the virtual filesystem. + /// @param path - Path in the virtual filesystem. + /// @param is_directory - If the path in the virtual filesystem corresponds to a directory. + /// @param key_prefix - Optional key prefix for the generated object storage key. If provided, this prefix will be added to the beginning of the generated key. virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const = 0; }; diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h index 8b5037e3804..ea91d78600d 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.h @@ -9,6 +9,7 @@ namespace DB { +/// Deprecated. Used for backward compatibility with plain rewritable disks without a separate metadata layout. /// Object storage key generator used specifically with the /// MetadataStorageFromPlainObjectStorage if multiple writes are allowed. diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp similarity index 80% rename from src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp rename to src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp index 414aea2b08b..64959b729b6 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp @@ -1,4 +1,4 @@ -#include "FlatStructureKeyGenerator.h" +#include "FlatDirectoryStructureKeyGenerator.h" #include #include "Common/ObjectStorageKey.h" #include "Common/SharedMutex.h" @@ -11,12 +11,12 @@ namespace DB { -FlatStructureKeyGenerator::FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) +FlatDirectoryStructureKeyGenerator::FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_) : storage_key_prefix(storage_key_prefix_), path_map(std::move(path_map_)) { } -ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const +ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional & key_prefix) const { if (is_directory) chassert(path.ends_with('/')); @@ -24,8 +24,6 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i const auto p = std::filesystem::path(path); auto directory = p.parent_path(); - constexpr size_t part_size = 32; - std::optional remote_path; { auto ptr = path_map.lock(); @@ -38,6 +36,7 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i if (it != ptr->map.end()) remote_path = it->second; } + constexpr size_t part_size = 32; std::filesystem::path key = remote_path.has_value() ? *remote_path : is_directory ? std::filesystem::path(getRandomASCIIString(part_size)) : directory; diff --git a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h similarity index 64% rename from src/Disks/ObjectStorages/FlatStructureKeyGenerator.h rename to src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h index 6b5b2203bed..4dbac5d3003 100644 --- a/src/Disks/ObjectStorages/FlatStructureKeyGenerator.h +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h @@ -7,10 +7,10 @@ namespace DB { struct InMemoryPathMap; -class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator +class FlatDirectoryStructureKeyGenerator : public IObjectStorageKeysGenerator { public: - explicit FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); + explicit FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr path_map_); ObjectStorageKey generate(const String & path, bool is_directory, const std::optional & key_prefix) const override; diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 3ce2a0f4903..00ef4b63e6f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index fae82108789..ea08784719e 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -2,7 +2,7 @@ #include #include -#include "Common/SharedMutex.h" +#include namespace DB { diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 20ef135cdf7..5b61c57ca21 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 364d04e2b52..2036208c389 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -146,10 +146,8 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std return; auto normalized_path = normalizeDirectoryPath(path); - auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, "" /* key_prefix */).serialize(); auto op = std::make_unique( std::move(normalized_path), - std::move(key_prefix), *metadata_storage.getPathMap(), object_storage, metadata_storage.getMetadataKeyPrefix()); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index a9a1a648f96..2aac7158bd5 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -78,11 +78,10 @@ public: bool supportsStat() const override { return false; } protected: - /// Get the object storage prefix for storing metadata files. If stored behind a separate endpoint, - /// the metadata keys reflect the layout of the regular files. + /// Get the object storage prefix for storing metadata files. virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } - /// Returns a map of local paths to paths in object storage. + /// Returns a map of virtual filesystem paths to paths in the object storage. virtual std::shared_ptr getPathMap() const { throwNotImplemented(); } }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 6f5109faec4..9e18f6cdb08 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -29,25 +29,21 @@ ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const s } MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( - std::filesystem::path && path_, - std::string && key_prefix_, - InMemoryPathMap & path_map_, - ObjectStoragePtr object_storage_, - const std::string & metadata_key_prefix_) + std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) : path(std::move(path_)) - , key_prefix(key_prefix_) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) + , key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* key_prefix */).serialize()) { } void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::shared_lock lock(mutex); + auto & map = path_map.map; if (map.contains(path.parent_path())) return; } @@ -72,6 +68,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::unique_lock lock(mutex); + auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); } @@ -89,7 +86,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto & map = path_map.map; auto & mutex = path_map.mutex; auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); @@ -98,6 +94,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un { { std::unique_lock lock(mutex); + auto & map = path_map.map; map.erase(path.parent_path()); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -126,12 +123,12 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto & map = path_map.map; auto & mutex = path_map.mutex; std::filesystem::path remote_path; { std::shared_lock lock(mutex); + auto & map = path_map.map; auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) throw Exception( @@ -182,10 +179,10 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::unique_lock lock(mutex); + auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); chassert(result.second); } @@ -197,9 +194,9 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq { if (write_finalized) { - auto & map = path_map.map; auto & mutex = path_map.mutex; std::unique_lock lock(mutex); + auto & map = path_map.map; map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); } @@ -219,10 +216,10 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::shared_lock lock(mutex); + auto & map = path_map.map; auto path_it = map.find(path.parent_path()); if (path_it == map.end()) return; @@ -237,6 +234,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: { std::unique_lock lock(mutex); + auto & map = path_map.map; map.erase(path.parent_path()); } @@ -265,10 +263,10 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - auto & map = path_map.map; auto & mutex = path_map.mutex; { std::unique_lock lock(mutex); + auto & map = path_map.map; map.emplace(path.parent_path(), std::move(key_prefix)); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 778585fa758..3ac0ffef8d2 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -14,10 +14,10 @@ class MetadataStorageFromPlainObjectStorageCreateDirectoryOperation final : publ { private: std::filesystem::path path; - std::string key_prefix; InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; + const std::string key_prefix; bool write_created = false; bool write_finalized = false; @@ -26,7 +26,6 @@ public: // Assuming that paths are normalized. MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, - std::string && key_prefix_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index c312eae4077..fd3b9523df6 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -132,16 +132,20 @@ void getDirectChildrenOnDiskImpl( SharedMutex & shared_mutex, std::unordered_set & result) { + /// Directories are retrieved from the in-memory path map. { std::shared_lock lock(shared_mutex); auto end_it = local_path_prefixes.end(); for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) { - const auto & [k, v] = std::make_tuple(it->first.string(), it->second); + const auto & [k, _] = std::make_tuple(it->first.string(), it->second); if (!k.starts_with(local_path)) break; auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); + /// The local_path_prefixes comparator ensures that the paths with the smallest number of + /// hops from the local_path are iterated first. The paths do not end with '/', hence + /// break the loop if the number of slashes is greater than 0. if (slash_num != 0) break; @@ -149,6 +153,7 @@ void getDirectChildrenOnDiskImpl( } } + /// Files. auto skip_list = std::set{PREFIX_PATH_FILE_NAME}; for (const auto & elem : remote_paths) { @@ -189,7 +194,8 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita } else { - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + /// Use flat directory structure if the metadata is stored separately from the table data. + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } } From d4c13714abb6b307c4344c74bd4b7973c03e68df Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 13 Jul 2024 06:51:02 +0000 Subject: [PATCH 0654/1170] address feedback: TSA_GUARDED_BY --- .../CommonPathPrefixKeyGenerator.cpp | 5 +-- .../FlatDirectoryStructureKeyGenerator.cpp | 7 ++-- src/Disks/ObjectStorages/InMemoryPathMap.h | 4 +-- ...torageFromPlainObjectStorageOperations.cpp | 34 +++++++------------ ...torageFromPlainRewritableObjectStorage.cpp | 34 +++++++++++-------- 5 files changed, 40 insertions(+), 44 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 1fa06823bae..1d041626a7e 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -49,8 +50,8 @@ std::tuple> CommonPathPrefixKeyGenerator:: std::filesystem::path p(path); std::deque dq; - auto ptr = path_map.lock(); - std::shared_lock lock(ptr->mutex); + const auto ptr = path_map.lock(); + SharedLockGuard lock(ptr->mutex); while (p != p.root_path()) { diff --git a/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp index 64959b729b6..0f35bfd2427 100644 --- a/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp @@ -1,7 +1,8 @@ #include "FlatDirectoryStructureKeyGenerator.h" #include #include "Common/ObjectStorageKey.h" -#include "Common/SharedMutex.h" +#include +#include #include #include @@ -26,8 +27,8 @@ ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & pat std::optional remote_path; { - auto ptr = path_map.lock(); - std::shared_lock lock(ptr->mutex); + const auto ptr = path_map.lock(); + SharedLockGuard lock(ptr->mutex); auto it = ptr->map.find(p); if (it != ptr->map.end()) return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, it->second); diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index ea08784719e..dcd28dfaf6c 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -22,8 +22,8 @@ struct InMemoryPathMap } }; using Map = std::map; - Map map; - SharedMutex mutex; + mutable SharedMutex mutex; + Map map TSA_GUARDED_BY(mutex); }; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 9e18f6cdb08..8a06b204cfc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,4 +1,5 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" +#include "Common/SharedLockGuard.h" #include #include @@ -40,11 +41,9 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { - auto & mutex = path_map.mutex; { - std::shared_lock lock(mutex); - auto & map = path_map.map; - if (map.contains(path.parent_path())) + SharedLockGuard lock(path_map.mutex); + if (path_map.map.contains(path.parent_path())) return; } @@ -67,7 +66,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: write_created = true; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); chassert(result.second); @@ -86,16 +85,13 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto & mutex = path_map.mutex; - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); if (write_finalized) { { - std::unique_lock lock(mutex); - auto & map = path_map.map; - map.erase(path.parent_path()); + std::lock_guard lock(path_map.mutex); + path_map.map.erase(path.parent_path()); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -123,11 +119,9 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) { - auto & mutex = path_map.mutex; - std::filesystem::path remote_path; { - std::shared_lock lock(mutex); + SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) @@ -179,9 +173,8 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); - auto & mutex = path_map.mutex; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); chassert(result.second); @@ -194,8 +187,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq { if (write_finalized) { - auto & mutex = path_map.mutex; - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); } @@ -216,9 +208,8 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { - auto & mutex = path_map.mutex; { - std::shared_lock lock(mutex); + SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; auto path_it = map.find(path.parent_path()); if (path_it == map.end()) @@ -233,7 +224,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: object_storage->removeObject(metadata_object); { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.erase(path.parent_path()); } @@ -263,9 +254,8 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un writeString(path.string(), *buf); buf->finalize(); - auto & mutex = path_map.mutex; { - std::unique_lock lock(mutex); + std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.emplace(path.parent_path(), std::move(key_prefix)); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index fd3b9523df6..22e73e36372 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -9,6 +9,8 @@ #include #include #include +#include "Common/SharedLockGuard.h" +#include "Common/SharedMutex.h" #include "CommonPathPrefixKeyGenerator.h" @@ -39,14 +41,13 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) +std::shared_ptr loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { + auto result = std::make_shared(); using Map = InMemoryPathMap::Map; - Map result; ThreadPool & pool = getIOThreadPool().get(); ThreadPoolCallbackRunnerLocal runner(pool, "PlainRWMetaLoad"); - std::mutex mutex; LoggerPtr log = getLogger("MetadataStorageFromPlainObjectStorage"); @@ -66,7 +67,7 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_metadata_path, path, &object_storage, &result, &mutex, &log, &settings, &metadata_key_prefix] + runner([remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] { setThreadName("PlainRWMetaLoad"); @@ -99,8 +100,8 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, auto remote_path = std::filesystem::path(std::move(suffix)); std::pair res; { - std::lock_guard lock(mutex); - res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); + std::lock_guard lock(result->mutex); + res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); } /// This can happen if table replication is enabled, then the same local path is written @@ -117,10 +118,13 @@ InMemoryPathMap::Map loadPathPrefixMap(const std::string & metadata_key_prefix, } runner.waitForAllToFinishAndRethrowFirstError(); - LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result.size()); + { + SharedLockGuard lock(result->mutex); + LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size()); - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::add(metric, result.size()); + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::add(metric, result->map.size()); + } return result; } @@ -128,14 +132,14 @@ void getDirectChildrenOnDiskImpl( const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path, - const InMemoryPathMap::Map & local_path_prefixes, - SharedMutex & shared_mutex, + const InMemoryPathMap & path_map, std::unordered_set & result) { /// Directories are retrieved from the in-memory path map. { - std::shared_lock lock(shared_mutex); - auto end_it = local_path_prefixes.end(); + SharedLockGuard lock(path_map.mutex); + const auto & local_path_prefixes = path_map.map; + const auto end_it = local_path_prefixes.end(); for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) { const auto & [k, _] = std::make_tuple(it->first.string(), it->second); @@ -179,7 +183,7 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita ObjectStoragePtr object_storage_, String storage_path_prefix_) : MetadataStorageFromPlainObjectStorage(object_storage_, storage_path_prefix_) , metadata_key_prefix(DB::getMetadataKeyPrefix(object_storage)) - , path_map(std::make_shared(loadPathPrefixMap(metadata_key_prefix, object_storage))) + , path_map(loadPathPrefixMap(metadata_key_prefix, object_storage)) { if (object_storage->isWriteOnce()) throw Exception( @@ -261,7 +265,7 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, getPathMap()->map, getPathMap()->mutex, result); + getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result); } } From db13ba2c488303e90717fbcc5adf5304241ac474 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 13 Jul 2024 06:51:40 +0000 Subject: [PATCH 0655/1170] style fix --- .../CommonPathPrefixKeyGenerator.cpp | 2 +- ...torageFromPlainObjectStorageOperations.cpp | 2 +- ...torageFromPlainRewritableObjectStorage.cpp | 91 ++++++++++--------- 3 files changed, 48 insertions(+), 47 deletions(-) diff --git a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp index 1d041626a7e..521d5c037ab 100644 --- a/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp +++ b/src/Disks/ObjectStorages/CommonPathPrefixKeyGenerator.cpp @@ -1,8 +1,8 @@ #include #include -#include #include +#include #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 8a06b204cfc..76090411bb9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,6 +1,6 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" -#include "Common/SharedLockGuard.h" #include +#include "Common/SharedLockGuard.h" #include #include diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 22e73e36372..dba63bba321 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -7,10 +7,10 @@ #include #include #include -#include -#include #include "Common/SharedLockGuard.h" #include "Common/SharedMutex.h" +#include +#include #include "CommonPathPrefixKeyGenerator.h" @@ -67,54 +67,55 @@ std::shared_ptr loadPathPrefixMap(const std::string & metadata_ if (remote_metadata_path.filename() != PREFIX_PATH_FILE_NAME) continue; - runner([remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] - { - setThreadName("PlainRWMetaLoad"); - - StoredObject object{path}; - String local_path; - - try + runner( + [remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] { - auto read_buf = object_storage->readObject(object, settings); - readStringUntilEOF(local_path, *read_buf); - } + setThreadName("PlainRWMetaLoad"); + + StoredObject object{path}; + String local_path; + + try + { + auto read_buf = object_storage->readObject(object, settings); + readStringUntilEOF(local_path, *read_buf); + } #if USE_AWS_S3 - catch (const S3Exception & e) - { - /// It is ok if a directory was removed just now. - /// We support attaching a filesystem that is concurrently modified by someone else. - if (e.getS3ErrorCode() == Aws::S3::S3Errors::NO_SUCH_KEY) - return; - throw; - } + catch (const S3Exception & e) + { + /// It is ok if a directory was removed just now. + /// We support attaching a filesystem that is concurrently modified by someone else. + if (e.getS3ErrorCode() == Aws::S3::S3Errors::NO_SUCH_KEY) + return; + throw; + } #endif - catch (...) - { - throw; - } + catch (...) + { + throw; + } - chassert(remote_metadata_path.has_parent_path()); - chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); - auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); - auto remote_path = std::filesystem::path(std::move(suffix)); - std::pair res; - { - std::lock_guard lock(result->mutex); - res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); - } + chassert(remote_metadata_path.has_parent_path()); + chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); + auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); + auto remote_path = std::filesystem::path(std::move(suffix)); + std::pair res; + { + std::lock_guard lock(result->mutex); + res = result->map.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path()); + } - /// This can happen if table replication is enabled, then the same local path is written - /// in `prefix.path` of each replica. - /// TODO: should replicated tables (e.g., RMT) be explicitly disallowed? - if (!res.second) - LOG_WARNING( - log, - "The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'", - local_path, - res.first->second, - remote_path.parent_path().string()); - }); + /// This can happen if table replication is enabled, then the same local path is written + /// in `prefix.path` of each replica. + /// TODO: should replicated tables (e.g., RMT) be explicitly disallowed? + if (!res.second) + LOG_WARNING( + log, + "The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'", + local_path, + res.first->second, + remote_path.parent_path().string()); + }); } runner.waitForAllToFinishAndRethrowFirstError(); From 912bddf86f53f207b76ba453e43b6724b24ef6df Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:49:37 -0700 Subject: [PATCH 0656/1170] Update src/Disks/ObjectStorages/InMemoryPathMap.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/ObjectStorages/InMemoryPathMap.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index dcd28dfaf6c..2ac291dfaf0 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -21,6 +21,7 @@ struct InMemoryPathMap return path1 < path2; } }; + /// Local -> Remote path. using Map = std::map; mutable SharedMutex mutex; Map map TSA_GUARDED_BY(mutex); From 727f5ed108e3b92c81d4ed295e0de438de8bae2b Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:49:57 -0700 Subject: [PATCH 0657/1170] Update src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index dba63bba321..3380dec60ca 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -29,7 +29,7 @@ constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; constexpr auto METADATA_PATH_TOKEN = "__meta/"; /// Use a separate layout for metadata iff: -/// 1. The disk endpoint does not contain objects, OR +/// 1. The disk endpoint does not contain any objects yet (empty), OR /// 2. The metadata is already stored behind a separate endpoint. /// Otherwise, store metadata along with regular data for backward compatibility. std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) From 4c78531c9c0681a84309e02ecfde17a36f1c1ad5 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sun, 14 Jul 2024 20:50:20 -0700 Subject: [PATCH 0658/1170] Update src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 3380dec60ca..40aed32c047 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -28,7 +28,7 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; constexpr auto METADATA_PATH_TOKEN = "__meta/"; -/// Use a separate layout for metadata iff: +/// Use a separate layout for metadata if: /// 1. The disk endpoint does not contain any objects yet (empty), OR /// 2. The metadata is already stored behind a separate endpoint. /// Otherwise, store metadata along with regular data for backward compatibility. From 3b986ef3400021cf18797a7872b926dcc191b547 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 05:50:42 +0000 Subject: [PATCH 0659/1170] address feedback: useSeparateLayoutForMetadata --- ...StorageFromPlainRewritableObjectStorage.cpp | 18 +++++++++++------- ...taStorageFromPlainRewritableObjectStorage.h | 4 +++- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 40aed32c047..39b11d9a3e3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -192,15 +192,15 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita "MetadataStorageFromPlainRewritableObjectStorage is not compatible with write-once storage '{}'", object_storage->getName()); - if (getMetadataKeyPrefix() == object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + /// Use flat directory structure if the metadata is stored separately from the table data. + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } else { - /// Use flat directory structure if the metadata is stored separately from the table data. - auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); + auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } } @@ -216,7 +216,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & if (MetadataStorageFromPlainObjectStorage::exists(path)) return true; - if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto key_prefix = object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize(); return object_storage->existsOrHasAnyChild(key_prefix); @@ -227,7 +227,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string & bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::string & path) const { - if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, getMetadataKeyPrefix()).serialize()) / ""; return object_storage->existsOrHasAnyChild(directory); @@ -249,7 +249,7 @@ std::vector MetadataStorageFromPlainRewritableObjectStorage::listDi getDirectChildrenOnDisk(abs_key, files, std::filesystem::path(path) / "", directories); /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// metadata along with regular files. - if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) + if (useSeparateLayoutForMetadata()) { auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / ""; RelativePathsWithMetadata metadata_files; @@ -269,4 +269,8 @@ void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result); } +bool MetadataStorageFromPlainRewritableObjectStorage::useSeparateLayoutForMetadata() const +{ + return getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix(); +} } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index 8fd147e15b9..82d93e3e7ae 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -24,7 +24,6 @@ public: bool isDirectory(const std::string & path) const override; std::vector listDirectory(const std::string & path) const override; - protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } @@ -33,6 +32,9 @@ protected: const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const; + +private: + bool useSeparateLayoutForMetadata() const; }; } From 359b42738a25aa02436c1bebc49d0b751e456ccb Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 06:11:51 +0000 Subject: [PATCH 0660/1170] address feedback: key_prefix -> object_key_prefix --- ...tadataStorageFromPlainObjectStorageOperations.cpp | 12 ++++++------ ...MetadataStorageFromPlainObjectStorageOperations.h | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 76090411bb9..31fb8c7ef97 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -22,9 +22,9 @@ namespace constexpr auto PREFIX_PATH_FILE_NAME = "prefix.path"; -ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const std::string & metadata_key_prefix) +ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, const std::string & metadata_key_prefix) { - auto prefix = std::filesystem::path(metadata_key_prefix) / key_prefix; + auto prefix = std::filesystem::path(metadata_key_prefix) / object_key_prefix; return ObjectStorageKey::createAsRelative(prefix.string(), PREFIX_PATH_FILE_NAME); } } @@ -35,7 +35,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) - , key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* key_prefix */).serialize()) + , object_key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* object_key_prefix */).serialize()) { } @@ -47,7 +47,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: return; } - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); LOG_TRACE( getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), @@ -68,7 +68,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix)); + [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(object_key_prefix)); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -85,7 +85,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); + auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); if (write_finalized) { diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 3ac0ffef8d2..02305767faf 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -17,7 +17,7 @@ private: InMemoryPathMap & path_map; ObjectStoragePtr object_storage; const std::string metadata_key_prefix; - const std::string key_prefix; + const std::string object_key_prefix; bool write_created = false; bool write_finalized = false; From 79a8cbe0c595e877a750c26cc27a8c68202279c7 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 06:16:00 +0000 Subject: [PATCH 0661/1170] address feedback: documentation --- .../MetadataStorageFromPlainObjectStorageOperations.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 31fb8c7ef97..be5168c5385 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -55,7 +55,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: path, metadata_object_key.serialize()); - auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( metadata_object, WriteMode::Rewrite, @@ -137,7 +137,8 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo auto metadata_object_key = createMetadataObjectKey(remote_path, metadata_key_prefix); - auto metadata_object = StoredObject(metadata_object_key.serialize(), expected_path / PREFIX_PATH_FILE_NAME); + auto metadata_object + = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ expected_path / PREFIX_PATH_FILE_NAME); if (validate_content) { @@ -220,7 +221,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation"), "Removing directory '{}'", path); auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); - auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); + auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME); object_storage->removeObject(metadata_object); { From 42bd49dae6244590ba406ad502083bf610276eb9 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 07:05:51 +0000 Subject: [PATCH 0662/1170] address feedback: parent_path() for directories --- ...torageFromPlainObjectStorageOperations.cpp | 30 ++++++++++++------- ...aStorageFromPlainObjectStorageOperations.h | 4 ++- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index be5168c5385..c0e3f8e1fc9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -31,7 +31,7 @@ ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path(std::move(path_)) + : path((chassert(path_.string().ends_with('/')), std::move(path_))) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) @@ -41,9 +41,11 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) { + /// parent_path() removes the trailing '/' + const auto base_path = path.parent_path(); { SharedLockGuard lock(path_map.mutex); - if (path_map.map.contains(path.parent_path())) + if (path_map.map.contains(base_path)) return; } @@ -68,7 +70,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(object_key_prefix)); + [[maybe_unused]] auto result = map.emplace(base_path, std::move(object_key_prefix)); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -89,9 +91,10 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un if (write_finalized) { + const auto base_path = path.parent_path(); { std::lock_guard lock(path_map.mutex); - path_map.map.erase(path.parent_path()); + path_map.map.erase(base_path); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); @@ -108,8 +111,8 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path_from(std::move(path_from_)) - , path_to(std::move(path_to_)) + : path_from((chassert(path_from_.string().ends_with('/')), std::move(path_from_))) + , path_to((chassert(path_to_.string().ends_with('/')), std::move(path_to_))) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) @@ -123,6 +126,7 @@ std::unique_ptr MetadataStorageFromPlainObjectStorageMo { SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; + /// parent_path() removes the trailing '/'. auto expected_it = map.find(expected_path.parent_path()); if (expected_it == map.end()) throw Exception( @@ -174,10 +178,14 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u writeString(path_to.string(), *write_buf); write_buf->finalize(); + /// parent_path() removes the trailing '/'. + auto base_path_to = path_to.parent_path(); + auto base_path_from = path_from.parent_path(); + { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped()); + [[maybe_unused]] auto result = map.emplace(base_path_to, map.extract(base_path_from).mapped()); chassert(result.second); } @@ -203,16 +211,18 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) + : path((chassert(path_.string().ends_with('/')), std::move(path_))), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) { + /// parent_path() removes the trailing '/' + const auto base_path = path.parent_path(); { SharedLockGuard lock(path_map.mutex); auto & map = path_map.map; - auto path_it = map.find(path.parent_path()); + auto path_it = map.find(base_path); if (path_it == map.end()) return; key_prefix = path_it->second; @@ -227,7 +237,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - map.erase(path.parent_path()); + map.erase(base_path); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 02305767faf..93ebe668d56 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -23,8 +23,8 @@ private: bool write_finalized = false; public: - // Assuming that paths are normalized. MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( + /// path_ must end with a trailing '/'. std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, @@ -51,6 +51,7 @@ private: public: MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( + /// Both path_from_ and path_to_ must end with a trailing '/'. std::filesystem::path && path_from_, std::filesystem::path && path_to_, InMemoryPathMap & path_map_, @@ -76,6 +77,7 @@ private: public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( + /// path_ must end with a trailing '/'. std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, From 41fc84bb2df441d117681c589dbea5c516ed4748 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 15 Jul 2024 21:16:27 +0000 Subject: [PATCH 0663/1170] fix build --- src/Disks/ObjectStorages/InMemoryPathMap.h | 1 + ...taStorageFromPlainObjectStorageOperations.cpp | 16 ++++++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index 2ac291dfaf0..e319c187ca7 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index c0e3f8e1fc9..bfd203ef2e0 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,10 +1,10 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" #include -#include "Common/SharedLockGuard.h" #include #include #include +#include #include namespace DB @@ -31,12 +31,13 @@ ObjectStorageKey createMetadataObjectKey(const std::string & object_key_prefix, MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path((chassert(path_.string().ends_with('/')), std::move(path_))) + : path(std::move(path_)) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) , object_key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* object_key_prefix */).serialize()) { + chassert(path.string().ends_with('/')); } void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock &) @@ -70,7 +71,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; - [[maybe_unused]] auto result = map.emplace(base_path, std::move(object_key_prefix)); + [[maybe_unused]] auto result = map.emplace(base_path, object_key_prefix); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -111,12 +112,14 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path_from((chassert(path_from_.string().ends_with('/')), std::move(path_from_))) - , path_to((chassert(path_to_.string().ends_with('/')), std::move(path_to_))) + : path_from(std::move(path_from_)) + , path_to(std::move(path_to_)) , path_map(path_map_) , object_storage(object_storage_) , metadata_key_prefix(metadata_key_prefix_) { + chassert(path_from.string().ends_with('/')); + chassert(path_to.string().ends_with('/')); } std::unique_ptr MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( @@ -211,8 +214,9 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_) - : path((chassert(path_.string().ends_with('/')), std::move(path_))), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) + : path(std::move(path_)), path_map(path_map_), object_storage(object_storage_), metadata_key_prefix(metadata_key_prefix_) { + chassert(path.string().ends_with('/')); } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock & /* metadata_lock */) From 9b4e02e8dabb649076389ee96a271da025913ddf Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 16 Jul 2024 22:48:37 +0000 Subject: [PATCH 0664/1170] fix macOs build --- src/Disks/ObjectStorages/InMemoryPathMap.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/InMemoryPathMap.h b/src/Disks/ObjectStorages/InMemoryPathMap.h index e319c187ca7..a9859d5e2b8 100644 --- a/src/Disks/ObjectStorages/InMemoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryPathMap.h @@ -25,7 +25,13 @@ struct InMemoryPathMap /// Local -> Remote path. using Map = std::map; mutable SharedMutex mutex; - Map map TSA_GUARDED_BY(mutex); + +#ifdef OS_LINUX + Map TSA_GUARDED_BY(mutex) map; +/// std::shared_mutex may not be annotated with the 'capability' attribute in libcxx. +#else + Map map; +#endif }; } From 0ec292a65f190f69c24420d5ca85d5658bffba0a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:32:01 -0400 Subject: [PATCH 0665/1170] 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 774cba09dfd4ab347d05caf45f8135a3a51771c3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 08:48:41 +0200 Subject: [PATCH 0666/1170] Fix flaky test_replicated_table_attach --- .../test_replicated_table_attach/configs/config.xml | 2 +- tests/integration/test_replicated_table_attach/test.py | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml index fea3eab4126..3f72f638776 100644 --- a/tests/integration/test_replicated_table_attach/configs/config.xml +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -1,6 +1,6 @@ 1 - 5 + 3 diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index de60b7ec291..4fe8064b26a 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -80,4 +80,8 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): assert_values() # check that we activate it in the end - node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + node.query_with_retry( + "INSERT INTO replicated_table_partitioned VALUES(20, 30)", + retry_count=20, + sleep_time=3, + ) From 797144270b3e20e9e4306949bde95c9a9a32c5e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 07:09:39 +0000 Subject: [PATCH 0667/1170] Update version_date.tsv and changelogs after v24.4.4.113-stable --- docs/changelogs/v24.4.4.113-stable.md | 73 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 74 insertions(+) create mode 100644 docs/changelogs/v24.4.4.113-stable.md diff --git a/docs/changelogs/v24.4.4.113-stable.md b/docs/changelogs/v24.4.4.113-stable.md new file mode 100644 index 00000000000..1f8a221a0a2 --- /dev/null +++ b/docs/changelogs/v24.4.4.113-stable.md @@ -0,0 +1,73 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.4.113-stable (d63a54957bd) FIXME as compared to v24.4.3.25-stable (a915dd4eda4) + +#### Improvement +* Backported in [#65884](https://github.com/ClickHouse/ClickHouse/issues/65884): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65303](https://github.com/ClickHouse/ClickHouse/issues/65303): Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#65894](https://github.com/ClickHouse/ClickHouse/issues/65894): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65372](https://github.com/ClickHouse/ClickHouse/issues/65372): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#66883](https://github.com/ClickHouse/ClickHouse/issues/66883): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65435](https://github.com/ClickHouse/ClickHouse/issues/65435): Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65448](https://github.com/ClickHouse/ClickHouse/issues/65448): Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65710](https://github.com/ClickHouse/ClickHouse/issues/65710): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66689](https://github.com/ClickHouse/ClickHouse/issues/66689): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67499](https://github.com/ClickHouse/ClickHouse/issues/67499): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65353](https://github.com/ClickHouse/ClickHouse/issues/65353): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65060](https://github.com/ClickHouse/ClickHouse/issues/65060): Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.`. [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65329](https://github.com/ClickHouse/ClickHouse/issues/65329): Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Backported in [#64833](https://github.com/ClickHouse/ClickHouse/issues/64833): Fix bug which could lead to non-working TTLs with expressions. [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Backported in [#65086](https://github.com/ClickHouse/ClickHouse/issues/65086): Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65540](https://github.com/ClickHouse/ClickHouse/issues/65540): Fix crash for `ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY`. [#64957](https://github.com/ClickHouse/ClickHouse/pull/64957) ([pufit](https://github.com/pufit)). +* Backported in [#65578](https://github.com/ClickHouse/ClickHouse/issues/65578): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65161](https://github.com/ClickHouse/ClickHouse/issues/65161): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65616](https://github.com/ClickHouse/ClickHouse/issues/65616): Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#65730](https://github.com/ClickHouse/ClickHouse/issues/65730): Eliminate injective function in argument of functions `uniq*` recursively. This used to work correctly but was broken in the new analyzer. [#65140](https://github.com/ClickHouse/ClickHouse/pull/65140) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65668](https://github.com/ClickHouse/ClickHouse/issues/65668): Disable `non-intersecting-parts` optimization for queries with `FINAL` in case of `read-in-order` optimization was enabled. This could lead to an incorrect query result. As a workaround, disable `do_not_merge_across_partitions_select_final` and `split_parts_ranges_into_intersecting_and_non_intersecting_final` before this fix is merged. [#65505](https://github.com/ClickHouse/ClickHouse/pull/65505) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65786](https://github.com/ClickHouse/ClickHouse/issues/65786): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65810](https://github.com/ClickHouse/ClickHouse/issues/65810): Fix invalid exceptions in function `parseDateTime` with `%F` and `%D` placeholders. [#65768](https://github.com/ClickHouse/ClickHouse/pull/65768) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65931](https://github.com/ClickHouse/ClickHouse/issues/65931): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65826](https://github.com/ClickHouse/ClickHouse/issues/65826): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66299](https://github.com/ClickHouse/ClickHouse/issues/66299): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66326](https://github.com/ClickHouse/ClickHouse/issues/66326): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66153](https://github.com/ClickHouse/ClickHouse/issues/66153): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66459](https://github.com/ClickHouse/ClickHouse/issues/66459): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66224](https://github.com/ClickHouse/ClickHouse/issues/66224): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66267](https://github.com/ClickHouse/ClickHouse/issues/66267): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66678](https://github.com/ClickHouse/ClickHouse/issues/66678): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66603](https://github.com/ClickHouse/ClickHouse/issues/66603): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66358](https://github.com/ClickHouse/ClickHouse/issues/66358): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66971](https://github.com/ClickHouse/ClickHouse/issues/66971): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66968](https://github.com/ClickHouse/ClickHouse/issues/66968): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66719](https://github.com/ClickHouse/ClickHouse/issues/66719): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66950](https://github.com/ClickHouse/ClickHouse/issues/66950): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66947](https://github.com/ClickHouse/ClickHouse/issues/66947): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67631](https://github.com/ClickHouse/ClickHouse/issues/67631): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67195](https://github.com/ClickHouse/ClickHouse/issues/67195): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67377](https://github.com/ClickHouse/ClickHouse/issues/67377): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67240](https://github.com/ClickHouse/ClickHouse/issues/67240): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#67574](https://github.com/ClickHouse/ClickHouse/issues/67574): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65410](https://github.com/ClickHouse/ClickHouse/issues/65410): Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65903](https://github.com/ClickHouse/ClickHouse/issues/65903): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66385](https://github.com/ClickHouse/ClickHouse/issues/66385): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66424](https://github.com/ClickHouse/ClickHouse/issues/66424): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66542](https://github.com/ClickHouse/ClickHouse/issues/66542): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66857](https://github.com/ClickHouse/ClickHouse/issues/66857): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66873](https://github.com/ClickHouse/ClickHouse/issues/66873): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67057](https://github.com/ClickHouse/ClickHouse/issues/67057): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66944](https://github.com/ClickHouse/ClickHouse/issues/66944): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67250](https://github.com/ClickHouse/ClickHouse/issues/67250): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67410](https://github.com/ClickHouse/ClickHouse/issues/67410): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index b1391c2d781..7b5dcda82e3 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 +v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 From dc65c0aa078cf06357291c0fe68f6c035698320f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 2 Aug 2024 07:15:40 +0000 Subject: [PATCH 0668/1170] Fix doc for parallel test execution Copy-pasterino strikes again. I forgot to remove the single quote. With it, pytest thinks the whole argument is a file: (no name '/ClickHouse/tests/integration/test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' in any of []) --- tests/integration/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index a8deb97b526..85146c79b1e 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -142,7 +142,7 @@ of parallel workers for `pytest-xdist`. $ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/programs/server/ $ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/programs/clickhouse $ export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=$HOME/ClickHouse/programs/clickhouse-odbc-bridge -$ ./runner 'test_storage_s3_queue/test.py::test_max_set_age -- --count 10 -n 5' +$ ./runner test_storage_s3_queue/test.py::test_max_set_age --count 10 -n 5 Start tests =============================================================================== test session starts ================================================================================ platform linux -- Python 3.10.12, pytest-7.4.4, pluggy-1.5.0 -- /usr/bin/python3 From 9c05a0ad5a0269af02ae2234e1d01dc3ce64bce2 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 09:34:32 +0100 Subject: [PATCH 0669/1170] rm dirs in test_storage_delta --- tests/integration/test_storage_delta/test.py | 28 +++++++------------- 1 file changed, 10 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 384b8296f66..92a870ab360 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -8,6 +8,7 @@ import os import json import time import glob +import shutil import pyspark import delta @@ -52,15 +53,6 @@ def get_spark(): return builder.master("local").getOrCreate() -def remove_local_directory_contents(full_path): - for path in glob.glob(f"{full_path}/**"): - if os.path.isfile(path): - os.unlink(path) - else: - remove_local_directory_contents(path) - os.rmdir(path) - - @pytest.fixture(scope="module") def started_cluster(): try: @@ -179,7 +171,7 @@ def test_single_log_file(started_cluster): ) os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_partition_by(started_cluster): @@ -203,7 +195,7 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_checkpoint(started_cluster): @@ -280,7 +272,7 @@ def test_checkpoint(started_cluster): ).strip() ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") spark.sql(f"DROP TABLE {TABLE_NAME}") @@ -321,7 +313,7 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_metadata(started_cluster): @@ -357,7 +349,7 @@ def test_metadata(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_types(started_cluster): @@ -431,7 +423,7 @@ def test_types(started_cluster): ] ) - remove_local_directory_contents(f"/{result_file}") + shutil.rmtree(f"/{result_file}") spark.sql(f"DROP TABLE {TABLE_NAME}") @@ -496,7 +488,7 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_restart_broken_table_function(started_cluster): @@ -553,7 +545,7 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 os.unlink(parquet_data_path) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") def test_partition_columns(started_cluster): @@ -753,5 +745,5 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini == 1 ) - remove_local_directory_contents(f"/{TABLE_NAME}") + shutil.rmtree(f"/{TABLE_NAME}") spark.sql(f"DROP TABLE {TABLE_NAME}") From 01ca36cb5a157ab961dbd4460acc7e2ebb37e72a Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 09:37:47 +0100 Subject: [PATCH 0670/1170] empty From 27f4e1808e4cf299cd8eaf4a19c3bb979aa4e5bd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Aug 2024 10:43:02 +0200 Subject: [PATCH 0671/1170] Update tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh --- .../0_stateless/02434_cancel_insert_when_client_dies.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 45f4194104e..dca8dae22c3 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 @@ -10,10 +10,7 @@ export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE -$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A - settings non_replicated_deduplication_window=1000 - , merge_tree_clear_old_temporary_directories_interval_seconds = 1 - ;" +$CLICKHOUSE_CLIENT -q "create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000, merge_tree_clear_old_temporary_directories_interval_seconds = 1;" $CLICKHOUSE_CLIENT -q "create table dedup_dist(A Int64) Engine = Distributed('test_cluster_one_shard_two_replicas', currentDatabase(), dedup_test)" function insert_data From 6c8f458b0bf9981068c7fecfdd9cef627406419b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 11:13:41 +0200 Subject: [PATCH 0672/1170] Fix reloading SQL UDFs with UNION --- .../UserDefinedSQLFunctionFactory.cpp | 8 +++++-- .../UserDefinedSQLObjectsDiskStorage.cpp | 6 ++--- .../UserDefinedSQLObjectsDiskStorage.h | 1 - .../UserDefinedSQLObjectsStorageBase.cpp | 15 +++++++++--- .../UserDefinedSQLObjectsStorageBase.h | 4 ++++ .../UserDefinedSQLObjectsZooKeeperStorage.cpp | 2 +- .../UserDefinedSQLObjectsZooKeeperStorage.h | 2 -- .../NormalizeSelectWithUnionQueryVisitor.h | 2 -- .../test.py | 23 +++++++++++++++++-- .../test.py | 12 ++++++++++ .../03215_udf_with_union.reference | 1 + .../0_stateless/03215_udf_with_union.sql | 14 +++++++++++ 12 files changed, 74 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03215_udf_with_union.reference create mode 100644 tests/queries/0_stateless/03215_udf_with_union.sql diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp index e6796874e50..d0bc812f91d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -9,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -80,13 +82,15 @@ namespace validateFunctionRecursiveness(*function_body, name); } - ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query) + ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query, const ContextPtr & context) { auto ptr = create_function_query.clone(); auto & res = typeid_cast(*ptr); res.if_not_exists = false; res.or_replace = false; FunctionNameNormalizer::visit(res.function_core.get()); + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(res.function_core); return ptr; } } @@ -125,7 +129,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeUnregistered(const ContextPtr & co bool UserDefinedSQLFunctionFactory::registerFunction(const ContextMutablePtr & context, const String & function_name, ASTPtr create_function_query, bool throw_if_exists, bool replace_if_exists) { checkCanBeRegistered(context, function_name, *create_function_query); - create_function_query = normalizeCreateFunctionQuery(*create_function_query); + create_function_query = normalizeCreateFunctionQuery(*create_function_query, context); try { diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp index 4c004d2537c..8910b45e79d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp @@ -1,7 +1,7 @@ #include "Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h" -#include "Functions/UserDefined/UserDefinedSQLFunctionFactory.h" -#include "Functions/UserDefined/UserDefinedSQLObjectType.h" +#include +#include #include #include @@ -54,7 +54,7 @@ namespace } UserDefinedSQLObjectsDiskStorage::UserDefinedSQLObjectsDiskStorage(const ContextPtr & global_context_, const String & dir_path_) - : global_context(global_context_) + : UserDefinedSQLObjectsStorageBase(global_context_) , dir_path{makeDirectoryPathCanonical(dir_path_)} , log{getLogger("UserDefinedSQLObjectsLoaderFromDisk")} { diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h index ae0cbd0c589..cafbd140598 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.h @@ -42,7 +42,6 @@ private: ASTPtr tryLoadObject(UserDefinedSQLObjectType object_type, const String & object_name, const String & file_path, bool check_file_exists); String getFilePath(UserDefinedSQLObjectType object_type, const String & object_name) const; - ContextPtr global_context; String dir_path; LoggerPtr log; std::atomic objects_loaded = false; diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp index f251d11789f..225e919301d 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.cpp @@ -2,7 +2,10 @@ #include +#include +#include #include +#include #include namespace DB @@ -17,18 +20,24 @@ namespace ErrorCodes namespace { -ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query) +ASTPtr normalizeCreateFunctionQuery(const IAST & create_function_query, const ContextPtr & context) { auto ptr = create_function_query.clone(); auto & res = typeid_cast(*ptr); res.if_not_exists = false; res.or_replace = false; FunctionNameNormalizer::visit(res.function_core.get()); + NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(res.function_core); return ptr; } } +UserDefinedSQLObjectsStorageBase::UserDefinedSQLObjectsStorageBase(ContextPtr global_context_) + : global_context(std::move(global_context_)) +{} + ASTPtr UserDefinedSQLObjectsStorageBase::get(const String & object_name) const { std::lock_guard lock(mutex); @@ -148,7 +157,7 @@ void UserDefinedSQLObjectsStorageBase::setAllObjects(const std::vector normalized_functions; for (const auto & [function_name, create_query] : new_objects) - normalized_functions[function_name] = normalizeCreateFunctionQuery(*create_query); + normalized_functions[function_name] = normalizeCreateFunctionQuery(*create_query, global_context); std::lock_guard lock(mutex); object_name_to_create_object_map = std::move(normalized_functions); @@ -166,7 +175,7 @@ std::vector> UserDefinedSQLObjectsStorageBase::getAllO void UserDefinedSQLObjectsStorageBase::setObject(const String & object_name, const IAST & create_object_query) { std::lock_guard lock(mutex); - object_name_to_create_object_map[object_name] = normalizeCreateFunctionQuery(create_object_query); + object_name_to_create_object_map[object_name] = normalizeCreateFunctionQuery(create_object_query, global_context); } void UserDefinedSQLObjectsStorageBase::removeObject(const String & object_name) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h index cab63a3bfcf..0dbc5586f08 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsStorageBase.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -13,6 +14,7 @@ namespace DB class UserDefinedSQLObjectsStorageBase : public IUserDefinedSQLObjectsStorage { public: + explicit UserDefinedSQLObjectsStorageBase(ContextPtr global_context_); ASTPtr get(const String & object_name) const override; ASTPtr tryGet(const String & object_name) const override; @@ -64,6 +66,8 @@ protected: std::unordered_map object_name_to_create_object_map; mutable std::recursive_mutex mutex; + + ContextPtr global_context; }; } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp index 01e7e3995fa..12c1302a3fe 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp @@ -48,7 +48,7 @@ namespace UserDefinedSQLObjectsZooKeeperStorage::UserDefinedSQLObjectsZooKeeperStorage( const ContextPtr & global_context_, const String & zookeeper_path_) - : global_context{global_context_} + : UserDefinedSQLObjectsStorageBase(global_context_) , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} , watch_queue{std::make_shared>>(std::numeric_limits::max())} diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h index 61002be2bfd..0aa9b198398 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.h @@ -68,8 +68,6 @@ private: void refreshObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); void syncObjects(const zkutil::ZooKeeperPtr & zookeeper, UserDefinedSQLObjectType object_type); - ContextPtr global_context; - zkutil::ZooKeeperCachingGetter zookeeper_getter; String zookeeper_path; std::atomic objects_loaded = false; diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h index b2f55003da5..b642b5def91 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h @@ -4,8 +4,6 @@ #include #include -#include - namespace DB { diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py index e5f6683b90b..92d86a8fd2c 100644 --- a/tests/integration/test_replicated_user_defined_functions/test.py +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -141,6 +141,9 @@ def test_drop_if_exists(): def test_replication(): node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + node1.query( + "CREATE FUNCTION f3 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) assert ( node1.query("SELECT create_query FROM system.functions WHERE name='f2'") @@ -154,7 +157,11 @@ def test_replication(): assert node1.query("SELECT f2(12,3)") == "9\n" assert node2.query("SELECT f2(12,3)") == "9\n" + assert node1.query("SELECT f3()") == "2\n" + assert node2.query("SELECT f3()") == "2\n" + node1.query("DROP FUNCTION f2") + node1.query("DROP FUNCTION f3") assert ( node1.query("SELECT create_query FROM system.functions WHERE name='f2'") == "" ) @@ -214,7 +221,9 @@ def test_reload_zookeeper(): ) # config reloads, but can still work - node1.query("CREATE FUNCTION f2 AS (x, y) -> x - y") + node1.query( + "CREATE FUNCTION f2 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) assert_eq_with_retry( node2, "SELECT name FROM system.functions WHERE name IN ['f1', 'f2'] ORDER BY name", @@ -269,7 +278,7 @@ def test_reload_zookeeper(): TSV(["f1", "f2", "f3"]), ) - assert node2.query("SELECT f1(12, 3), f2(12, 3), f3(12, 3)") == TSV([[15, 9, 4]]) + assert node2.query("SELECT f1(12, 3), f2(), f3(12, 3)") == TSV([[15, 2, 4]]) active_zk_connections = get_active_zk_connections() assert ( @@ -307,3 +316,13 @@ def test_start_without_zookeeper(): "CREATE FUNCTION f1 AS (x, y) -> (x + y)\n", ) node1.query("DROP FUNCTION f1") + + +def test_server_restart(): + node1.query( + "CREATE FUNCTION f1 AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" + ) + assert node1.query("SELECT f1()") == "2\n" + node1.restart_clickhouse() + assert node1.query("SELECT f1()") == "2\n" + node1.query("DROP FUNCTION f1") diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index 986438a4eed..bd491dfa195 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -18,20 +18,25 @@ def started_cluster(): def test_persistence(): create_function_query1 = "CREATE FUNCTION MySum1 AS (a, b) -> a + b" create_function_query2 = "CREATE FUNCTION MySum2 AS (a, b) -> MySum1(a, b) + b" + create_function_query3 = "CREATE FUNCTION MyUnion AS () -> (SELECT sum(s) FROM (SELECT 1 as s UNION ALL SELECT 1 as s))" instance.query(create_function_query1) instance.query(create_function_query2) + instance.query(create_function_query3) assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" + assert instance.query("SELECT MyUnion()") == "2\n" instance.restart_clickhouse() assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" + assert instance.query("SELECT MyUnion()") == "2\n" instance.query("DROP FUNCTION MySum2") instance.query("DROP FUNCTION MySum1") + instance.query("DROP FUNCTION MyUnion") instance.restart_clickhouse() @@ -48,3 +53,10 @@ def test_persistence(): or "Function with name 'MySum2' does not exist. In scope SELECT MySum2(1, 2)" in error_message ) + + error_message = instance.query_and_get_error("SELECT MyUnion()") + assert ( + "Unknown function MyUnion" in error_message + or "Function with name 'MyUnion' does not exist. In scope SELECT MyUnion" + in error_message + ) diff --git a/tests/queries/0_stateless/03215_udf_with_union.reference b/tests/queries/0_stateless/03215_udf_with_union.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03215_udf_with_union.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03215_udf_with_union.sql b/tests/queries/0_stateless/03215_udf_with_union.sql new file mode 100644 index 00000000000..00390c5d930 --- /dev/null +++ b/tests/queries/0_stateless/03215_udf_with_union.sql @@ -0,0 +1,14 @@ +DROP FUNCTION IF EXISTS 03215_udf_with_union; +CREATE FUNCTION 03215_udf_with_union AS () -> ( + SELECT sum(s) + FROM + ( + SELECT 1 AS s + UNION ALL + SELECT 1 AS s + ) +); + +SELECT 03215_udf_with_union(); + +DROP FUNCTION 03215_udf_with_union; From 900e08d6e7c433758d13f5a669c2112bb3856007 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 11:32:09 +0200 Subject: [PATCH 0673/1170] Try fix --- tests/ci/unit_tests_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 716625d7077..6430fa78801 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -180,7 +180,7 @@ def main(): run_command = ( f"docker run --cap-add=SYS_PTRACE --volume={tests_binary}:/unit_tests_dbms " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--volume={test_output}:/test_output {docker_image} ${gdb_enabled}" + f"--volume={test_output}:/test_output {docker_image} {gdb_enabled}" ) run_log_path = test_output / "run.log" From af53ed4c02ba52b3f57e97941b37a5931620d447 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Aug 2024 12:08:49 +0200 Subject: [PATCH 0674/1170] Ping CI From 62f0e09ecbb226ea72b5ee8d812436ef75038e33 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 12:17:08 +0200 Subject: [PATCH 0675/1170] Fix setting changes --- src/Core/SettingsChangesHistory.cpp | 264 +--------------------------- 1 file changed, 2 insertions(+), 262 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2438202f6a3..b6ef654438e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,268 +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.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_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe 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."}, - {"restore_replace_external_table_functions_to_null", false, false, "New setting."}, - {"restore_replace_external_engines_to_null", false, false, "New setting."} - }}, - {"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."}, - {"allow_archive_path_syntax", false, true, "Added new setting to allow disabling archive path syntax."}, - }}, - {"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 +76,7 @@ static std::initializer_list Date: Mon, 29 Jul 2024 12:54:36 +0000 Subject: [PATCH 0676/1170] Trying to fix test_cache_evicted_by_temporary_data and print debug info --- .../config.d/storage_configuration.xml | 6 +-- .../test_temporary_data_in_cache/test.py | 44 ++++++++++++------- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml index 5a087d03266..107864fde0c 100644 --- a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -10,9 +10,9 @@ cache local_disk /tiny_local_cache/ - 10M - 1M - 1M + 12M + 100K + 100K 1 diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index cab134dcce2..abdfb5f4064 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -7,6 +7,9 @@ import fnmatch from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException + +MB = 1024 * 1024 + cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -36,15 +39,28 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(size) FROM system.filesystem_cache").strip() ) - assert get_cache_size() == 0 + def dump_debug_info(): + return "\n".join( + [ + ">>> filesystem_cache <<<", + q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), + ">>> remote_data_paths <<<", + q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), + ">>> tiny_local_cache_local_disk <<<", + q( + "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" + ), + ] + ) - assert get_free_space() > 8 * 1024 * 1024 + assert get_cache_size() == 0, dump_debug_info() + assert get_free_space() > 8 * MB, dump_debug_info() # Codec is NONE to make cache size predictable q( - "CREATE TABLE t1 (x UInt64 CODEC(NONE), y UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + "CREATE TABLE t1 (x UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" ) - q("INSERT INTO t1 SELECT number, number FROM numbers(1024 * 1024)") + q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") # To be sure that nothing is reading the cache and entries for t1 can be evited q("OPTIMIZE TABLE t1 FINAL") @@ -54,11 +70,11 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(x) FROM t1") cache_size_with_t1 = get_cache_size() - assert cache_size_with_t1 > 8 * 1024 * 1024 + assert cache_size_with_t1 > 8 * MB, dump_debug_info() # Almost all disk space is occupied by t1 cache free_space_with_t1 = get_free_space() - assert free_space_with_t1 < 4 * 1024 * 1024 + assert free_space_with_t1 < 4 * MB, dump_debug_info() # Try to sort the table, but fail because of lack of disk space with pytest.raises(QueryRuntimeException) as exc: @@ -76,31 +92,27 @@ def test_cache_evicted_by_temporary_data(start_cluster): # Some data evicted from cache by temporary data cache_size_after_eviction = get_cache_size() - assert cache_size_after_eviction < cache_size_with_t1 + assert cache_size_after_eviction < cache_size_with_t1, dump_debug_info() # Disk space freed, at least 3 MB, because temporary data tried to write 4 MB - assert get_free_space() > free_space_with_t1 + 3 * 1024 * 1024 + assert get_free_space() > free_space_with_t1 + 3 * MB, dump_debug_info() # Read some data to fill the cache again - q("SELECT avg(y) FROM t1") + q("SELECT avg(x) FROM t1") cache_size_with_t1 = get_cache_size() - assert cache_size_with_t1 > 8 * 1024 * 1024, q( - "SELECT * FROM system.filesystem_cache FORMAT Vertical" - ) + assert cache_size_with_t1 > 8 * MB, dump_debug_info() # Almost all disk space is occupied by t1 cache free_space_with_t1 = get_free_space() - assert free_space_with_t1 < 4 * 1024 * 1024, q( - "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" - ) + assert free_space_with_t1 < 4 * MB, dump_debug_info() node.http_query( "SELECT randomPrintableASCII(1024) FROM numbers(8 * 1024) FORMAT TSV", params={"buffer_size": 0, "wait_end_of_query": 1}, ) - assert get_free_space() > free_space_with_t1 + 3 * 1024 * 1024 + assert get_free_space() > free_space_with_t1 + 3 * MB, dump_debug_info() # not enough space for buffering 32 MB with pytest.raises(Exception) as exc: From e2b686efea175e4ddc6472849934aa953f13138d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 12:26:38 +0200 Subject: [PATCH 0677/1170] Fix test --- src/Common/tests/gtest_lsan.cpp | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/src/Common/tests/gtest_lsan.cpp b/src/Common/tests/gtest_lsan.cpp index f6e1984ec58..7fc4ad2749e 100644 --- a/src/Common/tests/gtest_lsan.cpp +++ b/src/Common/tests/gtest_lsan.cpp @@ -14,20 +14,21 @@ /// because of broken getauxval() [1]. /// /// [1]: https://github.com/ClickHouse/ClickHouse/pull/33957 -TEST(Common, LSan) +TEST(SanitizerDeathTest, LSan) { - int sanitizers_exit_code = 1; - - ASSERT_EXIT({ - std::thread leak_in_thread([]() + EXPECT_DEATH( { - void * leak = malloc(4096); - ASSERT_NE(leak, nullptr); - }); - leak_in_thread.join(); + std::thread leak_in_thread( + []() + { + void * leak = malloc(4096); + ASSERT_NE(leak, nullptr); + }); + leak_in_thread.join(); - __lsan_do_leak_check(); - }, ::testing::ExitedWithCode(sanitizers_exit_code), ".*LeakSanitizer: detected memory leaks.*"); + __lsan_do_leak_check(); + }, + ".*LeakSanitizer: detected memory leaks.*"); } #endif From 1c533f714529dd3065cfcdb0d69e5bbd28f51c29 Mon Sep 17 00:00:00 2001 From: skyoct Date: Fri, 2 Aug 2024 18:32:43 +0800 Subject: [PATCH 0678/1170] CI From 064c0eb9587d9dbd1fa81cdbae8554c22dd11734 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Fri, 2 Aug 2024 10:35:32 +0000 Subject: [PATCH 0679/1170] even better healthcheck for ldap --- tests/integration/compose/docker_compose_ldap.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/compose/docker_compose_ldap.yml b/tests/integration/compose/docker_compose_ldap.yml index 1f50b34735d..f49e00400a2 100644 --- a/tests/integration/compose/docker_compose_ldap.yml +++ b/tests/integration/compose/docker_compose_ldap.yml @@ -19,6 +19,7 @@ services: ldapsearch -x -H ldap://localhost:$$LDAP_PORT_NUMBER -D $$LDAP_ADMIN_DN -w $$LDAP_ADMIN_PASSWORD -b $$LDAP_ROOT | grep -c -E "member: cn=j(ohn|ane)doe" | grep 2 >> /dev/null + && cat /run/slapd/slapd.pid interval: 10s retries: 10 timeout: 2s From 02e48436057e45a884a1381e1c9cda9e1fe7de17 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 Jul 2024 09:30:39 +0000 Subject: [PATCH 0680/1170] test_cache_evicted_by_temporary_data drop cache --- .../test_temporary_data_in_cache/test.py | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index abdfb5f4064..87192a20975 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -39,19 +39,21 @@ def test_cache_evicted_by_temporary_data(start_cluster): q("SELECT sum(size) FROM system.filesystem_cache").strip() ) - def dump_debug_info(): - return "\n".join( - [ - ">>> filesystem_cache <<<", - q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), - ">>> remote_data_paths <<<", - q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), - ">>> tiny_local_cache_local_disk <<<", - q( - "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" - ), - ] - ) + dump_debug_info = lambda: "\n".join( + [ + ">>> filesystem_cache <<<", + q("SELECT * FROM system.filesystem_cache FORMAT Vertical"), + ">>> remote_data_paths <<<", + q("SELECT * FROM system.remote_data_paths FORMAT Vertical"), + ">>> tiny_local_cache_local_disk <<<", + q( + "SELECT * FROM system.disks WHERE name = 'tiny_local_cache_local_disk' FORMAT Vertical" + ), + ] + ) + + q("SYSTEM DROP FILESYSTEM CACHE") + q("DROP TABLE IF EXISTS t1 SYNC") assert get_cache_size() == 0, dump_debug_info() assert get_free_space() > 8 * MB, dump_debug_info() @@ -124,4 +126,4 @@ def test_cache_evicted_by_temporary_data(start_cluster): str(exc.value), "*Failed to reserve * for temporary file*" ), exc.value - q("DROP TABLE IF EXISTS t1") + q("DROP TABLE IF EXISTS t1 SYNC") From 97f1f6e22174916cc3b401fcebdf01dcd0fb0107 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 12:46:37 +0200 Subject: [PATCH 0681/1170] Don't hide errors on clickhouse local runs --- docker/test/stateless/stress_tests.lib | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 682da1df837..2a833b17f14 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -308,7 +308,8 @@ function collect_query_and_trace_logs() { for table in query_log trace_log metric_log do - clickhouse-local --config-file=/etc/clickhouse-server/config.xml --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: + # Don't ignore errors here, it leads to ignore sanitizer reports when running clickhouse-local + clickhouse-local --config-file=/etc/clickhouse-server/config.xml --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst done } From e5cf376c4ff3742fb0d3127ef6b50e08180eb153 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 12:56:49 +0200 Subject: [PATCH 0682/1170] Don't attach gdb in ASAN runs of fuzzer, stateless or stress checks --- docker/test/fuzzer/run-fuzzer.sh | 106 +++++++++++++++------------ docker/test/stateless/attach_gdb.lib | 78 +++++++++++--------- 2 files changed, 101 insertions(+), 83 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index b8f967ed9c2..ae1b9e94bed 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -193,53 +193,60 @@ function fuzz kill -0 $server_pid - # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog - # and clickhouse-server can do fork-exec, for example, to run some bridge. - # Do not set nostop noprint for all signals, because some it may cause gdb to hang, - # explicitly ignore non-fatal signals that are used by server. - # Number of SIGRTMIN can be determined only in runtime. - RTMIN=$(kill -l SIGRTMIN) - echo " -set follow-fork-mode parent -handle SIGHUP nostop noprint pass -handle SIGINT nostop noprint pass -handle SIGQUIT nostop noprint pass -handle SIGPIPE nostop noprint pass -handle SIGTERM nostop noprint pass -handle SIGUSR1 nostop noprint pass -handle SIGUSR2 nostop noprint pass -handle SIG$RTMIN nostop noprint pass -info signals -continue -backtrace full -thread apply all backtrace full -info registers -disassemble /s -up -disassemble /s -up -disassemble /s -p \"done\" -detach -quit -" > script.gdb + IS_ASAN=$(clickhouse-client --query "SELECT count() FROM system.build_options WHERE name = 'CXX_FLAGS' AND position('sanitize=address' IN value)") + if [[ "$IS_ASAN" = "1" ]]; + then + echo "ASAN build detected. Not using gdb since it disables LeakSanitizer detections" + else + # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog + # and clickhouse-server can do fork-exec, for example, to run some bridge. + # Do not set nostop noprint for all signals, because some it may cause gdb to hang, + # explicitly ignore non-fatal signals that are used by server. + # Number of SIGRTMIN can be determined only in runtime. + RTMIN=$(kill -l SIGRTMIN) + echo " + set follow-fork-mode parent + handle SIGHUP nostop noprint pass + handle SIGINT nostop noprint pass + handle SIGQUIT nostop noprint pass + handle SIGPIPE nostop noprint pass + handle SIGTERM nostop noprint pass + handle SIGUSR1 nostop noprint pass + handle SIGUSR2 nostop noprint pass + handle SIG$RTMIN nostop noprint pass + info signals + continue + backtrace full + thread apply all backtrace full + info registers + disassemble /s + up + disassemble /s + up + disassemble /s + p \"done\" + detach + quit + " > script.gdb - gdb -batch -command script.gdb -p $server_pid & - sleep 5 - # gdb will send SIGSTOP, spend some time loading debug info, and then send SIGCONT, wait for it (up to send_timeout, 300s) - time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: + gdb -batch -command script.gdb -p $server_pid & + sleep 5 + # gdb will send SIGSTOP, spend some time loading debug info, and then send SIGCONT, wait for it (up to send_timeout, 300s) + time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: + + # Check connectivity after we attach gdb, because it might cause the server + # to freeze, and the fuzzer will fail. In debug build, it can take a lot of time. + for _ in {1..180} + do + if clickhouse-client --query "select 1" + then + break + fi + sleep 1 + done + kill -0 $server_pid # This checks that it is our server that is started and not some other one + fi - # Check connectivity after we attach gdb, because it might cause the server - # to freeze, and the fuzzer will fail. In debug build, it can take a lot of time. - for _ in {1..180} - do - if clickhouse-client --query "select 1" - then - break - fi - sleep 1 - done - kill -0 $server_pid # This checks that it is our server that is started and not some other one echo 'Server started and responded.' setup_logs_replication @@ -264,8 +271,13 @@ quit # The fuzzer_pid belongs to the timeout process. actual_fuzzer_pid=$(ps -o pid= --ppid "$fuzzer_pid") - echo "Attaching gdb to the fuzzer itself" - gdb -batch -command script.gdb -p $actual_fuzzer_pid & + if [[ "$IS_ASAN" = "1" ]]; + then + echo "ASAN build detected. Not using gdb since it disables LeakSanitizer detections" + else + echo "Attaching gdb to the fuzzer itself" + gdb -batch -command script.gdb -p $actual_fuzzer_pid & + fi # Wait for the fuzzer to complete. # Note that the 'wait || ...' thing is required so that the script doesn't diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index d288288bb17..d0cac24481f 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -5,43 +5,49 @@ source /utils.lib function attach_gdb_to_clickhouse() { - # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog - # and clickhouse-server can do fork-exec, for example, to run some bridge. - # Do not set nostop noprint for all signals, because some it may cause gdb to hang, - # explicitly ignore non-fatal signals that are used by server. - # Number of SIGRTMIN can be determined only in runtime. - RTMIN=$(kill -l SIGRTMIN) - echo " -set follow-fork-mode parent -handle SIGHUP nostop noprint pass -handle SIGINT nostop noprint pass -handle SIGQUIT nostop noprint pass -handle SIGPIPE nostop noprint pass -handle SIGTERM nostop noprint pass -handle SIGUSR1 nostop noprint pass -handle SIGUSR2 nostop noprint pass -handle SIG$RTMIN nostop noprint pass -info signals -continue -backtrace full -thread apply all backtrace full -info registers -disassemble /s -up -disassemble /s -up -disassemble /s -p \"done\" -detach -quit -" > script.gdb + IS_ASAN=$(clickhouse-client --query "SELECT count() FROM system.build_options WHERE name = 'CXX_FLAGS' AND position('sanitize=address' IN value)") + if [[ "$IS_ASAN" = "1" ]]; + then + echo "ASAN build detected. Not using gdb since it disables LeakSanitizer detections" + else + # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog + # and clickhouse-server can do fork-exec, for example, to run some bridge. + # Do not set nostop noprint for all signals, because some it may cause gdb to hang, + # explicitly ignore non-fatal signals that are used by server. + # Number of SIGRTMIN can be determined only in runtime. + RTMIN=$(kill -l SIGRTMIN) + echo " + set follow-fork-mode parent + handle SIGHUP nostop noprint pass + handle SIGINT nostop noprint pass + handle SIGQUIT nostop noprint pass + handle SIGPIPE nostop noprint pass + handle SIGTERM nostop noprint pass + handle SIGUSR1 nostop noprint pass + handle SIGUSR2 nostop noprint pass + handle SIG$RTMIN nostop noprint pass + info signals + continue + backtrace full + thread apply all backtrace full + info registers + disassemble /s + up + disassemble /s + up + disassemble /s + p \"done\" + detach + quit + " > script.gdb - # FIXME Hung check may work incorrectly because of attached gdb - # We cannot attach another gdb to get stacktraces if some queries hung - gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & - sleep 5 - # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) - run_with_retry 60 clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" + # FIXME Hung check may work incorrectly because of attached gdb + # We cannot attach another gdb to get stacktraces if some queries hung + gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & + sleep 5 + # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) + run_with_retry 60 clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" + fi } # vi: ft=bash From 092c837119a9be11cfcc85b4696e9a9c74d9bbc8 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 12:13:26 +0100 Subject: [PATCH 0683/1170] randomize table name in test_storage_delta --- tests/integration/test_storage_delta/test.py | 53 +++++++------------- 1 file changed, 17 insertions(+), 36 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 92a870ab360..054b79ff6fe 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -8,7 +8,8 @@ import os import json import time import glob -import shutil +import random +import string import pyspark import delta @@ -53,6 +54,11 @@ def get_spark(): return builder.master("local").getOrCreate() +def randomize_table_name(table_name, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{table_name}{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -152,7 +158,7 @@ def test_single_log_file(started_cluster): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_single_log_file" + TABLE_NAME = randomize_table_name("test_single_log_file") inserted_data = "SELECT number as a, toString(number + 1) as b FROM numbers(100)" parquet_data_path = create_initial_data_file( @@ -170,16 +176,13 @@ def test_single_log_file(started_cluster): inserted_data ) - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_partition_by(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_by" + TABLE_NAME = randomize_table_name("test_partition_by") write_delta_from_df( spark, @@ -195,15 +198,13 @@ def test_partition_by(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 - shutil.rmtree(f"/{TABLE_NAME}") - def test_checkpoint(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_checkpoint" + TABLE_NAME = randomize_table_name("test_checkpoint") write_delta_from_df( spark, @@ -272,16 +273,13 @@ def test_checkpoint(started_cluster): ).strip() ) - shutil.rmtree(f"/{TABLE_NAME}") - spark.sql(f"DROP TABLE {TABLE_NAME}") - def test_multiple_log_files(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_log_files" + TABLE_NAME = randomize_table_name("test_multiple_log_files") write_delta_from_df( spark, generate_data(spark, 0, 100), f"/{TABLE_NAME}", mode="overwrite" @@ -313,15 +311,13 @@ def test_multiple_log_files(started_cluster): "SELECT number, toString(number + 1) FROM numbers(200)" ) - shutil.rmtree(f"/{TABLE_NAME}") - def test_metadata(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata" + TABLE_NAME = randomize_table_name("test_metadata") parquet_data_path = create_initial_data_file( started_cluster, @@ -348,14 +344,11 @@ def test_metadata(started_cluster): create_delta_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_types(started_cluster): - TABLE_NAME = "test_types" + TABLE_NAME = randomize_table_name("test_types") spark = started_cluster.spark_session - result_file = f"{TABLE_NAME}_result_2" + result_file = randomize_table_name(f"{TABLE_NAME}_result_2") delta_table = ( DeltaTable.create(spark) @@ -423,16 +416,13 @@ def test_types(started_cluster): ] ) - shutil.rmtree(f"/{result_file}") - spark.sql(f"DROP TABLE {TABLE_NAME}") - def test_restart_broken(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = "broken" - TABLE_NAME = "test_restart_broken" + TABLE_NAME = randomize_table_name("test_restart_broken") if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) @@ -487,16 +477,13 @@ def test_restart_broken(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_restart_broken_table_function(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = "broken2" - TABLE_NAME = "test_restart_broken_table_function" + TABLE_NAME = randomize_table_name("test_restart_broken_table_function") if not minio_client.bucket_exists(bucket): minio_client.make_bucket(bucket) @@ -544,16 +531,13 @@ def test_restart_broken_table_function(started_cluster): assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - os.unlink(parquet_data_path) - shutil.rmtree(f"/{TABLE_NAME}") - def test_partition_columns(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_columns" + TABLE_NAME = randomize_table_name("test_partition_columns") result_file = f"{TABLE_NAME}" partition_columns = ["b", "c", "d", "e"] @@ -744,6 +728,3 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) - - shutil.rmtree(f"/{TABLE_NAME}") - spark.sql(f"DROP TABLE {TABLE_NAME}") From 7d1e958097e716f3ea1e0b7e51d6dfa575229c4c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 13:32:59 +0200 Subject: [PATCH 0684/1170] Integration tests: fix ports clashing problem --- tests/integration/conftest.py | 44 ++++++++++++++++++ tests/integration/helpers/cluster.py | 67 ++++++++++++++++++++++++---- 2 files changed, 103 insertions(+), 8 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index f4be31cc532..0a47840ede3 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -2,6 +2,8 @@ import logging import os +import socket +import multiprocessing import pytest # pylint:disable=import-error; for style check from helpers.cluster import run_and_check @@ -11,6 +13,7 @@ from helpers.network import _NetworkManager # # [1]: https://github.com/pytest-dev/pytest/issues/5502 logging.raiseExceptions = False +PORTS_PER_WORKER = 50 @pytest.fixture(scope="session", autouse=True) @@ -111,5 +114,46 @@ def pytest_addoption(parser): ) +def get_n_free_ports(total): + ports = [] + + while len(ports) < total: + with socket.socket() as s: + s.bind(("", 0)) + ports.append(s.getsockname()[1]) + + return ports + + def pytest_configure(config): os.environ["INTEGRATION_TESTS_RUN_ID"] = config.option.run_id + + # When running tests without pytest-xdist, + # the `pytest_xdist_setupnodes` hook is not executed + worker_ports = os.getenv("WORKER_FREE_PORTS", None) + if worker_ports is None: + os.environ["WORKER_FREE_PORTS"] = " ".join( + ([str(p) for p in get_n_free_ports(PORTS_PER_WORKER)]) + ) + + +def pytest_xdist_setupnodes(config, specs): + # Find {PORTS_PER_WORKER} * {number of xdist workers} ports and + # allocate pool of {PORTS_PER_WORKER} ports to each worker + + # Get number of xdist workers + num_workers = 1 + if os.environ.get("PYTEST_XDIST_WORKER", "master") == "master": + num_workers = config.getoption("numprocesses", 1) + if num_workers == "auto": + num_workers = multiprocessing.cpu_count() + + # Get free ports which will be distributed across workers + ports = get_n_free_ports(num_workers * PORTS_PER_WORKER) + + # Iterate over specs of workers and add allocated ports to env variable + for i, spec in enumerate(specs): + start_range = i * PORTS_PER_WORKER + spec.env["WORKER_FREE_PORTS"] = " ".join( + ([str(p) for p in ports[start_range : start_range + PORTS_PER_WORKER]]) + ) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6bc0ece63ca..3480a3089fe 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -135,6 +135,52 @@ def get_free_port(): return s.getsockname()[1] +def is_port_free(port: int) -> bool: + try: + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.bind(("", port)) + return True + except socket.error: + return False + + +class PortPoolManager: + """ + This class is used for distribution of ports allocated to single pytest-xdist worker + It can be used by multiple ClickHouseCluster instances + """ + + # Shared between instances + all_ports = None + free_ports = None + + def __init__(self): + self.used_ports = [] + + if self.all_ports is None: + worker_ports = os.getenv("WORKER_FREE_PORTS") + ports = [int(p) for p in worker_ports.split(" ")] + + # Static vars + PortPoolManager.all_ports = ports + PortPoolManager.free_ports = ports + + def get_port(self): + for port in self.free_ports: + if is_port_free(port): + self.free_ports.remove(port) + self.used_ports.append(port) + return port + + raise Exception( + f"No free ports: {self.all_ports}", + ) + + def return_used_ports(self): + self.free_ports.extend(self.used_ports) + self.used_ports.clear() + + def retry_exception(num, delay, func, exception=Exception, *args, **kwargs): """ Retry if `func()` throws, `num` times. @@ -716,62 +762,67 @@ class ClickHouseCluster: .stop() ) + self.port_pool = PortPoolManager() + @property def kafka_port(self): if self._kafka_port: return self._kafka_port - self._kafka_port = get_free_port() + self._kafka_port = self.port_pool.get_port() return self._kafka_port @property def schema_registry_port(self): if self._schema_registry_port: return self._schema_registry_port - self._schema_registry_port = get_free_port() + self._schema_registry_port = self.port_pool.get_port() return self._schema_registry_port @property def schema_registry_auth_port(self): if self._schema_registry_auth_port: return self._schema_registry_auth_port - self._schema_registry_auth_port = get_free_port() + self._schema_registry_auth_port = self.port_pool.get_port() return self._schema_registry_auth_port @property def kerberized_kafka_port(self): if self._kerberized_kafka_port: return self._kerberized_kafka_port - self._kerberized_kafka_port = get_free_port() + self._kerberized_kafka_port = self.port_pool.get_port() return self._kerberized_kafka_port @property def azurite_port(self): if self._azurite_port: return self._azurite_port - self._azurite_port = get_free_port() + self._azurite_port = self.port_pool.get_port() return self._azurite_port @property def mongo_port(self): if self._mongo_port: return self._mongo_port - self._mongo_port = get_free_port() + self._mongo_port = self.port_pool.get_port() return self._mongo_port @property def mongo_no_cred_port(self): if self._mongo_no_cred_port: return self._mongo_no_cred_port - self._mongo_no_cred_port = get_free_port() + self._mongo_no_cred_port = self.port_pool.get_port() return self._mongo_no_cred_port @property def redis_port(self): if self._redis_port: return self._redis_port - self._redis_port = get_free_port() + self._redis_port = self.port_pool.get_port() return self._redis_port + def __exit__(self, exc_type, exc_val, exc_tb): + self.port_pool.return_used_ports() + def print_all_docker_pieces(self): res_networks = subprocess.check_output( f"docker network ls --filter name='{self.project_name}*'", From 7d45529fe8d28a6b39deb32d060343bb5d03b64f Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Fri, 2 Aug 2024 12:35:40 +0100 Subject: [PATCH 0685/1170] randomize query id in test_checking_s3_blobs_paranoid --- .../test_checking_s3_blobs_paranoid/test.py | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index afe8449b44a..c22142046c7 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -4,6 +4,8 @@ import logging import pytest import os import minio +import random +import string from helpers.cluster import ClickHouseCluster from helpers.mock_servers import start_s3_mock @@ -45,6 +47,11 @@ def cluster(): cluster.shutdown() +def randomize_query_id(query_id, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{query_id}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + @pytest.fixture(scope="module") def init_broken_s3(cluster): yield start_s3_mock(cluster, "broken_s3", "8083") @@ -128,7 +135,7 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression broken_s3.setup_at_create_multi_part_upload() - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}") error = node.query_and_get_error( f""" INSERT INTO @@ -170,7 +177,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=1, after=2) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}") error = node.query_and_get_error( f""" INSERT INTO @@ -222,7 +229,7 @@ def test_when_error_is_retried(cluster, broken_s3, action_and_message): broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=3, after=2, action=action) - insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_{action}_RETRIED") node.query( f""" INSERT INTO @@ -251,7 +258,7 @@ def test_when_error_is_retried(cluster, broken_s3, action_and_message): assert s3_errors == 3 broken_s3.setup_at_part_upload(count=1000, after=2, action=action) - insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED_1" + insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_{action}_RETRIED_1") error = node.query_and_get_error( f""" INSERT INTO @@ -286,7 +293,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): action="broken_pipe", ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + insert_query_id = randomize_query_id(f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD") node.query( f""" INSERT INTO @@ -320,7 +327,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): after=2, action="broken_pipe", ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + insert_query_id = randomize_query_id(f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1") error = node.query_and_get_error( f""" INSERT INTO @@ -362,7 +369,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}" ) node.query( @@ -399,7 +406,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action="connection_reset_by_peer", action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}_1" ) error = node.query_and_get_error( @@ -444,7 +451,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}" ) node.query( @@ -482,7 +489,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = ( + insert_query_id = randomize_query_id( f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}_1" ) error = node.query_and_get_error( @@ -522,7 +529,7 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): action="connection_refused", ) - insert_query_id = f"TEST_QUERY_IS_CANCELED_WITH_INF_RETRIES" + insert_query_id = randomize_query_id(f"TEST_QUERY_IS_CANCELED_WITH_INF_RETRIES") request = node.get_query_request( f""" INSERT INTO @@ -580,7 +587,7 @@ def test_adaptive_timeouts(cluster, broken_s3, node_name): count=1000000, ) - insert_query_id = f"TEST_ADAPTIVE_TIMEOUTS_{node_name}" + insert_query_id = randomize_query_id(f"TEST_ADAPTIVE_TIMEOUTS_{node_name}") node.query( f""" INSERT INTO From 34cba1bdda55cdd2409c535be56e4fe6165c894a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 11:46:03 +0000 Subject: [PATCH 0686/1170] Automatic style fix --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index c22142046c7..b995b4d6461 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -135,7 +135,9 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression broken_s3.setup_at_create_multi_part_upload() - insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}") + insert_query_id = randomize_query_id( + f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -177,7 +179,9 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( broken_s3.setup_fake_multpartuploads() broken_s3.setup_at_part_upload(count=1, after=2) - insert_query_id = randomize_query_id(f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}") + insert_query_id = randomize_query_id( + f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" + ) error = node.query_and_get_error( f""" INSERT INTO From d2d8a16ca6c5c3df31a62894fe2bcfb26d570061 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Aug 2024 13:51:03 +0200 Subject: [PATCH 0687/1170] Fix refreshable MVs --- src/Databases/DatabaseOrdinary.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 3ab5d3fa697..8808261654f 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -250,6 +251,8 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables convertMergeTreeToReplicatedIfNeeded(ast, qualified_name, file_name); + NormalizeSelectWithUnionQueryVisitor::Data data{local_context->getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); std::lock_guard lock{metadata.mutex}; metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast}; metadata.total_dictionaries += create_query->is_dictionary; From c9b29ad11351b774d834ada642951a81d33b14e7 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Fri, 2 Aug 2024 13:53:48 +0200 Subject: [PATCH 0688/1170] squash! fix for parallel execution --- tests/integration/test_parquet_page_index/test.py | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_parquet_page_index/test.py b/tests/integration/test_parquet_page_index/test.py index db291e20b74..59dbab09be5 100644 --- a/tests/integration/test_parquet_page_index/test.py +++ b/tests/integration/test_parquet_page_index/test.py @@ -6,9 +6,6 @@ import time cluster = ClickHouseCluster(__file__) path_to_userfiles = "/var/lib/clickhouse/user_files/" -path_to_external_dirs = ( - "/ClickHouse/tests/integration/test_parquet_page_index/_instances" -) node = cluster.add_instance("node", external_dirs=[path_to_userfiles]) @@ -45,7 +42,7 @@ def delete_if_exists(file_path): True, ), ( - "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SELECT number, number+1 FROM system.numbers LIMIT 100 " "INTO OUTFILE '{file_name}' FORMAT Parquet " "SETTINGS output_format_parquet_use_custom_encoder = false, " "output_format_parquet_write_page_index = false;", @@ -54,7 +51,7 @@ def delete_if_exists(file_path): # # default settings: # # output_format_parquet_use_custom_encoder = true ( - "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SELECT number, number+1 FROM system.numbers LIMIT 100 " "INTO OUTFILE '{file_name}' FORMAT Parquet;", False, ), @@ -102,7 +99,7 @@ def test_parquet_page_index_insert_into_table_function_file( ): file_name = f"export{time.time()}.parquet" query = query.format(file_name=file_name) - file_path = f"{path_to_external_dirs}{path_to_userfiles}{file_name}" + file_path = f"{cluster.instances_dir}{path_to_userfiles}{file_name}" delete_if_exists(file_path) assert node.query(query) == "" assert ( From cae76458504f46fefe52c7d93594870d0bac4479 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:00:50 +0200 Subject: [PATCH 0689/1170] Make 02514_null_dictionary_source parallelizable --- tests/queries/0_stateless/02514_null_dictionary_source.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02514_null_dictionary_source.sql b/tests/queries/0_stateless/02514_null_dictionary_source.sql index 74fb57707ff..bfd36042f57 100644 --- a/tests/queries/0_stateless/02514_null_dictionary_source.sql +++ b/tests/queries/0_stateless/02514_null_dictionary_source.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP DICTIONARY IF EXISTS null_dict; CREATE DICTIONARY null_dict ( id UInt64, From 45f54c3633a388075898e3dcabc850dd1347587d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:01:01 +0200 Subject: [PATCH 0690/1170] Make 01086_window_view_cleanup parallelizable --- .../0_stateless/01086_window_view_cleanup.sh | 47 ++++++++++--------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 113bcffb2af..0bce08523e2 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel # Creation of a database with Ordinary engine emits a warning. CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal @@ -12,38 +11,40 @@ opts=( "--allow_experimental_analyzer=0" ) -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 < Date: Fri, 2 Aug 2024 14:03:27 +0200 Subject: [PATCH 0691/1170] Integration tests: fix ports clashing problem 2 --- tests/integration/conftest.py | 36 ++++++++++++++--------------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 0a47840ede3..a386ed53009 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -2,11 +2,9 @@ import logging import os -import socket -import multiprocessing import pytest # pylint:disable=import-error; for style check -from helpers.cluster import run_and_check +from helpers.cluster import run_and_check, is_port_free from helpers.network import _NetworkManager # This is a workaround for a problem with logging in pytest [1]. @@ -114,15 +112,16 @@ def pytest_addoption(parser): ) -def get_n_free_ports(total): +def get_unique_free_ports(total): ports = [] + for port in range(30000, 55000): + if is_port_free(port) and port not in ports: + ports.append(port) - while len(ports) < total: - with socket.socket() as s: - s.bind(("", 0)) - ports.append(s.getsockname()[1]) + if len(ports) == total: + return ports - return ports + raise Exception(f"Can't collect {total} ports. Collected: {len(ports)}") def pytest_configure(config): @@ -132,9 +131,8 @@ def pytest_configure(config): # the `pytest_xdist_setupnodes` hook is not executed worker_ports = os.getenv("WORKER_FREE_PORTS", None) if worker_ports is None: - os.environ["WORKER_FREE_PORTS"] = " ".join( - ([str(p) for p in get_n_free_ports(PORTS_PER_WORKER)]) - ) + master_ports = get_unique_free_ports(PORTS_PER_WORKER) + os.environ["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in master_ports])) def pytest_xdist_setupnodes(config, specs): @@ -142,18 +140,12 @@ def pytest_xdist_setupnodes(config, specs): # allocate pool of {PORTS_PER_WORKER} ports to each worker # Get number of xdist workers - num_workers = 1 - if os.environ.get("PYTEST_XDIST_WORKER", "master") == "master": - num_workers = config.getoption("numprocesses", 1) - if num_workers == "auto": - num_workers = multiprocessing.cpu_count() - + num_workers = len(specs) # Get free ports which will be distributed across workers - ports = get_n_free_ports(num_workers * PORTS_PER_WORKER) + ports = get_unique_free_ports(num_workers * PORTS_PER_WORKER) # Iterate over specs of workers and add allocated ports to env variable for i, spec in enumerate(specs): start_range = i * PORTS_PER_WORKER - spec.env["WORKER_FREE_PORTS"] = " ".join( - ([str(p) for p in ports[start_range : start_range + PORTS_PER_WORKER]]) - ) + per_workrer_ports = ports[start_range : start_range + PORTS_PER_WORKER] + spec.env["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in per_workrer_ports])) From 85cecf990ddefc25ff0903b6f97f84cd46dc471a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:06:51 +0200 Subject: [PATCH 0692/1170] Parallelize 02703_row_policies_for_database_combination --- ...3_row_policies_for_database_combination.sh | 92 +++++++++++++++++++ ..._row_policies_for_database_combination.sql | 88 ------------------ 2 files changed, 92 insertions(+), 88 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policies_for_database_combination.sh delete mode 100644 tests/queries/0_stateless/02703_row_policies_for_database_combination.sql diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh new file mode 100755 index 00000000000..f7b7c814d29 --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh @@ -0,0 +1,92 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery " + +DROP TABLE IF EXISTS 02703_rptable; +DROP TABLE IF EXISTS 02703_rptable_another; +CREATE TABLE 02703_rptable (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x; + +INSERT INTO 02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); + +CREATE TABLE 02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; + + +DROP ROW POLICY IF EXISTS 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; + +-- the test assumes users_without_row_policies_can_read_rows is true + +SELECT 'None'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; +SELECT 'R1: x == 1'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; +SELECT 'R1, R2: (x == 1) OR (x == 2)'; +SELECT * FROM 02703_rptable; + +SELECT 'R1, R2: (x == 2) FROM ANOTHER'; +SELECT * FROM 02703_rptable_another; + +CREATE ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; +SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; +SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; +SELECT * FROM 02703_rptable; + +CREATE ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; +SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +CREATE TABLE 02703_after_rp ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; +SELECT * FROM 02703_after_rp; + +-- does not matter if policies or table are created first +SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM AFTER_RP'; +SELECT * FROM 02703_after_rp; + +SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; +SELECT * FROM 02703_rptable_another; + +DROP ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; +SELECT * FROM 02703_rptable; + +DROP ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +SELECT 'R5: (x >= 2)'; +SELECT * FROM 02703_rptable; + +CREATE TABLE 02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree ORDER BY xx; +SELECT 'Policy not applicable'; +SELECT * FROM 02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query + +DROP ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +SELECT 'None'; +SELECT * FROM 02703_rptable; + +SELECT 'No problematic policy, select works'; +SELECT 'Ok' FROM ${CLICKHOUSE_DATABASE}.02703_unexpected_columns; + +DROP TABLE 02703_rptable; +DROP TABLE 02703_rptable_another; +DROP TABLE 02703_unexpected_columns; + +" diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql deleted file mode 100644 index 8c93fc595ba..00000000000 --- a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sql +++ /dev/null @@ -1,88 +0,0 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS 02703_db; -CREATE DATABASE 02703_db; -DROP TABLE IF EXISTS 02703_db.02703_rptable; -DROP TABLE IF EXISTS 02703_db.02703_rptable_another; -CREATE TABLE 02703_db.02703_rptable (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x; - -INSERT INTO 02703_db.02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); - -CREATE TABLE 02703_db.02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_db.02703_rptable; - - -DROP ROW POLICY IF EXISTS 02703_filter_1 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_2 ON 02703_db.*; -DROP ROW POLICY IF EXISTS 02703_filter_3 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_4 ON 02703_db.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_5 ON 02703_db.*; - --- the test assumes users_without_row_policies_can_read_rows is true - -SELECT 'None'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_1 ON 02703_db.02703_rptable USING x=1 AS permissive TO ALL; -SELECT 'R1: x == 1'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_2 ON 02703_db.* USING x=2 AS permissive TO ALL; -SELECT 'R1, R2: (x == 1) OR (x == 2)'; -SELECT * FROM 02703_db.02703_rptable; - -SELECT 'R1, R2: (x == 2) FROM ANOTHER'; -SELECT * FROM 02703_db.02703_rptable_another; - -CREATE ROW POLICY 02703_filter_3 ON 02703_db.02703_rptable USING x=3 AS permissive TO ALL; -SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_4 ON 02703_db.02703_rptable USING x<=2 AS restrictive TO ALL; -SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE ROW POLICY 02703_filter_5 ON 02703_db.* USING y>=20 AS restrictive TO ALL; -SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE TABLE 02703_db.02703_after_rp ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_db.02703_rptable; -SELECT * FROM 02703_db.02703_after_rp; - --- does not matter if policies or table are created first -SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM AFTER_RP'; -SELECT * FROM 02703_db.02703_after_rp; - -SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; -SELECT * FROM 02703_db.02703_rptable_another; - -DROP ROW POLICY 02703_filter_1 ON 02703_db.02703_rptable; -SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_2 ON 02703_db.*; -SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_3 ON 02703_db.02703_rptable; -SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; -SELECT * FROM 02703_db.02703_rptable; - -DROP ROW POLICY 02703_filter_4 ON 02703_db.02703_rptable; -SELECT 'R5: (x >= 2)'; -SELECT * FROM 02703_db.02703_rptable; - -CREATE TABLE 02703_db.02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree ORDER BY xx; -SELECT 'Policy not applicable'; -SELECT * FROM 02703_db.02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query - -DROP ROW POLICY 02703_filter_5 ON 02703_db.*; -SELECT 'None'; -SELECT * FROM 02703_db.02703_rptable; - -SELECT 'No problematic policy, select works'; -SELECT 'Ok' FROM 02703_db.02703_unexpected_columns; - -DROP TABLE 02703_db.02703_rptable; -DROP TABLE 02703_db.02703_rptable_another; -DROP TABLE 02703_db.02703_unexpected_columns; -DROP DATABASE 02703_db; From f55784c636c1dcb503dadb2f75bd6b586271bf0d Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Aug 2024 12:07:05 +0000 Subject: [PATCH 0693/1170] Fix 03203_client_benchmark_options --- tests/queries/0_stateless/03203_client_benchmark_options.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh index 37a1f2cd3ac..967db056c0b 100755 --- a/tests/queries/0_stateless/03203_client_benchmark_options.sh +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -5,10 +5,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh output=$(${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } +{ number=$(echo "$output" | grep -o "^[0-9]"); [[ -n "$number" && "$number" -ge 2 ]] && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1) -{ echo "$output" | grep -q "^2\." && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } +{ number=$(echo "$output" | grep -o "^[0-9]"); [[ -n "$number" && "$number" -ge 2 ]] && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } output=$(${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1) { echo "$output" | grep -q "^[0-9]\+$" && echo "Ok"; } || { echo "Fail"; echo "'$output'"; } From cb83274d37f8e90ef9c79451ce01a2ba95363158 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:18:31 +0200 Subject: [PATCH 0694/1170] Make 02703_row_policy_for_database parallelizable --- .../02703_row_policy_for_database.reference | 22 +++---- .../02703_row_policy_for_database.sh | 59 +++++++++++++++++++ .../02703_row_policy_for_database.sql | 53 ----------------- 3 files changed, 70 insertions(+), 64 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policy_for_database.sh delete mode 100644 tests/queries/0_stateless/02703_row_policy_for_database.sql diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.reference b/tests/queries/0_stateless/02703_row_policy_for_database.reference index b67ea69ae72..56b0d8c6f20 100644 --- a/tests/queries/0_stateless/02703_row_policy_for_database.reference +++ b/tests/queries/0_stateless/02703_row_policy_for_database.reference @@ -1,20 +1,20 @@ -- row policies for database - -- SHOW CREATE POLICY db1_02703 ON db1_02703.* -CREATE ROW POLICY db1_02703 ON db1_02703.* FOR SELECT USING 1 TO ALL - -- SHOW CREATE POLICY ON db1_02703.* -CREATE ROW POLICY db1_02703 ON db1_02703.* FOR SELECT USING 1 TO ALL -CREATE ROW POLICY tbl1_02703 ON db1_02703.`table` FOR SELECT USING 1 TO ALL - -- SHOW CREATE POLICY ON db1_02703.`*` + -- SHOW CREATE POLICY default ON default.* +CREATE ROW POLICY default_db_policy ON default.* FOR SELECT USING 1 TO ALL + -- SHOW CREATE POLICY ON default.* +CREATE ROW POLICY default_db_policy ON default.* FOR SELECT USING 1 TO ALL +CREATE ROW POLICY default_tb_policy ON default.`table` FOR SELECT USING 1 TO ALL + -- SHOW CREATE POLICY ON default.`*` R1, R2: (x == 1) OR (x == 2) 1 2 Check system.query_log SELECT \'-- row policies for database\'; [] -SELECT \' -- SHOW CREATE POLICY db1_02703 ON db1_02703.*\'; [] -SELECT \' -- SHOW CREATE POLICY ON db1_02703.*\'; [] -SELECT \' -- SHOW CREATE POLICY ON db1_02703.`*`\'; [] +SELECT \' -- SHOW CREATE POLICY default ON default.*\'; [] +SELECT \' -- SHOW CREATE POLICY ON default.*\'; [] +SELECT \' -- SHOW CREATE POLICY ON default.`*`\'; [] SELECT \'R1, R2: (x == 1) OR (x == 2)\'; [] -SELECT * FROM 02703_rqtable_default; ['`02703_filter_11_db` ON default.*','`02703_filter_11` ON default.`02703_rqtable_default`'] +SELECT * FROM 02703_rqtable_default; ['default_filter_11_db_policy ON default.*','default_filter_11_policy ON default.`02703_rqtable_default`'] SELECT \'Check system.query_log\'; [] -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE -CREATE ROW POLICY db2_02703 ON db1_02703.* TO u1_02703 +CREATE ROW POLICY db2_02703 ON default.* TO user_default diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.sh b/tests/queries/0_stateless/02703_row_policy_for_database.sh new file mode 100755 index 00000000000..e94bc7acd5e --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policy_for_database.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CLICKHOUSE_USER="user_$CLICKHOUSE_DATABASE" + +$CLICKHOUSE_CLIENT --multiquery " + +DROP USER IF EXISTS ${CLICKHOUSE_USER}; +CREATE USER ${CLICKHOUSE_USER}; + +CREATE TABLE ${CLICKHOUSE_DATABASE}.02703_rqtable (x UInt8) ENGINE = MergeTree ORDER BY x; +INSERT INTO ${CLICKHOUSE_DATABASE}.02703_rqtable VALUES (1), (2), (3), (4); + +SELECT '-- row policies for database'; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.* USING 1 AS PERMISSIVE TO ALL; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_tb_policy ON ${CLICKHOUSE_DATABASE}.table USING 1 AS PERMISSIVE TO ALL; +SELECT ' -- SHOW CREATE POLICY ${CLICKHOUSE_DATABASE} ON ${CLICKHOUSE_DATABASE}.*'; +SHOW CREATE POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.*; +SELECT ' -- SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.*'; +SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.*; +SELECT ' -- SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.\`*\`'; +SHOW CREATE POLICY ON ${CLICKHOUSE_DATABASE}.\`*\`; +DROP POLICY ${CLICKHOUSE_DATABASE}_db_policy ON ${CLICKHOUSE_DATABASE}.*; +DROP POLICY ${CLICKHOUSE_DATABASE}_tb_policy ON ${CLICKHOUSE_DATABASE}.table; +" + +$CLICKHOUSE_CLIENT --query "CREATE ROW POLICY any_02703 ON *.some_table USING 1 AS PERMISSIVE TO ALL;" 2>&1 | grep -q "SYNTAX_ERROR" + +$CLICKHOUSE_CLIENT --multiquery " +CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; + +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON * USING x=1 AS permissive TO ALL; +CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_policy ON 02703_rqtable_default USING x=2 AS permissive TO ALL; + +INSERT INTO 02703_rqtable_default VALUES (1), (2), (3), (4); + +SELECT 'R1, R2: (x == 1) OR (x == 2)'; +SELECT * FROM 02703_rqtable_default; + +DROP TABLE 02703_rqtable_default; + +SELECT 'Check system.query_log'; +SYSTEM FLUSH LOGS; +SELECT query, used_row_policies FROM system.query_log WHERE current_database == currentDatabase() AND type == 'QueryStart' AND query_kind == 'Select' ORDER BY event_time_microseconds; + +DROP ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON *; +DROP ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_policy ON 02703_rqtable_default; + +USE ${CLICKHOUSE_DATABASE}; +SELECT ' -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE'; +CREATE ROW POLICY db2_02703 ON * TO ${CLICKHOUSE_USER}; +SHOW CREATE POLICY db2_02703 ON *; + +DROP ROW POLICY db2_02703 ON *; + +DROP USER ${CLICKHOUSE_USER}; +" diff --git a/tests/queries/0_stateless/02703_row_policy_for_database.sql b/tests/queries/0_stateless/02703_row_policy_for_database.sql deleted file mode 100644 index 51ce5f4f870..00000000000 --- a/tests/queries/0_stateless/02703_row_policy_for_database.sql +++ /dev/null @@ -1,53 +0,0 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS db1_02703; -DROP USER IF EXISTS u1_02703; -CREATE USER u1_02703; - -CREATE DATABASE db1_02703; - -CREATE TABLE db1_02703.02703_rqtable (x UInt8) ENGINE = MergeTree ORDER BY x; -INSERT INTO db1_02703.02703_rqtable VALUES (1), (2), (3), (4); - - -SELECT '-- row policies for database'; -CREATE ROW POLICY db1_02703 ON db1_02703.* USING 1 AS PERMISSIVE TO ALL; -CREATE ROW POLICY tbl1_02703 ON db1_02703.table USING 1 AS PERMISSIVE TO ALL; -SELECT ' -- SHOW CREATE POLICY db1_02703 ON db1_02703.*'; -SHOW CREATE POLICY db1_02703 ON db1_02703.*; -SELECT ' -- SHOW CREATE POLICY ON db1_02703.*'; -SHOW CREATE POLICY ON db1_02703.*; -SELECT ' -- SHOW CREATE POLICY ON db1_02703.`*`'; -SHOW CREATE POLICY ON db1_02703.`*`; -DROP POLICY db1_02703 ON db1_02703.*; -DROP POLICY tbl1_02703 ON db1_02703.table; - -CREATE ROW POLICY any_02703 ON *.some_table USING 1 AS PERMISSIVE TO ALL; -- { clientError SYNTAX_ERROR } - -CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; - -CREATE ROW POLICY 02703_filter_11_db ON * USING x=1 AS permissive TO ALL; -CREATE ROW POLICY 02703_filter_11 ON 02703_rqtable_default USING x=2 AS permissive TO ALL; - -INSERT INTO 02703_rqtable_default VALUES (1), (2), (3), (4); - -SELECT 'R1, R2: (x == 1) OR (x == 2)'; -SELECT * FROM 02703_rqtable_default; - -DROP TABLE 02703_rqtable_default; - -SELECT 'Check system.query_log'; -SYSTEM FLUSH LOGS; -SELECT query, used_row_policies FROM system.query_log WHERE current_database == currentDatabase() AND type == 'QueryStart' AND query_kind == 'Select' ORDER BY event_time_microseconds; - -DROP ROW POLICY 02703_filter_11_db ON *; -DROP ROW POLICY 02703_filter_11 ON 02703_rqtable_default; - -USE db1_02703; -SELECT ' -- CREATE DATABASE-LEVEL POLICY IN CURRENT DATABASE'; -CREATE ROW POLICY db2_02703 ON * TO u1_02703; -SHOW CREATE POLICY db2_02703 ON *; - -DROP ROW POLICY db2_02703 ON *; - -DROP USER u1_02703; From 829d07c3a54716bb62cd36d4f2fa24da64ea6a0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 14:22:37 +0200 Subject: [PATCH 0695/1170] Make 02703_row_policies_for_asterisk parallelizable --- .../02703_row_policies_for_asterisk.sh | 14 +++++++++ .../02703_row_policies_for_asterisk.sql | 11 ------- ...3_row_policies_for_database_combination.sh | 30 +++++++++---------- 3 files changed, 29 insertions(+), 26 deletions(-) create mode 100755 tests/queries/0_stateless/02703_row_policies_for_asterisk.sh delete mode 100644 tests/queries/0_stateless/02703_row_policies_for_asterisk.sql diff --git a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh new file mode 100755 index 00000000000..f9670e5f6f8 --- /dev/null +++ b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery " + SELECT 'Policy for table \`*\` does not affect other tables in the database'; + CREATE ROW POLICY 02703_asterisk_${CLICKHOUSE_DATABASE}_policy ON ${CLICKHOUSE_DATABASE}.\`*\` USING x=1 AS permissive TO ALL; + CREATE TABLE ${CLICKHOUSE_DATABASE}.\`*\` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; + CREATE TABLE ${CLICKHOUSE_DATABASE}.\`other\` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; + SELECT 'star', * FROM ${CLICKHOUSE_DATABASE}.\`*\`; + SELECT 'other', * FROM ${CLICKHOUSE_DATABASE}.other; + DROP ROW POLICY 02703_asterisk_${CLICKHOUSE_DATABASE}_policy ON ${CLICKHOUSE_DATABASE}.\`*\`; +" diff --git a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql b/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql deleted file mode 100644 index 96b1c01a6d6..00000000000 --- a/tests/queries/0_stateless/02703_row_policies_for_asterisk.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-parallel - -SELECT 'Policy for table `*` does not affect other tables in the database'; -CREATE DATABASE 02703_db_asterisk; -CREATE ROW POLICY 02703_asterisk ON 02703_db_asterisk.`*` USING x=1 AS permissive TO ALL; -CREATE TABLE 02703_db_asterisk.`*` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; -CREATE TABLE 02703_db_asterisk.`other` (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY x AS SELECT 100, 20; -SELECT 'star', * FROM 02703_db_asterisk.`*`; -SELECT 'other', * FROM 02703_db_asterisk.other; -DROP ROW POLICY 02703_asterisk ON 02703_db_asterisk.`*`; -DROP DATABASE 02703_db_asterisk; diff --git a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh index f7b7c814d29..35151eed220 100755 --- a/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh +++ b/tests/queries/0_stateless/02703_row_policies_for_database_combination.sh @@ -14,37 +14,37 @@ INSERT INTO 02703_rptable VALUES (1, 10), (2, 20), (3, 30), (4, 40); CREATE TABLE 02703_rptable_another ENGINE = MergeTree ORDER BY x AS SELECT * FROM 02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; -DROP ROW POLICY IF EXISTS 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; -DROP ROW POLICY IF EXISTS 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY IF EXISTS 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.*; -- the test assumes users_without_row_policies_can_read_rows is true SELECT 'None'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=1 AS permissive TO ALL; SELECT 'R1: x == 1'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.* USING x=2 AS permissive TO ALL; SELECT 'R1, R2: (x == 1) OR (x == 2)'; SELECT * FROM 02703_rptable; SELECT 'R1, R2: (x == 2) FROM ANOTHER'; SELECT * FROM 02703_rptable_another; -CREATE ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x=3 AS permissive TO ALL; SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable USING x<=2 AS restrictive TO ALL; SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)'; SELECT * FROM 02703_rptable; -CREATE ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; +CREATE ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.* USING y>=20 AS restrictive TO ALL; SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; @@ -58,19 +58,19 @@ SELECT * FROM 02703_after_rp; SELECT 'R1, R2, R3, R4, R5: (x == 2) AND (y >= 20) FROM ANOTHER'; SELECT * FROM 02703_rptable_another; -DROP ROW POLICY 02703_filter_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_1 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_2 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_2 ON ${CLICKHOUSE_DATABASE}.*; SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_3 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R4, R5: (x <= 2) AND (y >= 20)'; SELECT * FROM 02703_rptable; -DROP ROW POLICY 02703_filter_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_4 ON ${CLICKHOUSE_DATABASE}.02703_rptable; SELECT 'R5: (x >= 2)'; SELECT * FROM 02703_rptable; @@ -78,7 +78,7 @@ CREATE TABLE 02703_unexpected_columns (xx UInt8, yy UInt8) ENGINE = MergeTree OR SELECT 'Policy not applicable'; SELECT * FROM 02703_unexpected_columns; -- { serverError UNKNOWN_IDENTIFIER } -- Missing columns: 'x' while processing query -DROP ROW POLICY 02703_filter_5 ON ${CLICKHOUSE_DATABASE}.*; +DROP ROW POLICY 02703_filter_policy_${CLICKHOUSE_DATABASE}_5 ON ${CLICKHOUSE_DATABASE}.*; SELECT 'None'; SELECT * FROM 02703_rptable; From 5256e8e6d08b5076be49b56fdebadb668892771c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 14:36:33 +0200 Subject: [PATCH 0696/1170] Integration tests: fix ports clashing problem 3 --- tests/integration/conftest.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index a386ed53009..b4c86a1cd2f 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -132,7 +132,7 @@ def pytest_configure(config): worker_ports = os.getenv("WORKER_FREE_PORTS", None) if worker_ports is None: master_ports = get_unique_free_ports(PORTS_PER_WORKER) - os.environ["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in master_ports])) + os.environ["WORKER_FREE_PORTS"] = " ".join([str(p) for p in master_ports]) def pytest_xdist_setupnodes(config, specs): @@ -148,4 +148,4 @@ def pytest_xdist_setupnodes(config, specs): for i, spec in enumerate(specs): start_range = i * PORTS_PER_WORKER per_workrer_ports = ports[start_range : start_range + PORTS_PER_WORKER] - spec.env["WORKER_FREE_PORTS"] = " ".join(([str(p) for p in per_workrer_ports])) + spec.env["WORKER_FREE_PORTS"] = " ".join([str(p) for p in per_workrer_ports]) From db0bce33526abf16e705b9e56d178d6e2c45a36b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Aug 2024 15:01:15 +0200 Subject: [PATCH 0697/1170] Try make the code more understandable --- src/Interpreters/Cache/FileCache.cpp | 164 ++++++++++++------------- src/Interpreters/Cache/FileCache.h | 12 +- tests/config/config.d/storage_conf.xml | 2 +- 3 files changed, 80 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0a03f5dcc7d..4c17afb79be 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -318,7 +318,29 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + + /// Consider this example to understand why we need to account here for both `size` and `aligned_size`. + /// [________________]__________________] <-- requested range + /// ^ ^ + /// right offset aligned_right_offset + /// [_________] <-- last cached file segment, e.g. we have uncovered suffix of the requested range + /// [________________] + /// size + /// [____________________________________] + /// aligned_size + /// + /// So it is possible that we split this hole range into sub-segments by `max_file_segment_size` + /// and get something like this: + /// + /// [________________________] + /// ^ ^ + /// right_offset right_offset + max_file_segment_size + /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. + /// Because its left offset would be bigger than right_offset. + /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. + std::vector ranges; size_t current_pos = offset; @@ -339,42 +361,23 @@ std::vector FileCache::splitRange(size_t offset, size_t size return ranges; } -FileSegments FileCache::splitRangeIntoFileSegments( +FileSegments FileCache::createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - chassert(size > 0); - chassert(size <= aligned_size); - /// We take `size` as a soft limit and `aligned_size` as a hard limit. - - auto current_pos = offset; - auto end_pos_non_included = offset + size; - - size_t current_file_segment_size; - size_t remaining_size = aligned_size; - - FileSegments file_segments; - const size_t max_size = max_file_segment_size.load(); - while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit)) + FileSegments result; + for (const auto & r : ranges) { - current_file_segment_size = std::min(remaining_size, max_size); - remaining_size -= current_file_segment_size; - - auto file_segment_metadata_it = addFileSegment( - locked_key, current_pos, current_file_segment_size, state, create_settings, nullptr); - file_segments.push_back(file_segment_metadata_it->second->file_segment); - - current_pos += current_file_segment_size; + if (file_segments_limit && file_segments_count >= file_segments_limit) + break; + auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); + result.push_back(metadata_it->second->file_segment); + ++file_segments_count; } - - chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), - fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); - return file_segments; + return result; } void FileCache::fillHolesWithEmptyFileSegments( @@ -448,18 +451,9 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); } if (is_limit_reached()) @@ -493,29 +487,20 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; - auto non_aligned_size = non_aligned_right_offset - current_pos + 1; + auto non_aligned_hole_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { auto file_segment = std::make_shared( - locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, create_settings); + locked_key.getKey(), current_pos, non_aligned_hole_size, FileSegment::State::DETACHED, create_settings); file_segments.insert(file_segments.end(), file_segment); } else { - auto ranges = splitRange(current_pos, non_aligned_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, non_aligned_hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); if (is_limit_reached()) erase_unprocessed(); @@ -548,8 +533,9 @@ FileSegmentsHolderPtr FileCache::set( } else { - file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + const auto ranges = splitRange(offset, size, size); + size_t file_segments_count = 0; + file_segments = createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -569,23 +555,27 @@ FileCache::getOrSet( assertInitialized(); - FileSegment::Range range(offset, offset + size - 1); + FileSegment::Range initial_range(offset, offset + size - 1); + /// result_range is initial range, which will be adjusted according to + /// 1. aligned offset, alighed_end_offset + /// 2. max_file_segments_limit + FileSegment::Range result_range = initial_range; - const auto aligned_offset = roundDownToMultiple(range.left, boundary_alignment); - auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; + const auto aligned_offset = roundDownToMultiple(initial_range.left, boundary_alignment); + auto aligned_end_offset = std::min(roundUpToMultiple(initial_range.right + 1, boundary_alignment), file_size) - 1; - chassert(aligned_offset <= range.left); - chassert(aligned_end_offset >= range.right); + chassert(aligned_offset <= initial_range.left); + chassert(aligned_end_offset >= initial_range.right); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user); /// Get all segments which intersect with the given range. - auto file_segments = getImpl(*locked_key, range, file_segments_limit); + auto file_segments = getImpl(*locked_key, initial_range, file_segments_limit); if (file_segments_limit) { chassert(file_segments.size() <= file_segments_limit); if (file_segments.size() == file_segments_limit) - range.right = aligned_end_offset = file_segments.back()->range().right; + result_range.right = aligned_end_offset = file_segments.back()->range().right; } /// Check case if we have uncovered prefix, e.g. @@ -597,11 +587,11 @@ FileCache::getOrSet( /// [ ] /// ^----^ /// uncovered prefix. - const bool has_uncovered_prefix = file_segments.empty() || range.left < file_segments.front()->range().left; + const bool has_uncovered_prefix = file_segments.empty() || result_range.left < file_segments.front()->range().left; - if (aligned_offset < range.left && has_uncovered_prefix) + if (aligned_offset < result_range.left && has_uncovered_prefix) { - auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? range.left - 1 : file_segments.front()->range().left - 1); + auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? result_range.left - 1 : file_segments.front()->range().left - 1); auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); if (prefix_file_segments.empty()) @@ -610,7 +600,7 @@ FileCache::getOrSet( /// ^ ^ ^ /// aligned_offset range.left range.right /// [___] [__________] <-- current cache (example) - range.left = aligned_offset; + result_range.left = aligned_offset; } else { @@ -621,10 +611,10 @@ FileCache::getOrSet( /// ^ /// prefix_file_segments.back().right - chassert(prefix_file_segments.back()->range().right < range.left); + chassert(prefix_file_segments.back()->range().right < result_range.left); chassert(prefix_file_segments.back()->range().right >= aligned_offset); - range.left = prefix_file_segments.back()->range().right + 1; + result_range.left = prefix_file_segments.back()->range().right + 1; } } @@ -637,11 +627,11 @@ FileCache::getOrSet( /// [___] /// ^---^ /// uncovered_suffix - const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < range.right; + const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < result_range.right; - if (range.right < aligned_end_offset && has_uncovered_suffix) + if (result_range.right < aligned_end_offset && has_uncovered_suffix) { - auto suffix_range = FileSegment::Range(range.right, aligned_end_offset); + auto suffix_range = FileSegment::Range(result_range.right, aligned_end_offset); /// We need to get 1 file segment, so file_segments_limit = 1 here. auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); @@ -652,7 +642,7 @@ FileCache::getOrSet( /// range.left range.right aligned_end_offset /// [___] [___] <-- current cache (example) - range.right = aligned_end_offset; + result_range.right = aligned_end_offset; } else { @@ -662,35 +652,33 @@ FileCache::getOrSet( /// [___] [___] [_________] <-- current cache (example) /// ^ /// suffix_file_segments.front().left - range.right = suffix_file_segments.front()->range().left - 1; + result_range.right = suffix_file_segments.front()->range().left - 1; } } - chassert(range.left >= aligned_offset); - if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments( - *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), - FileSegment::State::EMPTY, file_segments_limit, create_settings); + auto ranges = splitRange(result_range.left, initial_range.size() + (initial_range.left - result_range.left), result_range.size()); + size_t file_segments_count = file_segments.size(); + file_segments.splice(file_segments.end(), createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, file_segments_limit, create_settings)); } else { - chassert(file_segments.front()->range().right >= range.left); - chassert(file_segments.back()->range().left <= range.right); + chassert(file_segments.front()->range().right >= result_range.left); + chassert(file_segments.back()->range().left <= result_range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, result_range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(range.left)) + if (!file_segments.front()->range().contains(result_range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", - file_segments.front()->range().toString(), offset, range.right, aligned_offset, aligned_end_offset); + file_segments.front()->range().toString(), offset, result_range.right, aligned_offset, aligned_end_offset); } } - chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); + chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 3f7eec73b56..07be802a940 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -265,16 +265,10 @@ private: /// each subrange size must be less or equal to max_file_segment_size. std::vector splitRange(size_t offset, size_t size, size_t aligned_size); - /// Split range into subranges by max_file_segment_size (same as in splitRange()) - /// and create a new file segment for each subrange. - /// If `file_segments_limit` > 0, create no more than first file_segments_limit - /// file segments. - FileSegments splitRangeIntoFileSegments( + FileSegments createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 4daa64b520d..e106e3a0e6b 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,7 @@ cache s3_disk s3_cache/ - 100Mi + 104857600 5Mi 1 100 From a94df1fb9af032c8ccccb704186513561e28e11c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 Jul 2024 23:50:48 +0200 Subject: [PATCH 0698/1170] enable parallel_view_processing in perf tests --- tests/performance/materialized_view_parallel_insert.xml | 4 ++++ tests/performance/views_max_insert_threads.xml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tests/performance/materialized_view_parallel_insert.xml b/tests/performance/materialized_view_parallel_insert.xml index df0a23182c3..c3e7cdc50c0 100644 --- a/tests/performance/materialized_view_parallel_insert.xml +++ b/tests/performance/materialized_view_parallel_insert.xml @@ -1,4 +1,8 @@ + + 1 + + CREATE MATERIALIZED VIEW hits_mv ENGINE MergeTree PARTITION BY toYYYYMM(EventDate) diff --git a/tests/performance/views_max_insert_threads.xml b/tests/performance/views_max_insert_threads.xml index 2988984f5d8..c16fb330b35 100644 --- a/tests/performance/views_max_insert_threads.xml +++ b/tests/performance/views_max_insert_threads.xml @@ -1,5 +1,9 @@ + + 1 + + create table views_max_insert_threads_null (a UInt64) Engine = Null create materialized view views_max_insert_threads_mv Engine = Null AS select now() as ts, max(a) from views_max_insert_threads_null group by ts From c3a1381d70e6d5da59e4b29e7a147e24e7fcd90b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Aug 2024 00:10:59 +0200 Subject: [PATCH 0699/1170] some corner case optimize --- src/Interpreters/Squashing.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 488177c3b4f..483112df6cb 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -106,6 +106,17 @@ Chunk Squashing::convertToChunk(CurrentData && data) const Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos) { + if (input_chunks.size() == 1) + { + /// this is just optimization, no logic changes + Chunk result = std::move(input_chunks.front()); + infos.appendIfUniq(std::move(result.getChunkInfos())); + result.setChunkInfos(infos); + + chassert(result); + return result; + } + std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) From 8d979680060f10e6bcec3fc83fb3bdbaa7bb3deb Mon Sep 17 00:00:00 2001 From: Lennard Eijsackers Date: Fri, 2 Aug 2024 15:13:37 +0200 Subject: [PATCH 0700/1170] Use FunctionArgumentDescriptors to check bitSlice function + add test case Signed-off-by: Lennard Eijsackers --- src/Functions/bitSlice.cpp | 30 ++++++++----------- ...214_bitslice_argument_evaluation.reference | 0 .../03214_bitslice_argument_evaluation.sql | 10 +++++++ 3 files changed, 22 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference create mode 100644 tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql diff --git a/src/Functions/bitSlice.cpp b/src/Functions/bitSlice.cpp index e2b455846d8..f1d3bb57221 100644 --- a/src/Functions/bitSlice.cpp +++ b/src/Functions/bitSlice.cpp @@ -40,28 +40,22 @@ public: bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const size_t number_of_arguments = arguments.size(); + FunctionArgumentDescriptors mandatory_args{ + {"s", static_cast(&isStringOrFixedString), nullptr, "String"}, + {"offset", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + }; - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), number_of_arguments); + FunctionArgumentDescriptors optional_args{ + {"length", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + }; - if (!isString(arguments[0]) && !isStringOrFixedString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - if (arguments[0]->onlyNull()) - return arguments[0]; + validateFunctionArguments(*this, arguments, mandatory_args, optional_args); - if (!isNativeNumber(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", - arguments[1]->getName(), getName()); - - if (number_of_arguments == 3 && !isNativeNumber(arguments[2])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", - arguments[2]->getName(), getName()); + const auto & type = arguments[0].type; + if (type->onlyNull()) + return type; return std::make_shared(); } diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql new file mode 100644 index 00000000000..b8488600fcb --- /dev/null +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql @@ -0,0 +1,10 @@ +-- No arguments passed +SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Invalid 1st argument passed +SELECT bitSlice(1, 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument, invalid 2nd argument passed +SELECT bitSlice('Hello', 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument & 2nd argument, invalid 3rd argument passed +SELECT bitSlice('Hello', 1, 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- More arguments then expected +SELECT bitSlice('Hello', 1, 1, 'World'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 83096249a7480b0bfa1d9246c17136727bba904c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 13:54:32 +0000 Subject: [PATCH 0701/1170] Update version_date.tsv and changelogs after v24.3.6.48-lts --- docs/changelogs/v24.3.6.48-lts.md | 39 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 40 insertions(+) create mode 100644 docs/changelogs/v24.3.6.48-lts.md diff --git a/docs/changelogs/v24.3.6.48-lts.md b/docs/changelogs/v24.3.6.48-lts.md new file mode 100644 index 00000000000..f045afc619b --- /dev/null +++ b/docs/changelogs/v24.3.6.48-lts.md @@ -0,0 +1,39 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.6.48-lts (b2d33c3c45d) FIXME as compared to v24.3.5.46-lts (fe54cead6b6) + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#66889](https://github.com/ClickHouse/ClickHouse/issues/66889): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66687](https://github.com/ClickHouse/ClickHouse/issues/66687): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67497](https://github.com/ClickHouse/ClickHouse/issues/67497): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#66324](https://github.com/ClickHouse/ClickHouse/issues/66324): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66151](https://github.com/ClickHouse/ClickHouse/issues/66151): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66451](https://github.com/ClickHouse/ClickHouse/issues/66451): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66222](https://github.com/ClickHouse/ClickHouse/issues/66222): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66676](https://github.com/ClickHouse/ClickHouse/issues/66676): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66602](https://github.com/ClickHouse/ClickHouse/issues/66602): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66356](https://github.com/ClickHouse/ClickHouse/issues/66356): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66970](https://github.com/ClickHouse/ClickHouse/issues/66970): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66967](https://github.com/ClickHouse/ClickHouse/issues/66967): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66718](https://github.com/ClickHouse/ClickHouse/issues/66718): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66949](https://github.com/ClickHouse/ClickHouse/issues/66949): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66946](https://github.com/ClickHouse/ClickHouse/issues/66946): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67629](https://github.com/ClickHouse/ClickHouse/issues/67629): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67193](https://github.com/ClickHouse/ClickHouse/issues/67193): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67375](https://github.com/ClickHouse/ClickHouse/issues/67375): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67572](https://github.com/ClickHouse/ClickHouse/issues/67572): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66422](https://github.com/ClickHouse/ClickHouse/issues/66422): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66855](https://github.com/ClickHouse/ClickHouse/issues/66855): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#67055](https://github.com/ClickHouse/ClickHouse/issues/67055): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66943](https://github.com/ClickHouse/ClickHouse/issues/66943): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 7b5dcda82e3..24488066190 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -10,6 +10,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.6.48-lts 2024-08-02 v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 From 9c7464e0653782af385dbc884dd3acecfc69c6cc Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:04:11 +0200 Subject: [PATCH 0702/1170] Stateless tests: reduce pure_http_client timeout to get reasons of timed out tests --- tests/queries/0_stateless/helpers/pure_http_client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 0e7a4d27f4f..7a8efec36bb 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -18,7 +18,7 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=1500, settings=dict(), binary_result=False + self, query, connection_timeout=500, settings=dict(), binary_result=False ): NUMBER_OF_TRIES = 30 DELAY = 10 @@ -47,12 +47,12 @@ class ClickHouseClient: else: raise ValueError(r.text) - def query_return_df(self, query, connection_timeout=1500): + def query_return_df(self, query, connection_timeout=500): data = self.query(query, connection_timeout) df = pd.read_csv(io.StringIO(data), sep="\t") return df - def query_with_data(self, query, data, connection_timeout=1500, settings=dict()): + def query_with_data(self, query, data, connection_timeout=500, settings=dict()): params = { "query": query, "timeout_before_checking_execution_speed": 120, From 2c9cef38e56c65ec9bbe7f3af21d4865662f6e9a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:05:39 +0200 Subject: [PATCH 0703/1170] Stateless tests: fix hanging tests `02473_multistep_prewhere*` `00411_long_accurate_number_comparison*` --- ...411_long_accurate_number_comparison.python | 19 ++++++------------- .../02473_multistep_prewhere.python | 4 ++-- .../02473_multistep_split_prewhere.python | 4 ++-- .../0_stateless/helpers/pure_http_client.py | 17 ++++++++++++++++- 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index 045de9ee7ee..38b108a696f 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -2,23 +2,16 @@ import os, itertools, urllib.request, urllib.parse, urllib.error, urllib.request, urllib.error, urllib.parse, sys +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) -def get_ch_answer(query): - return ( - urllib.request.urlopen( - os.environ.get( - "CLICKHOUSE_URL", - "http://localhost:" + os.environ.get("CLICKHOUSE_PORT_HTTP", "8123"), - ), - data=query.encode(), - ) - .read() - .decode() - ) +from pure_http_client import ClickHouseClient + +client = ClickHouseClient() def check_answers(query, answer): - ch_answer = get_ch_answer(query) + ch_answer = client.query(query) if ch_answer.strip() != answer.strip(): print("FAIL on query:", query) print("Expected answer:", answer) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index 11095202039..09326b6365d 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -195,7 +195,7 @@ def main(): default_index_granularity = 10 total_rows = 7 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [ default_index_granularity - 1, default_index_granularity, diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.python b/tests/queries/0_stateless/02473_multistep_split_prewhere.python index 19444994fd2..10e94059171 100644 --- a/tests/queries/0_stateless/02473_multistep_split_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -161,7 +161,7 @@ def main(): default_index_granularity = 10 total_rows = 8 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [default_index_granularity - 1, default_index_granularity]: tester = Tester(session, url, index_granularity, total_rows) # Test combinations of ranges of columns c and d diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 7a8efec36bb..a31a91e0550 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -1,7 +1,8 @@ import os import io -import sys import requests +from requests.adapters import HTTPAdapter +from requests.packages.urllib3.util.retry import Retry import time import pandas as pd @@ -77,3 +78,17 @@ class ClickHouseClient: return result else: raise ValueError(r.text) + + +def requests_session_with_retries(retries=3, timeout=180): + session = requests.Session() + retry = Retry( + total=retries, + read=retries, + connect=retries, + ) + adapter = HTTPAdapter(max_retries=retry) + session.mount("http://", adapter) + session.mount("https://", adapter) + session.timeout = timeout + return session From aefed7cdd62e874f7507afe69d803c9164a283ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Aug 2024 16:06:53 +0200 Subject: [PATCH 0704/1170] Update tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql --- .../0_stateless/03164_s3_settings_for_queries_and_merges.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index 001ef382850..a6932e0536c 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -21,6 +21,7 @@ SYSTEM DROP MARK CACHE; SELECT count() FROM t_compact_bytes_s3 WHERE NOT ignore(c2, c4); SYSTEM FLUSH LOGS; +-- Errors in S3 requests will be automatically retried, however ProfileEvents can be wrong. That is why we subtract errors. SELECT ProfileEvents['S3ReadRequestsCount'] - ProfileEvents['S3ReadRequestsErrors'], ProfileEvents['ReadBufferFromS3Bytes'] < ProfileEvents['ReadCompressedBytes'] * 1.1 From 86bab5a78a9593862815869c5964f557159352ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Aug 2024 16:09:43 +0200 Subject: [PATCH 0705/1170] Unit test: Don't return ok if leaks are detected --- docker/test/unit/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/unit/run.sh b/docker/test/unit/run.sh index ba11f568218..210f31170a5 100644 --- a/docker/test/unit/run.sh +++ b/docker/test/unit/run.sh @@ -1,6 +1,9 @@ #!/bin/bash set -x +# Need to keep error from tests after `tee`. Otherwise we don't alert on asan errors +set -o pipefail +set -e if [ "$#" -ne 1 ]; then echo "Expected exactly one argument" From 664e131f4f2e46fc216305c440e840a5a5784328 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:09:48 +0200 Subject: [PATCH 0706/1170] Integration tests: fix ports clashing problem 4 --- tests/integration/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index b4c86a1cd2f..aa235118aed 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -1,4 +1,6 @@ #!/usr/bin/env python3 +# pylint: disable=unused-argument +# pylint: disable=broad-exception-raised import logging import os From b3e2ce695514d4d314ed8ac1ecdb111c5f94ac7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0707/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 2110b765d19ce4e68d0c23eab219e199aec0ea96 Mon Sep 17 00:00:00 2001 From: Lennard Eijsackers Date: Fri, 2 Aug 2024 17:53:12 +0200 Subject: [PATCH 0708/1170] Style check fix + adding debug info to query output Signed-off-by: Lennard Eijsackers --- src/Functions/bitSlice.cpp | 2 -- .../03214_bitslice_argument_evaluation.reference | 11 +++++++++++ .../03214_bitslice_argument_evaluation.sql | 1 + 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Functions/bitSlice.cpp b/src/Functions/bitSlice.cpp index f1d3bb57221..f24473351ae 100644 --- a/src/Functions/bitSlice.cpp +++ b/src/Functions/bitSlice.cpp @@ -18,9 +18,7 @@ using namespace GatherUtils; namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionBitSlice : public IFunction diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference index e69de29bb2d..1731dfa0d79 100644 --- a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.reference @@ -0,0 +1,11 @@ +-- { echo } +-- No arguments passed +SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Invalid 1st argument passed +SELECT bitSlice(1, 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument, invalid 2nd argument passed +SELECT bitSlice('Hello', 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Valid 1st argument & 2nd argument, invalid 3rd argument passed +SELECT bitSlice('Hello', 1, 'World'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- More arguments then expected +SELECT bitSlice('Hello', 1, 1, 'World'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql index b8488600fcb..1731dfa0d79 100644 --- a/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql +++ b/tests/queries/0_stateless/03214_bitslice_argument_evaluation.sql @@ -1,3 +1,4 @@ +-- { echo } -- No arguments passed SELECT bitSlice(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -- Invalid 1st argument passed From 364622f567028ffc70785b681fc246d7151eef04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0709/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From fcb0ce7361f74dd8d97a3007f77248f293b2ce5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 18:17:32 +0200 Subject: [PATCH 0710/1170] Fix docs build --- docker/docs/builder/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/docs/builder/run.sh b/docker/docs/builder/run.sh index 01c15cb4b0f..d73adb5d279 100755 --- a/docker/docs/builder/run.sh +++ b/docker/docs/builder/run.sh @@ -26,7 +26,6 @@ sed -i '/onBrokenMarkdownLinks:/ s/ignore/error/g' docusaurus.config.js if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then export CI=true - yarn install exec yarn build "$@" fi From cebb3668380f65187b201e638013df40f8ac8ada Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 09:23:40 +0200 Subject: [PATCH 0711/1170] more fixes --- .github/actions/check_workflow/action.yml | 21 ++++ .github/workflows/create_release.yml | 27 +++-- .github/workflows/pull_request.yml | 9 +- tests/ci/artifactory.py | 6 +- tests/ci/auto_release.py | 6 +- tests/ci/ci.py | 4 +- tests/ci/ci_buddy.py | 30 +++-- tests/ci/ci_cache.py | 14 +-- tests/ci/ci_config.py | 3 +- tests/ci/ci_definitions.py | 3 +- tests/ci/ci_metadata.py | 4 +- tests/ci/ci_utils.py | 30 ++++- tests/ci/create_release.py | 131 ++++++++++++++-------- tests/ci/docker_server.py | 73 +++--------- tests/ci/test_docker.py | 46 +------- 15 files changed, 213 insertions(+), 194 deletions(-) create mode 100644 .github/actions/check_workflow/action.yml diff --git a/.github/actions/check_workflow/action.yml b/.github/actions/check_workflow/action.yml new file mode 100644 index 00000000000..19a3cec76f5 --- /dev/null +++ b/.github/actions/check_workflow/action.yml @@ -0,0 +1,21 @@ +name: CheckWorkflowResults + +description: Check overall workflow status and post error to slack if any + +inputs: + needs: + description: github needs context as a json string + required: true + type: string + +runs: + using: "composite" + steps: + - name: Check Workflow + shell: bash + run: | + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ inputs.needs }} + EOF + python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index e27db1b09a4..29094cc51a6 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -24,7 +24,7 @@ concurrency: dry-run: description: 'Dry run' required: false - default: true + default: false type: boolean jobs: @@ -43,16 +43,27 @@ jobs: - name: Prepare Release Info shell: bash run: | + if [ ${{ inputs.only-repo }} == "true" ]; then + git tag -l ${{ inputs.ref }} || { echo "With only-repo option ref must be a valid release tag"; exit 1; } + fi python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ + ${{ inputs.dry-run == true && '--dry-run' || '' }} \ + ${{ inputs.only-repo == true && '--skip-tag-check' || '' }} echo "::group::Release Info" python3 -m json.tool /tmp/release_info.json echo "::endgroup::" release_tag=$(jq -r '.release_tag' /tmp/release_info.json) commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + is_latest=$(jq -r '.latest' /tmp/release_info.json) echo "Release Tag: $release_tag" echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + if [ "$is_latest" == "true" ]; then + echo "DOCKER_TAG_TYPE=release-latest" >> "$GITHUB_ENV" + else + echo "DOCKER_TAG_TYPE=release" >> "$GITHUB_ENV" + fi - name: Download All Release Artifacts if: ${{ inputs.type == 'patch' }} shell: bash @@ -85,10 +96,11 @@ jobs: echo "Generate ChangeLog" export CI=1 docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --volume=".:/wd" --workdir="/wd" \ + clickhouse/style-test \ + ./tests/ci/changelog.py -v --debug-helpers \ --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + --output="./docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md echo "Generate Security" python3 ./utils/security-generator/generate_security.py > SECURITY.md @@ -160,7 +172,7 @@ jobs: cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker server release" export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building if: ${{ inputs.type == 'patch' }} @@ -169,7 +181,7 @@ jobs: cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker keeper release" export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} python3 ./create_release.py --set-progress-completed - name: Update release info. Merge created PRs shell: bash @@ -178,6 +190,7 @@ jobs: - name: Set current Release progress to Completed with OK shell: bash run: | + # dummy stage to finalize release info with "progress: completed; status: OK" python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" python3 ./tests/ci/create_release.py --set-progress-completed - name: Post Slack Message diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 04bef1460a6..071f0f1e20a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -172,12 +172,9 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} - name: Check Workflow results - run: | - export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat > "$WORKFLOW_RESULT_FILE" << 'EOF' - ${{ toJson(needs) }} - EOF - python3 ./tests/ci/ci_buddy.py --check-wf-status + uses: ./.github/actions/check_workflow + with: + needs: ${{ toJson(needs) }} ################################# Stage Final ################################# # diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 8bba7bca30e..f3d7d24f717 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -158,7 +158,7 @@ class DebianArtifactory: print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) - self.release_info.debian_command = debian_command + self.release_info.debian = debian_command self.release_info.dump() @@ -240,7 +240,7 @@ class RpmArtifactory: print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) - self.release_info.rpm_command = rpm_command + self.release_info.rpm = rpm_command self.release_info.dump() @@ -304,7 +304,7 @@ class TgzArtifactory: expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" Shell.check("rm /tmp/tmp.tgz*", verbose=True) - self.release_info.tgz_command = cmd + self.release_info.tgz = cmd self.release_info.dump() diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 6c17b4c74ad..3cc88634004 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -127,15 +127,13 @@ def _prepare(token): ) commit_num -= 1 - is_completed = CI.GHActions.check_wf_completed( - token=token, commit_sha=commit - ) + is_completed = CI.GH.check_wf_completed(token=token, commit_sha=commit) if not is_completed: print(f"CI is in progress for [{commit}] - check previous commit") commits_to_branch_head += 1 continue - commit_ci_status = CI.GHActions.get_commit_status_by_name( + commit_ci_status = CI.GH.get_commit_status_by_name( token=token, commit_sha=commit, status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 935fe472e50..2565c8944e4 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -16,7 +16,7 @@ import upload_result_helper from build_check import get_release_or_pr from ci_config import CI from ci_metadata import CiMetadata -from ci_utils import GHActions, normalize_string, Utils +from ci_utils import GH, normalize_string, Utils from clickhouse_helper import ( CiLogsCredentials, ClickHouseHelper, @@ -368,7 +368,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): ) to_be_skipped = True # skip_status = SUCCESS already there - GHActions.print_in_group("Commit Status Data", job_status) + GH.print_in_group("Commit Status Data", job_status) # create pre report jr = JobReport.create_pre_report(status=skip_status, job_skipped=to_be_skipped) diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index 138909c1db0..f0e73e925fe 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -8,7 +8,7 @@ import requests from botocore.exceptions import ClientError from pr_info import PRInfo -from ci_utils import Shell, GHActions +from ci_config import CI class CIBuddy: @@ -31,10 +31,19 @@ class CIBuddy: self.sha = pr_info.sha[:10] def check_workflow(self): - GHActions.print_workflow_results() - res = GHActions.get_workflow_job_result(GHActions.ActionsNames.RunConfig) - if res != GHActions.ActionStatuses.SUCCESS: - self.post_job_error("Workflow Configuration Failed", critical=True) + CI.GH.print_workflow_results() + if CI.Envs.GITHUB_WORKFLOW == CI.WorkFlowNames.CreateRelease: + if not CI.GH.is_workflow_ok(): + self.post_job_error( + f"{CI.Envs.GITHUB_WORKFLOW} Workflow Failed", critical=True + ) + else: + res = CI.GH.get_workflow_job_result(CI.GH.ActionsNames.RunConfig) + if res != CI.GH.ActionStatuses.SUCCESS: + print(f"ERROR: RunConfig status is [{res}] - post report to slack") + self.post_job_error( + f"{CI.Envs.GITHUB_WORKFLOW} Workflow Failed", critical=True + ) @staticmethod def _get_webhooks(): @@ -74,10 +83,13 @@ class CIBuddy: message = title if isinstance(body, dict): for name, value in body.items(): - if "commit_sha" in name: + if "sha" in name and value and len(value) == 40: value = ( f"" ) + elif isinstance(value, str) and value.startswith("https://github.com/"): + value_shorten = value.split("/")[-1] + value = f"<{value}|{value_shorten}>" message += f" *{name}*: {value}\n" else: message += body + "\n" @@ -120,9 +132,11 @@ class CIBuddy: ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: - instance_id = Shell.get_output("ec2metadata --instance-id") or instance_id + instance_id = ( + CI.Shell.get_output("ec2metadata --instance-id") or instance_id + ) instance_type = ( - Shell.get_output("ec2metadata --instance-type") or instance_type + CI.Shell.get_output("ec2metadata --instance-type") or instance_type ) if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 85eabb84f9f..4846233ab03 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -7,7 +7,7 @@ from typing import Dict, Optional, Any, Union, Sequence, List, Set from ci_config import CI -from ci_utils import is_hex, GHActions +from ci_utils import is_hex, GH from commit_status_helper import CommitStatusData from env_helper import ( TEMP_PATH, @@ -258,15 +258,15 @@ class CiCache: def print_status(self): print(f"Cache enabled: [{self.enabled}]") for record_type in self.RecordType: - GHActions.print_in_group( + GH.print_in_group( f"Cache records: [{record_type}]", list(self.records[record_type]) ) - GHActions.print_in_group( + GH.print_in_group( "Jobs to do:", list(self.jobs_to_do.items()), ) - GHActions.print_in_group("Jobs to skip:", self.jobs_to_skip) - GHActions.print_in_group( + GH.print_in_group("Jobs to skip:", self.jobs_to_skip) + GH.print_in_group( "Jobs to wait:", list(self.jobs_to_wait.items()), ) @@ -788,7 +788,7 @@ class CiCache: while round_cnt < MAX_ROUNDS_TO_WAIT: round_cnt += 1 - GHActions.print_in_group( + GH.print_in_group( f"Wait pending jobs, round [{round_cnt}/{MAX_ROUNDS_TO_WAIT}]:", list(self.jobs_to_wait), ) @@ -853,7 +853,7 @@ class CiCache: # make up for 2 iterations in dry_run expired_sec += int(TIMEOUT / 2) + 1 - GHActions.print_in_group( + GH.print_in_group( "Remaining jobs:", [list(self.jobs_to_wait)], ) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a7df884a091..c031ca9b805 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -34,7 +34,8 @@ class CI: from ci_definitions import Runners as Runners from ci_utils import Envs as Envs from ci_utils import Utils as Utils - from ci_utils import GHActions as GHActions + from ci_utils import GH as GH + from ci_utils import Shell as Shell from ci_definitions import Labels as Labels from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS from ci_definitions import WorkFlowNames as WorkFlowNames diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 51de8c63509..de6791acda8 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -112,6 +112,7 @@ class WorkFlowNames(metaclass=WithIter): """ JEPSEN = "JepsenWorkflow" + CreateRelease = "CreateRelease" class BuildNames(metaclass=WithIter): @@ -578,7 +579,7 @@ class CommonJobConfigs: DOCKER_SERVER = JobConfig( job_name_keyword="docker", required_on_release_branch=True, - run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', + run_command='docker_server.py --check-name "$CHECK_NAME" --tag-type head --allow-build-reuse', digest=DigestConfig( include_paths=[ "tests/ci/docker_server.py", diff --git a/tests/ci/ci_metadata.py b/tests/ci/ci_metadata.py index a767d102811..67106262634 100644 --- a/tests/ci/ci_metadata.py +++ b/tests/ci/ci_metadata.py @@ -9,7 +9,7 @@ from env_helper import ( S3_BUILDS_BUCKET_PUBLIC, ) from s3_helper import S3Helper -from ci_utils import GHActions +from ci_utils import GH from synchronizer_utils import SYNC_BRANCH_PREFIX @@ -111,7 +111,7 @@ class CiMetadata: else: log_title = f"Storing workflow metadata: PR [{self.pr_number}], upstream PR [{self.upstream_pr_number}]" - GHActions.print_in_group( + GH.print_in_group( log_title, [f"run_id: {self.run_id}"], ) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 4f696a2c55a..dae1520afb6 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -16,6 +16,8 @@ class Envs: WORKFLOW_RESULT_FILE = os.getenv( "WORKFLOW_RESULT_FILE", "/tmp/workflow_results.json" ) + S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") + GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") LABEL_CATEGORIES = { @@ -83,7 +85,7 @@ def normalize_string(string: str) -> str: return res -class GHActions: +class GH: class ActionsNames: RunConfig = "RunConfig" @@ -117,6 +119,14 @@ class GHActions: results = [f"{job}: {data['result']}" for job, data in res.items()] cls.print_in_group("Workflow results", results) + @classmethod + def is_workflow_ok(cls) -> bool: + res = cls._get_workflow_results() + for _job, data in res.items(): + if data["result"] == "failure": + return False + return bool(res) + @classmethod def get_workflow_job_result(cls, wf_job_name: str) -> Optional[str]: res = cls._get_workflow_results() @@ -189,15 +199,25 @@ class GHActions: return False @staticmethod - def get_pr_url_by_branch(repo, branch): - get_url_cmd = ( - f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" - ) + def get_pr_url_by_branch(branch, repo=None): + repo = repo or Envs.GITHUB_REPOSITORY + get_url_cmd = f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url' --state open" url = Shell.get_output(get_url_cmd) + if not url: + print(f"WARNING: No open PR found, branch [{branch}] - search for merged") + get_url_cmd = f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url' --state merged" + url = Shell.get_output(get_url_cmd) if not url: print(f"ERROR: PR nor found, branch [{branch}]") return url + @staticmethod + def is_latest_release_branch(branch): + latest_branch = Shell.get_output( + 'gh pr list --label release --repo ClickHouse/ClickHouse --search "sort:created" -L1 --json headRefName' + ) + return latest_branch == branch + class Shell: @classmethod diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index c407a74fbf0..b4e08f29dbe 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -10,9 +10,8 @@ from typing import Iterator, List from git_helper import Git, GIT_PREFIX from ssh import SSHAgent -from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper -from ci_utils import Shell, GHActions +from ci_utils import Shell, GH from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, @@ -69,13 +68,14 @@ class ReleaseContextManager: previous_release_tag="NA", previous_release_sha="NA", release_progress=ReleaseProgress.STARTED, + latest=False, ).dump() else: # fetch release info from fs and update self.release_info = ReleaseInfo.from_file() assert self.release_info assert ( - self.release_info.progress_description == ReleaseProgressDescription.OK + self.release_info.progress_status == ReleaseProgressDescription.OK ), "Must be OK on the start of new context" self.release_info.release_progress = self.release_progress self.release_info.dump() @@ -84,9 +84,9 @@ class ReleaseContextManager: def __exit__(self, exc_type, exc_value, traceback): assert self.release_info if exc_type is not None: - self.release_info.progress_description = ReleaseProgressDescription.FAILED + self.release_info.progress_status = ReleaseProgressDescription.FAILED else: - self.release_info.progress_description = ReleaseProgressDescription.OK + self.release_info.progress_status = ReleaseProgressDescription.OK self.release_info.dump() @@ -96,6 +96,7 @@ class ReleaseInfo: release_tag: str release_branch: str commit_sha: str + latest: bool # lts or stable codename: str previous_release_tag: str @@ -104,12 +105,12 @@ class ReleaseInfo: version_bump_pr: str = "" prs_merged: bool = False release_url: str = "" - debian_command: str = "" - rpm_command: str = "" - tgz_command: str = "" - docker_command: str = "" + debian: str = "" + rpm: str = "" + tgz: str = "" + docker: str = "" release_progress: str = "" - progress_description: str = "" + progress_status: str = "" def is_patch(self): return self.release_branch != "master" @@ -129,12 +130,15 @@ class ReleaseInfo: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) return self - def prepare(self, commit_ref: str, release_type: str) -> "ReleaseInfo": + def prepare( + self, commit_ref: str, release_type: str, skip_tag_check: bool + ) -> "ReleaseInfo": version = None release_branch = None release_tag = None previous_release_tag = None previous_release_sha = None + latest_release = False codename = "" assert release_type in ("patch", "new") if release_type == "new": @@ -145,7 +149,7 @@ class ReleaseInfo: verbose=True, ) with checkout(commit_ref): - commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") + commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -158,12 +162,12 @@ class ReleaseInfo: release_tag = version.describe previous_release_tag = expected_prev_tag previous_release_sha = Shell.get_output_or_raise( - f"git rev-parse {previous_release_tag}" + f"git rev-list -n1 {previous_release_tag}" ) assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - commit_sha = Shell.get_output_or_raise(f"git rev-parse {commit_ref}") + commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -200,16 +204,20 @@ class ReleaseInfo: expected_tag_prefix ) and git.latest_tag.endswith(expected_tag_suffix): pass - else: + elif not skip_tag_check: assert ( False - ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]" + ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" previous_release_sha = Shell.get_output_or_raise( - f"git rev-parse {previous_release_tag}" + f"git rev-list -n1 {previous_release_tag}" ) assert previous_release_sha + if CI.GH.is_latest_release_branch(release_branch): + print("This is going to be the latest release!") + latest_release = True + assert ( release_branch and previous_release_tag @@ -218,7 +226,7 @@ class ReleaseInfo: and release_tag and version and (codename in ("lts", "stable") or release_type == "new") - ) + ), f"Check: {release_branch}, {previous_release_tag}, {previous_release_sha}, {commit_sha}, {release_tag}, {version}" self.release_branch = release_branch self.commit_sha = commit_sha @@ -228,7 +236,8 @@ class ReleaseInfo: self.previous_release_tag = previous_release_tag self.previous_release_sha = previous_release_sha self.release_progress = ReleaseProgress.STARTED - self.progress_description = ReleaseProgressDescription.OK + self.progress_status = ReleaseProgressDescription.OK + self.latest = latest_release return self def push_release_tag(self, dry_run: bool) -> None: @@ -252,7 +261,7 @@ class ReleaseInfo: @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: - cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" + cmd = f"gh api repos/{CI.Envs.GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" Shell.check(cmd, dry_run=dry_run, strict=True) def push_new_release_branch(self, dry_run: bool) -> None: @@ -294,7 +303,7 @@ class ReleaseInfo: f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) Shell.check( - f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' + f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, @@ -303,9 +312,12 @@ class ReleaseInfo: verbose=True, ) + def get_version_bump_branch(self): + return f"bump_version_{self.version}" + def update_version_and_contributors_list(self, dry_run: bool) -> None: # Bump version, update contributors list, create PR - branch_upd_version_contributors = f"bump_version_{self.version}" + branch_upd_version_contributors = self.get_version_bump_branch() with checkout(self.commit_sha): git = Git() version = get_version_from_repo(git=git) @@ -323,9 +335,9 @@ class ReleaseInfo: update_contributors(raise_error=True) cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" - body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" - cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file {body_file} --label 'do not test' --assignee {actor}" + body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" + cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body \"{body}\" --assignee {actor}" Shell.check( cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True ) @@ -342,30 +354,42 @@ class ReleaseInfo: ) self.version_bump_pr = "dry-run" else: - self.version_bump_pr = GHActions.get_pr_url_by_branch( - repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors + self.version_bump_pr = GH.get_pr_url_by_branch( + branch=branch_upd_version_contributors ) + def get_change_log_branch(self): + return f"auto/{self.release_tag}" + def update_release_info(self, dry_run: bool) -> "ReleaseInfo": if self.release_branch != "master": - branch = f"auto/{release_info.release_tag}" - if not dry_run: - url = GHActions.get_pr_url_by_branch( - repo=GITHUB_REPOSITORY, branch=branch - ) - else: - url = "dry-run" - print(f"ChangeLog PR url [{url}]") - self.changelog_pr = url - print(f"Release url [{url}]") - self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" - if self.release_progress == ReleaseProgress.COMPLETED: - self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" + if not self.changelog_pr: + branch = self.get_change_log_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"ChangeLog PR url [{url}]") + self.changelog_pr = url + + if not self.version_bump_pr: + branch = self.get_version_bump_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"Version bump PR url [{url}]") + self.version_bump_pr = url + + self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + print(f"Release url [{self.release_url}]") + + self.docker = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" self.dump() return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: - repo = os.getenv("GITHUB_REPOSITORY") + repo = CI.Envs.GITHUB_REPOSITORY assert repo cmds = [ f"gh release create --repo {repo} --title 'Release {self.release_tag}' {self.release_tag}" @@ -375,7 +399,9 @@ class ReleaseInfo: if not dry_run: for cmd in cmds: Shell.check(cmd, strict=True, verbose=True) - self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + self.release_url = ( + f"https://github.com/{repo}/releases/tag/{self.release_tag}" + ) else: print("Dry-run, would run commands:") print("\n * ".join(cmds)) @@ -536,7 +562,7 @@ class PackageDownloader: ] ) self.s3.download_file( - bucket=S3_BUILDS_BUCKET, + bucket=CI.Envs.S3_BUILDS_BUCKET, s3_path=s3_path, local_file_path="/".join([self.LOCAL_DIR, package_file]), ) @@ -557,7 +583,7 @@ class PackageDownloader: ] ) self.s3.download_file( - bucket=S3_BUILDS_BUCKET, + bucket=CI.Envs.S3_BUILDS_BUCKET, s3_path=s3_path, local_file_path="/".join([self.LOCAL_DIR, destination_binary_name]), ) @@ -636,6 +662,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Initial step to prepare info like release branch, release tag, etc.", ) + parser.add_argument( + "--skip-tag-check", + action="store_true", + help="To skip check against latest git tag on a release branch", + ) parser.add_argument( "--push-release-tag", action="store_true", @@ -725,7 +756,11 @@ if __name__ == "__main__": assert ( args.ref and args.release_type ), "--ref and --release-type must be provided with --prepare-release-info" - release_info.prepare(commit_ref=args.ref, release_type=args.release_type) + release_info.prepare( + commit_ref=args.ref, + release_type=args.release_type, + skip_tag_check=args.skip_tag_check, + ) if args.download_packages: with ReleaseContextManager( @@ -776,7 +811,7 @@ if __name__ == "__main__": else: title = "New release" if ( - release_info.progress_description == ReleaseProgressDescription.OK + release_info.progress_status == ReleaseProgressDescription.OK and release_info.release_progress == ReleaseProgress.COMPLETED ): title = "Completed: " + title @@ -792,16 +827,16 @@ if __name__ == "__main__": if args.set_progress_started: ri = ReleaseInfo.from_file() ri.release_progress = args.progress - ri.progress_description = ReleaseProgressDescription.FAILED + ri.progress_status = ReleaseProgressDescription.FAILED ri.dump() assert args.progress, "Progress step name must be provided" if args.set_progress_completed: ri = ReleaseInfo.from_file() assert ( - ri.progress_description == ReleaseProgressDescription.FAILED + ri.progress_status == ReleaseProgressDescription.FAILED ), "Must be FAILED before set to OK" - ri.progress_description = ReleaseProgressDescription.OK + ri.progress_status = ReleaseProgressDescription.OK ri.dump() if args.merge_prs: diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 3e782c079c6..8f0474d5053 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -69,13 +69,14 @@ def parse_args() -> argparse.Namespace: help="sha of the commit to use packages from", ) parser.add_argument( - "--release-type", + "--tag-type", type=str, - choices=("auto", "latest", "major", "minor", "patch", "head"), + choices=("head", "release", "latest-release"), default="head", - help="version part that will be updated when '--version' is set; " - "'auto' is a special case, it will get versions from github and detect the " - "release type (latest, major, minor or patch) automatically", + help="defines required tags for resulting docker image. " + "head - for master image (tag: head) " + "release - for release image (tags: XX, XX.XX, XX.XX.XX, XX.XX.XX.XX) " + "release-latest - for latest release image (tags: XX, XX.XX, XX.XX.XX, XX.XX.XX.XX, latest) ", ) parser.add_argument( "--image-path", @@ -149,74 +150,35 @@ def retry_popen(cmd: str, log_file: Path) -> int: return retcode -def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: - if release_type != "auto": - return release_type - - git_versions = get_tagged_versions() - reference_version = git_versions[0] - for i in reversed(range(len(git_versions))): - if git_versions[i] <= version: - if i == len(git_versions) - 1: - return "latest" - reference_version = git_versions[i + 1] - break - - if version.major < reference_version.major: - return "major" - if version.minor < reference_version.minor: - return "minor" - if version.patch < reference_version.patch: - return "patch" - - raise ValueError( - "Release type 'tweak' is not supported for " - f"{version.string} < {reference_version.string}" - ) - - -def gen_tags(version: ClickHouseVersion, release_type: str) -> List[str]: +def gen_tags(version: ClickHouseVersion, tag_type: str) -> List[str]: """ - 22.2.2.2 + latest: + @tag_type release-latest, @version 22.2.2.2: - latest - 22 - 22.2 - 22.2.2 - 22.2.2.2 - 22.2.2.2 + major: + @tag_type release, @version 22.2.2.2: - 22 - 22.2 - 22.2.2 - 22.2.2.2 - 22.2.2.2 + minor: - - 22.2 - - 22.2.2 - - 22.2.2.2 - 22.2.2.2 + patch: - - 22.2.2 - - 22.2.2.2 - 22.2.2.2 + head: + @tag_type head: - head """ parts = version.string.split(".") tags = [] - if release_type == "latest": - tags.append(release_type) + if tag_type == "release-latest": + tags.append("latest") for i in range(len(parts)): tags.append(".".join(parts[: i + 1])) - elif release_type == "major": + elif tag_type == "head": + tags.append(tag_type) + elif tag_type == "release": for i in range(len(parts)): tags.append(".".join(parts[: i + 1])) - elif release_type == "minor": - for i in range(1, len(parts)): - tags.append(".".join(parts[: i + 1])) - elif release_type == "patch": - for i in range(2, len(parts)): - tags.append(".".join(parts[: i + 1])) - elif release_type == "head": - tags.append(release_type) else: - raise ValueError(f"{release_type} is not valid release part") + assert False, f"Invalid release type [{tag_type}]" return tags @@ -370,8 +332,7 @@ def main(): push = True image = DockerImageData(image_path, image_repo, False) - args.release_type = auto_release_type(args.version, args.release_type) - tags = gen_tags(args.version, args.release_type) + tags = gen_tags(args.version, args.tag_type) repo_urls = {} direct_urls: Dict[str, List[str]] = {} diff --git a/tests/ci/test_docker.py b/tests/ci/test_docker.py index 662143bfd9b..58ebe4ecbb1 100644 --- a/tests/ci/test_docker.py +++ b/tests/ci/test_docker.py @@ -1,61 +1,19 @@ #!/usr/bin/env python import unittest -from unittest.mock import patch, MagicMock from version_helper import get_version_from_string import docker_server as ds -# di.logging.basicConfig(level=di.logging.INFO) - class TestDockerServer(unittest.TestCase): def test_gen_tags(self): version = get_version_from_string("22.2.2.2") cases = ( - ("latest", ["latest", "22", "22.2", "22.2.2", "22.2.2.2"]), - ("major", ["22", "22.2", "22.2.2", "22.2.2.2"]), - ("minor", ["22.2", "22.2.2", "22.2.2.2"]), - ("patch", ["22.2.2", "22.2.2.2"]), + ("release-latest", ["latest", "22", "22.2", "22.2.2", "22.2.2.2"]), + ("release", ["22", "22.2", "22.2.2", "22.2.2.2"]), ("head", ["head"]), ) for case in cases: release_type = case[0] self.assertEqual(case[1], ds.gen_tags(version, release_type)) - - with self.assertRaises(ValueError): - ds.gen_tags(version, "auto") - - @patch("docker_server.get_tagged_versions") - def test_auto_release_type(self, mock_tagged_versions: MagicMock) -> None: - mock_tagged_versions.return_value = [ - get_version_from_string("1.1.1.1"), - get_version_from_string("1.2.1.1"), - get_version_from_string("2.1.1.1"), - get_version_from_string("2.2.1.1"), - get_version_from_string("2.2.2.1"), - ] - - cases_less = ( - (get_version_from_string("1.0.1.1"), "minor"), - (get_version_from_string("1.1.2.1"), "minor"), - (get_version_from_string("1.3.1.1"), "major"), - (get_version_from_string("2.1.2.1"), "minor"), - (get_version_from_string("2.2.1.3"), "patch"), - (get_version_from_string("2.2.3.1"), "latest"), - (get_version_from_string("2.3.1.1"), "latest"), - ) - for case in cases_less: - release = ds.auto_release_type(case[0], "auto") - self.assertEqual(case[1], release) - - cases_equal = ( - (get_version_from_string("1.1.1.1"), "minor"), - (get_version_from_string("1.2.1.1"), "major"), - (get_version_from_string("2.1.1.1"), "minor"), - (get_version_from_string("2.2.1.1"), "patch"), - (get_version_from_string("2.2.2.1"), "latest"), - ) - for case in cases_equal: - release = ds.auto_release_type(case[0], "auto") - self.assertEqual(case[1], release) From 51b39a6c745d61cb2e6feb39659ddb3cac57ad03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:24:45 +0100 Subject: [PATCH 0712/1170] 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 ae6d4bdd8aa2b010f27dd8ebbb6f816362eb2a9e Mon Sep 17 00:00:00 2001 From: Sasha Sheikin Date: Mon, 29 Jul 2024 10:31:35 +0200 Subject: [PATCH 0713/1170] Fix positionCaseInsensitive example --- .../en/sql-reference/functions/string-search-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index b7ba1d4feb7..e9ff7ebf33b 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -150,15 +150,15 @@ A case insensitive invariant of [position](#position). Query: ``` sql -SELECT position('Hello, world!', 'hello'); +SELECT positionCaseInsensitive('Hello, world!', 'hello'); ``` Result: ``` text -┌─position('Hello, world!', 'hello')─┐ -│ 0 │ -└────────────────────────────────────┘ +┌─positionCaseInsensitive('Hello, world!', 'hello')─┐ +│ 1 │ +└───────────────────────────────────────────────────┘ ``` ## positionUTF8 From 89c47df559ba23d988f8af3c342e0c8d5531f4b8 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Aug 2024 16:52:46 +0000 Subject: [PATCH 0714/1170] Fix variant as common type in if function with Tuples and Maps --- src/Functions/if.cpp | 10 ++++++++++ .../03215_varian_as_common_type_tuple_map.reference | 10 ++++++++++ .../03215_varian_as_common_type_tuple_map.sql | 7 +++++++ 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 07dbee27a9d..64da6e95a43 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -632,6 +632,11 @@ private: ColumnPtr executeTuple(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { + /// For different Tuples the result type can be Variant with this Tuples if use_variant_as_common_type=1. + /// In this case we should use generic implementation. + if (!isTuple(result_type)) + return nullptr; + /// Calculate function for each corresponding elements of tuples. const ColumnWithTypeAndName & arg1 = arguments[1]; @@ -677,6 +682,11 @@ private: ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { + /// For different Maps the result type can be Variant with this Maps if use_variant_as_common_type=1. + /// In this case we should use generic implementation. + if (!isMap(result_type)) + return nullptr; + auto extract_kv_from_map = [](const ColumnMap * map) { const ColumnTuple & tuple = map->getNestedData(); diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference new file mode 100644 index 00000000000..7dabd5388f4 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.reference @@ -0,0 +1,10 @@ +('0') Variant(Tuple(String), Tuple(\n number UInt64)) +(1) Variant(Tuple(String), Tuple(\n number UInt64)) +('2') Variant(Tuple(String), Tuple(\n number UInt64)) +(3) Variant(Tuple(String), Tuple(\n number UInt64)) +('4') Variant(Tuple(String), Tuple(\n number UInt64)) +{'0':'0'} Variant(Map(String, String), Map(UInt64, UInt64)) +{1:1} Variant(Map(String, String), Map(UInt64, UInt64)) +{'2':'2'} Variant(Map(String, String), Map(UInt64, UInt64)) +{3:3} Variant(Map(String, String), Map(UInt64, UInt64)) +{'4':'4'} Variant(Map(String, String), Map(UInt64, UInt64)) diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql new file mode 100644 index 00000000000..4a9a788ab18 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_tuple_map.sql @@ -0,0 +1,7 @@ +set use_variant_as_common_type = 1; +set allow_experimental_variant_type = 1; + +SELECT if(number % 2, tuple(number), tuple(toString(number))) as res, toTypeName(res) FROM numbers(5); +SELECT if(number % 2, map(number, number), map(toString(number), toString(number))) as res, toTypeName(res) FROM numbers(5); + + From b38c46a87d6eacbc7805562deb07ce586fd7e0fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0715/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From e3239c6ee11eb5bf0466fb750c58125868885ec8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 19:09:16 +0200 Subject: [PATCH 0716/1170] Fix bad log message in JIT for sorting --- src/Core/SortDescription.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 9edc79a1ff1..1b3f81f8547 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -103,7 +103,15 @@ static std::string getSortDescriptionDump(const SortDescription & description, c WriteBufferFromOwnString buffer; for (size_t i = 0; i < description.size(); ++i) - buffer << header_types[i]->getName() << ' ' << description[i].direction << ' ' << description[i].nulls_direction; + { + if (i != 0) + buffer << ", "; + + buffer << "(type: " << header_types[i]->getName() + << ", direction: " << description[i].direction + << ", nulls_direction: " << description[i].nulls_direction + << ")"; + } return buffer.str(); } From 3c4389ec4d78db55ce742e5d5a3b0ed050c9c9e6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 2 Aug 2024 14:57:19 -0300 Subject: [PATCH 0717/1170] doc/fix max_partitions_to_read description --- .../operations/settings/merge-tree-settings.md | 7 ++----- .../en/operations/settings/query-complexity.md | 18 ++++++++++++++++-- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 7278b91f90d..67fa45c20cd 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -119,11 +119,6 @@ Minimum size of blocks of uncompressed data required for compression when writin You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. -## max_partitions_to_read - -Limits the maximum number of partitions that can be accessed in one query. -You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. - ## max_suspicious_broken_parts If the number of broken parts in a single partition exceeds the `max_suspicious_broken_parts` value, automatic deletion is denied. @@ -691,6 +686,8 @@ Possible values: Default value: -1 (unlimited). +You can also specify a query complexity setting [max_partitions_to_read](query-complexity#max-partitions-to-read) at a query / session / profile level. + ## min_age_to_force_merge_seconds {#min_age_to_force_merge_seconds} Merge parts if every part in the range is older than the value of `min_age_to_force_merge_seconds`. diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 2a20e74e20f..14ccb1167f9 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -188,7 +188,7 @@ If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use c What to do if the query is run longer than `max_execution_time` or the estimated running time is longer than `max_estimated_execution_time`: `throw` or `break`. By default, `throw`. -# max_execution_time_leaf +## max_execution_time_leaf Similar semantic to `max_execution_time` but only apply on leaf node for distributed or remote queries. @@ -204,7 +204,7 @@ We can use `max_execution_time_leaf` as the query settings: SELECT count() FROM cluster(cluster, view(SELECT * FROM t)) SETTINGS max_execution_time_leaf = 10; ``` -# timeout_overflow_mode_leaf +## timeout_overflow_mode_leaf What to do when the query in leaf node run longer than `max_execution_time_leaf`: `throw` or `break`. By default, `throw`. @@ -426,3 +426,17 @@ Example: ``` Default value: 0 (Infinite count of simultaneous sessions). + +## max_partitions_to_read {#max-partitions-to-read} + +Limits the maximum number of partitions that can be accessed in one query. + +The setting value specified when the table is created can be overridden via query-level setting. + +Possible values: + +- Any positive integer. + +Default value: -1 (unlimited). + +You can also specify a MergeTree setting [max_partitions_to_read](merge-tree-settings#max-partitions-to-read) in tables' setting. From ce39957983af8bdd7d97e4a3729e2f97d3e0cb85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:11:25 +0200 Subject: [PATCH 0718/1170] Remove capitalization in test reports --- tests/ci/report.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index f50ed4c1f85..3f0fc596824 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -293,9 +293,9 @@ class JobReport: start_time: str duration: float additional_files: Union[Sequence[str], Sequence[Path]] - # clickhouse version, build job only + # ClickHouse version, build job only version: str = "" - # checkname to set in commit status, set if differs from jjob name + # check_name to be set in commit status, set it if it differs from the job name check_name: str = "" # directory with artifacts to upload on s3 build_dir_for_upload: Union[Path, str] = "" @@ -667,11 +667,7 @@ ColorTheme = Tuple[str, str, str] def _format_header( header: str, branch_name: str, branch_url: Optional[str] = None ) -> str: - # Following line does not lower CI->Ci and SQLancer->Sqlancer. It only - # capitalizes the first letter and doesn't touch the rest of the word - result = " ".join([w[0].upper() + w[1:] for w in header.split(" ") if w]) - result = result.replace("Clickhouse", "ClickHouse") - result = result.replace("clickhouse", "ClickHouse") + result = header if "ClickHouse" not in result: result = f"ClickHouse {result}" if branch_url: From 2c9b61d047c1afe22b0fa0a967a87db8bd4cf62f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:16:44 +0200 Subject: [PATCH 0719/1170] Miscellaneous --- tests/ci/ci.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 935fe472e50..6ca84a346e2 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1019,7 +1019,7 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: +def _cancel_pr_workflow(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: wf_data = CiMetadata(s3, pr_number).fetch_meta() if not cancel_sync: if not wf_data.run_id: @@ -1368,12 +1368,12 @@ def main() -> int: assert indata, "Run config must be provided via --infile" _update_gh_statuses_action(indata=indata, s3=s3) - ### CANCEL PREVIOUS WORKFLOW RUN + ### CANCEL THE PREVIOUS WORKFLOW RUN elif args.cancel_previous_run: if pr_info.is_merge_queue: - _cancel_pr_wf(s3, pr_info.merged_pr) + _cancel_pr_workflow(s3, pr_info.merged_pr) elif pr_info.is_pr: - _cancel_pr_wf(s3, pr_info.number, cancel_sync=True) + _cancel_pr_workflow(s3, pr_info.number, cancel_sync=True) else: assert False, "BUG! Not supported scenario" From bd3606dac4954c673ec6c38dd6fbdb70bc7b53cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:17:07 +0200 Subject: [PATCH 0720/1170] Fix typos --- tests/ci/commit_status_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index fdc9c002b66..908ac4a7dca 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -301,7 +301,7 @@ def get_worst_state(statuses: CommitStatuses) -> StatusType: def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: - """The function converst the statuses to TestResults and uploads the report + """The function converts the statuses to TestResults and uploads the report to S3 tests bucket. Then it returns the URL""" test_results = [] # type: TestResults for status in statuses: From 675afda17210ca7e8e71e0899a5ed14d7227fb55 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 20:22:08 +0200 Subject: [PATCH 0721/1170] Fix check names in the CI Logs database --- tests/ci/clickhouse_helper.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 287970cce9a..0725f7100d1 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -3,6 +3,7 @@ import fileinput import json import logging import time +import os from pathlib import Path from typing import Any, Dict, List, Optional @@ -298,6 +299,11 @@ class CiLogsCredentials: def get_docker_arguments( self, pr_info: PRInfo, check_start_time: str, check_name: str ) -> str: + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) + if run_by_hash_total > 1: + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + check_name = f"{check_name} [{run_by_hash_num + 1}/{run_by_hash_total}]" + self.create_ci_logs_credentials() if not self.config_path.exists(): logging.info("Do not use external logs pushing") From b282be83c500bac5544424378b9505fc8c28e432 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 20:03:43 +0200 Subject: [PATCH 0722/1170] remove old workflows --- .github/actions/release/action.yml | 166 --------------------------- .github/workflows/auto_release.yml | 111 ------------------ .github/workflows/create_release.yml | 2 + .github/workflows/release.yml | 71 ------------ .github/workflows/tags_stable.yml | 74 ------------ tests/ci/docker_server.py | 1 - 6 files changed, 2 insertions(+), 423 deletions(-) delete mode 100644 .github/actions/release/action.yml delete mode 100644 .github/workflows/auto_release.yml delete mode 100644 .github/workflows/release.yml delete mode 100644 .github/workflows/tags_stable.yml diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml deleted file mode 100644 index a287aa8b41d..00000000000 --- a/.github/actions/release/action.yml +++ /dev/null @@ -1,166 +0,0 @@ -name: Release - -description: Makes patch releases and creates new release branch - -inputs: - ref: - description: 'Git reference (branch or commit sha) from which to create the release' - required: true - type: string - type: - description: 'The type of release: "new" for a new release or "patch" for a patch release' - required: true - type: choice - options: - - patch - - new - dry-run: - description: 'Dry run' - required: true - type: boolean - token: - required: true - type: string - -runs: - using: "composite" - steps: - - name: Prepare Release Info - shell: bash - run: | - python3 ./tests/ci/create_release.py --prepare-release-info \ - --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} ${{ inputs.dry-run == true && '--dry-run' || '' }} - echo "::group::Release Info" - python3 -m json.tool /tmp/release_info.json - echo "::endgroup::" - release_tag=$(jq -r '.release_tag' /tmp/release_info.json) - commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) - echo "Release Tag: $release_tag" - echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" - echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" - - name: Download All Release Artifacts - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Push Git Tag for the Release - shell: bash - run: | - python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - shell: bash - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - git checkout master - python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" - echo "List versions" - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - echo "Update docker version" - ./utils/list-versions/update-docker-version.sh - echo "Generate ChangeLog" - export CI=1 - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token=${{ inputs.token }} --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} - git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md - echo "Generate Security" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} - uses: peter-evans/create-pull-request@v6 - with: - author: "robot-clickhouse " - token: ${{ inputs.token }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - branch: auto/${{ env.RELEASE_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) - - name: Complete previous steps and Restore git state - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --set-progress-completed - git reset --hard HEAD - git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - shell: bash - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export TGZ Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test TGZ Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export RPM Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test RPM Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Export Debian Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Test Debian Packages - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - - name: Docker clickhouse/clickhouse-server building - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - cd "./tests/ci" - python3 ./create_release.py --set-progress-started --progress "docker server release" - export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./create_release.py --set-progress-completed - - name: Docker clickhouse/clickhouse-keeper building - if: ${{ inputs.type == 'patch' }} - shell: bash - run: | - cd "./tests/ci" - python3 ./create_release.py --set-progress-started --progress "docker keeper release" - export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - python3 ./create_release.py --set-progress-completed - - name: Set current Release progress to Completed with OK - shell: bash - run: | - python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" - python3 ./tests/ci/create_release.py --set-progress-completed - - name: Post Slack Message - if: ${{ !cancelled() }} - shell: bash - run: | - python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run == true && '--dry-run' || '' }} diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml deleted file mode 100644 index 457ffacc7a8..00000000000 --- a/.github/workflows/auto_release.yml +++ /dev/null @@ -1,111 +0,0 @@ -name: AutoRelease - -env: - PYTHONUNBUFFERED: 1 - DRY_RUN: true - -concurrency: - group: release -on: # yamllint disable-line rule:truthy - # Workflow uses a test bucket for packages and dry run mode (no real releases) - schedule: - - cron: '0 9 * * *' - - cron: '0 15 * * *' - workflow_dispatch: - inputs: - dry-run: - description: 'Dry run' - required: false - default: true - type: boolean - -jobs: - AutoRelease: - runs-on: [self-hosted, release-maker] - steps: - - name: DebugInfo - uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" - - name: Set DRY_RUN for dispatch - if: ${{ github.event_name == 'workflow_dispatch' }} - run: echo "DRY_RUN=${{ github.event.inputs.dry-run }}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - fetch-depth: 0 - - name: Auto Release Prepare - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --prepare - echo "::group::Auto Release Info" - python3 -m json.tool /tmp/autorelease_info.json - echo "::endgroup::" - { - echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_ENV" - - name: Post Release Branch statuses - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --post-status - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[0].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} - uses: ./.github/actions/release - with: - ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} - type: patch - dry-run: ${{ env.DRY_RUN }} - token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - - name: Post Slack Message - if: ${{ !cancelled() }} - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} - - name: Clean up - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 29094cc51a6..d4993b373df 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -89,6 +89,8 @@ jobs: shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" + + git checkout master # in case WF started from feature branch echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv echo "Update docker version" diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml deleted file mode 100644 index 7dc4e3298a6..00000000000 --- a/.github/workflows/release.yml +++ /dev/null @@ -1,71 +0,0 @@ -name: PublishedReleaseCI -# - Gets artifacts from S3 -# - Sends it to JFROG Artifactory -# - Adds them to the release assets - -on: # yamllint disable-line rule:truthy - release: - types: - - published - workflow_dispatch: - inputs: - tag: - description: 'Release tag' - required: true - type: string - -jobs: - ReleasePublish: - runs-on: [self-hosted, style-checker] - steps: - - name: Set tag from input - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Set tag from REF - if: github.event_name == 'release' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Deploy packages and assets - run: | - curl --silent --data '' --no-buffer \ - '${{ secrets.PACKAGES_RELEASE_URL }}/release/'"${GITHUB_TAG}"'?binary=binary_darwin&binary=binary_darwin_aarch64&sync=true' - ############################################################################################ - ##################################### Docker images ####################################### - ############################################################################################ - DockerServerImages: - runs-on: [self-hosted, style-checker] - steps: - - name: Set tag from input - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Set tag from REF - if: github.event_name == 'release' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - fetch-depth: 0 # otherwise we will have no version info - filter: tree:0 - ref: ${{ env.GITHUB_TAG }} - - name: Check docker clickhouse/clickhouse-server building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - export CHECK_NAME="Docker server image" - SHA=$(git rev-list -n 1 "$GITHUB_TAG") - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - - name: Check docker clickhouse/clickhouse-keeper building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - export CHECK_NAME="Docker keeper image" - SHA=$(git rev-list -n 1 "$GITHUB_TAG") - python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --sha "$SHA" --check-name "$CHECK_NAME" --push - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml deleted file mode 100644 index 2aa7694bc41..00000000000 --- a/.github/workflows/tags_stable.yml +++ /dev/null @@ -1,74 +0,0 @@ -name: TagsStableWorkflow -# - Gets artifacts from S3 -# - Sends it to JFROG Artifactory -# - Adds them to the release assets - -env: - # Force the stdout and stderr streams to be unbuffered - PYTHONUNBUFFERED: 1 - -on: # yamllint disable-line rule:truthy - push: - tags: - - 'v*-prestable' - - 'v*-stable' - - 'v*-lts' - workflow_dispatch: - inputs: - tag: - description: 'Test tag' - required: true - type: string - - -jobs: - UpdateVersions: - runs-on: [self-hosted, style-checker] - steps: - - name: Set test tag - if: github.event_name == 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" - - name: Get tag name - if: github.event_name != 'workflow_dispatch' - run: | - echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - ref: master - fetch-depth: 0 - filter: tree:0 - - name: Update versions, docker version, changelog, security - env: - GITHUB_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - run: | - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - ./utils/list-versions/update-docker-version.sh - GID=$(id -g "${UID}") - # --network=host and CI=1 are required for the S3 access from a container - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume="${GITHUB_WORKSPACE}:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token="$GITHUB_TOKEN" --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" "${GITHUB_TAG}" - git add "./docs/changelogs/${GITHUB_TAG}.md" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create Pull Request - uses: peter-evans/create-pull-request@v6 - with: - author: "robot-clickhouse " - token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - branch: auto/${{ env.GITHUB_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.GITHUB_TAG }} - - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 8f0474d5053..3251ec5644e 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -27,7 +27,6 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from version_helper import ( ClickHouseVersion, - get_tagged_versions, get_version_from_repo, version_arg, ) From aa38e78d7238d843737d1d268de6ee189c19edc3 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 2 Aug 2024 20:27:59 +0200 Subject: [PATCH 0723/1170] update version_date.tsv --- docs/changelogs/v23.8.16.40-lts.md | 35 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 36 insertions(+) create mode 100644 docs/changelogs/v23.8.16.40-lts.md diff --git a/docs/changelogs/v23.8.16.40-lts.md b/docs/changelogs/v23.8.16.40-lts.md new file mode 100644 index 00000000000..75caf1ea277 --- /dev/null +++ b/docs/changelogs/v23.8.16.40-lts.md @@ -0,0 +1,35 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.16.40-lts (e143a9039ba) FIXME as compared to v23.8.15.35-lts (060ff8e813a) + +#### Improvement +* Backported in [#66962](https://github.com/ClickHouse/ClickHouse/issues/66962): Added support for parameterized view with analyzer to not analyze create parameterized view. Refactor existing parameterized view logic to not analyze create parameterized view. [#54211](https://github.com/ClickHouse/ClickHouse/pull/54211) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Backported in [#65461](https://github.com/ClickHouse/ClickHouse/issues/65461): Reload certificate chain during certificate reload. [#61671](https://github.com/ClickHouse/ClickHouse/pull/61671) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Backported in [#65880](https://github.com/ClickHouse/ClickHouse/issues/65880): Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#65912](https://github.com/ClickHouse/ClickHouse/issues/65912): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#65281](https://github.com/ClickHouse/ClickHouse/issues/65281): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65368](https://github.com/ClickHouse/ClickHouse/issues/65368): Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#65743](https://github.com/ClickHouse/ClickHouse/issues/65743): Fix crash in maxIntersections. [#65689](https://github.com/ClickHouse/ClickHouse/pull/65689) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65351](https://github.com/ClickHouse/ClickHouse/issues/65351): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66037](https://github.com/ClickHouse/ClickHouse/issues/66037): Fix crash on destroying AccessControl: add explicit shutdown. [#64993](https://github.com/ClickHouse/ClickHouse/pull/64993) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#65782](https://github.com/ClickHouse/ClickHouse/issues/65782): Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. [#65632](https://github.com/ClickHouse/ClickHouse/pull/65632) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#65926](https://github.com/ClickHouse/ClickHouse/issues/65926): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#65822](https://github.com/ClickHouse/ClickHouse/issues/65822): Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. [#65802](https://github.com/ClickHouse/ClickHouse/pull/65802) ([jsc0218](https://github.com/jsc0218)). +* Backported in [#66449](https://github.com/ClickHouse/ClickHouse/issues/66449): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66717](https://github.com/ClickHouse/ClickHouse/issues/66717): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65080](https://github.com/ClickHouse/ClickHouse/issues/65080): Follow up to [#56541](https://github.com/ClickHouse/ClickHouse/issues/56541). [#57141](https://github.com/ClickHouse/ClickHouse/pull/57141) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#65913](https://github.com/ClickHouse/ClickHouse/issues/65913): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66853](https://github.com/ClickHouse/ClickHouse/issues/66853): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 24488066190..cb6b8f588da 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -50,6 +50,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.16.40-lts 2024-08-02 v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 From a45ba44dbaa2ed43eb63e49fe609a01be978eac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 18:28:38 +0000 Subject: [PATCH 0724/1170] Automatic style fix --- tests/ci/ci.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 6ca84a346e2..805296d2bb2 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1019,7 +1019,9 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def _cancel_pr_workflow(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: +def _cancel_pr_workflow( + s3: S3Helper, pr_number: int, cancel_sync: bool = False +) -> None: wf_data = CiMetadata(s3, pr_number).fetch_meta() if not cancel_sync: if not wf_data.run_id: From a37eeb0f211c0c7b6251a9108d6b939e73c9a66e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Aug 2024 20:13:27 +0000 Subject: [PATCH 0725/1170] Allow types to change, re-resolve overloads --- src/Storages/MergeTree/KeyCondition.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index eaf9f0af623..91f054c3a71 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -627,7 +627,8 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - res = &inverted_dag.addFunction(node.function_base, children, ""); + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); handled_inversion = true; } else if (need_inversion && (name == "and" || name == "or")) @@ -668,8 +669,13 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - res = &inverted_dag.addFunction(node.function_base, children, ""); - chassert(res->result_type == node.result_type); + /// Can't just addFunction(node.function_base) because argument types may have + /// changed slightly because of our transformations, e.g. maybe some subexpression + /// changed constness, which caused some function return value to change LowCardinality-ness. + /// (I don't have a specific counterexample, but it seems likely that it exists. + /// One was fixed in the past: https://github.com/ClickHouse/ClickHouse/issues/65143 ) + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); } } } @@ -678,13 +684,6 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( if (!handled_inversion && need_inversion) res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, ""); - /// Make sure we don't change any data types (e.g. remove LowCardinality). - /// If it turns out that we actually want to change data types sometimes, it's ok to remove this - /// check *and* replace all `addFunction(node.function_base, ...)` calls above with - /// `addFunction(FunctionFactory::instance().get(name, context), ...)` to re-resolve overloads. - if (!node.result_type->equals(*res->result_type)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", node.result_type->getName(), res->result_type->getName(), node.result_name); - to_inverted[&node] = res; return *res; } From eb0e12099a8fa45d79b2cff96e02ee273e879efa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Aug 2024 21:18:35 +0000 Subject: [PATCH 0726/1170] Another attempt --- src/Storages/MergeTree/KeyCondition.cpp | 27 ++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 91f054c3a71..9115cb5608e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -627,8 +627,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } } - auto function_builder = FunctionFactory::instance().get(name, context); - res = &inverted_dag.addFunction(function_builder, children, ""); + res = &inverted_dag.addFunction(node.function_base, children, ""); handled_inversion = true; } else if (need_inversion && (name == "and" || name == "or")) @@ -669,13 +668,23 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - /// Can't just addFunction(node.function_base) because argument types may have - /// changed slightly because of our transformations, e.g. maybe some subexpression - /// changed constness, which caused some function return value to change LowCardinality-ness. - /// (I don't have a specific counterexample, but it seems likely that it exists. - /// One was fixed in the past: https://github.com/ClickHouse/ClickHouse/issues/65143 ) - auto function_builder = FunctionFactory::instance().get(name, context); - res = &inverted_dag.addFunction(function_builder, children, ""); + /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). + /// Otherwise the function may crash when passed columns of unexpected types. + /// * Why not check this for all subexperessions rather than function arguments? + /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` + /// the AND's args were UInt64, but OR's args are UInt8. + /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? + /// Because some functions can't be found through FunctionFactory, e.g. FunctionCapture. + /// (But maybe we could re-resolve only if argument types changed.) + for (size_t i = 0; i < children.size(); ++i) + { + if (!node.children[i]->result_type->equals(*children[i]->result_type)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", + node.children[i]->result_type->getName(), children[i]->result_type->getName(), node.children[i]->result_name); + } + + res = &inverted_dag.addFunction(node.function_base, children, ""); } } } From dd0ae04f90314ce6d5dbe748605e66f1a6d9024f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0727/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 08bde9cb44dce6fdf069527852faa8ec29a71b10 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 18:28:33 -0400 Subject: [PATCH 0728/1170] 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 0729/1170] 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 a431ab3e4b6f925924a81d99997e6c028ae7950f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:31:02 +0200 Subject: [PATCH 0730/1170] Improve dashboard --- programs/server/dashboard.html | 66 +++++++++++++++++++++++++++------- 1 file changed, 54 insertions(+), 12 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 45f988f7b1e..71880b9e228 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -17,7 +17,7 @@ --input-shadow-color: rgba(0, 255, 0, 1); --error-color: red; --global-error-color: white; - --legend-background: rgba(255, 255, 255, 0.75); + --legend-background: rgba(255, 255, 0, 0.75); --title-color: #666; --text-color: black; --edit-title-background: #FEE; @@ -41,7 +41,7 @@ --moving-shadow-color: rgba(255, 255, 255, 0.25); --input-shadow-color: rgba(255, 128, 0, 0.25); --error-color: #F66; - --legend-background: rgba(255, 255, 255, 0.25); + --legend-background: rgba(0, 96, 128, 0.75); --title-color: white; --text-color: white; --edit-title-background: #364f69; @@ -1004,14 +1004,14 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- className && legendEl.classList.add(className); uPlot.assign(legendEl.style, { - textAlign: "left", + textAlign: "right", pointerEvents: "none", display: "none", position: "absolute", left: 0, top: 0, - zIndex: 100, - boxShadow: "2px 2px 10px rgba(0,0,0,0.1)", + zIndex: 200, + boxShadow: "2px 2px 10px rgba(0, 0, 0, 0.1)", ...style }); @@ -1051,8 +1051,10 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- function update(u) { let { left, top } = u.cursor; - left -= legendEl.clientWidth / 2; - top -= legendEl.clientHeight / 2; + /// This will make the balloon to the right of the cursor when the cursor is on the left side, and vise-versa, + /// avoiding the borders of the chart. + left -= legendEl.clientWidth * (left / u.width); + top -= legendEl.clientHeight; legendEl.style.transform = "translate(" + left + "px, " + top + "px)"; if (multiline) { @@ -1229,14 +1231,53 @@ async function draw(idx, chart, url_params, query) { let sync = uPlot.sync("sync"); - let axis = { + function formatDateTime(t) { + return (new Date(t * 1000)).toISOString().replace('T', '\n').replace('.000Z', ''); + } + + function formatDateTimes(self, ticks) { + return ticks.map((t, idx) => { + let res = formatDateTime(t); + if (idx == 0 || res.substring(0, 10) != formatDateTime(ticks[idx - 1]).substring(0, 10)) { + return res; + } else { + return res.substring(11); + } + }); + } + + function formatValue(v) { + const a = Math.abs(v); + if (a >= 1000000000000000) { return (v / 1000000000000000) + 'P'; } + if (a >= 1000000000000) { return (v / 1000000000000) + 'T'; } + if (a >= 1000000000) { return (v / 1000000000) + 'G'; } + if (a >= 1000000) { return (v / 1000000) + 'M'; } + if (a >= 1000) { return (v / 1000) + 'K'; } + if (a > 0 && a < 0.001) { return (v * 1000000) + "μ"; } + return v; + } + + let axis_x = { stroke: axes_color, grid: { width: 1 / devicePixelRatio, stroke: grid_color }, - ticks: { width: 1 / devicePixelRatio, stroke: grid_color } + ticks: { width: 1 / devicePixelRatio, stroke: grid_color }, + values: formatDateTimes, + space: 80, + incrs: [1, 5, 10, 15, 30, + 60, 60 * 5, 60 * 10, 60 * 15, 60 * 30, + 3600, 3600 * 2, 3600 * 3, 3600 * 4, 3600 * 6, 3600 * 12, + 3600 * 24], }; - let axes = [axis, axis]; - let series = [{ label: "x" }]; + let axis_y = { + stroke: axes_color, + grid: { width: 1 / devicePixelRatio, stroke: grid_color }, + ticks: { width: 1 / devicePixelRatio, stroke: grid_color }, + values: (self, ticks) => ticks.map(formatValue) + }; + + let axes = [axis_x, axis_y]; + let series = [{ label: "time", value: (self, t) => formatDateTime(t) }]; let data = [reply.data[reply.meta[0].name]]; // Treat every column as series @@ -1254,9 +1295,10 @@ async function draw(idx, chart, url_params, query) { const opts = { width: chart.clientWidth, height: chart.clientHeight, + scales: { x: { time: false } }, /// Because we want to split and format time on our own. axes, series, - padding: [ null, null, null, (Math.round(max_value * 100) / 100).toString().length * 6 - 10 ], + padding: [ null, null, null, 3 ], plugins: [ legendAsTooltipPlugin() ], cursor: { sync: { From a6f9dd4447cbb475cbf77b07de35b40fbcad50b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:35:10 +0200 Subject: [PATCH 0731/1170] Improve dashboard --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 71880b9e228..c69acec7858 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1010,7 +1010,7 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- position: "absolute", left: 0, top: 0, - zIndex: 200, + zIndex: 100, boxShadow: "2px 2px 10px rgba(0, 0, 0, 0.1)", ...style }); From 95659de26573bdb17ab2b5649e6dad96fb75c479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:45:41 +0200 Subject: [PATCH 0732/1170] Fix invalid detection of an empty result --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index c69acec7858..238254f4ef8 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -1141,7 +1141,7 @@ async function draw(idx, chart, url_params, query) { let {reply, error} = await doFetch(query, url_params); if (!error) { - if (reply.rows.length == 0) { + if (reply.rows == 0) { error = "Query returned empty result."; } else if (reply.meta.length < 2) { error = "Query should return at least two columns: unix timestamp and value."; From a99f9bb603f78437fba8d3ebb031c2f41d00cd58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 01:50:27 +0200 Subject: [PATCH 0733/1170] Focus on the mass editor --- programs/server/dashboard.html | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 238254f4ef8..8fb07d5da3b 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -945,6 +945,7 @@ function showMassEditor() { let editor = document.getElementById('mass-editor-textarea'); editor.value = JSON.stringify({params: params, queries: queries}, null, 2); + editor.focus(); mass_editor_active = true; } From eeb8c1caac9e8e2ba2f3a1a86f5603281e161610 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 02:06:53 +0200 Subject: [PATCH 0734/1170] Improve margins when there are many parameters --- programs/server/dashboard.html | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 8fb07d5da3b..344de779065 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -256,6 +256,7 @@ font-weight: bold; user-select: none; cursor: pointer; + margin-bottom: 1rem; } #run:hover { @@ -309,7 +310,7 @@ color: var(--param-text-color); display: inline-block; box-shadow: 1px 1px 0 var(--shadow-color); - margin-bottom: 1rem; + margin-bottom: 0.5rem; } input:focus { From 090fb59194462324507d75f032aa803303c3e041 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 02:20:58 +0200 Subject: [PATCH 0735/1170] Automatic field width of chart parameters --- programs/server/dashboard.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 344de779065..0b099b15536 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -218,6 +218,7 @@ #chart-params .param { width: 6%; + font-family: monospace; } input { @@ -658,6 +659,10 @@ function insertParam(name, value) { param_value.value = value; param_value.spellcheck = false; + let setWidth = e => { e.style.width = (e.value.length + 1) + 'ch' }; + if (value) { setWidth(param_value); } + param_value.addEventListener('input', e => setWidth(e.target)); + param_wrapper.appendChild(param_name); param_wrapper.appendChild(param_value); document.getElementById('chart-params').appendChild(param_wrapper); From 9a017528a4685fc4ed7eec7ba37f9e9804972c3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0736/1170] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From dfeb1991164bd6c8b0efc8bdcfe9dcd5b8906928 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 03:06:37 +0200 Subject: [PATCH 0737/1170] Fix locking inside TimerDescriptor --- src/Common/TimerDescriptor.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 9a171ae9487..ce290a1cb31 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -75,10 +76,22 @@ void TimerDescriptor::drain() const /// or since the last successful read(2), then the buffer given to read(2) returns an unsigned 8-byte integer (uint64_t) /// containing the number of expirations that have occurred. /// (The returned value is in host byte order—that is, the native byte order for integers on the host machine.) + + /// Due to a bug in Linux Kernel, reading from timerfd in non-blocking mode can be still blocking. + /// Avoid it with polling. + Epoll epoll; + epoll.add(timer_fd); + epoll_event event; + event.data.fd = -1; + size_t ready_count = epoll.getManyReady(1, &event, 0); + if (!ready_count) + return; + uint64_t buf; while (true) { ssize_t res = ::read(timer_fd, &buf, sizeof(buf)); + if (res < 0) { /// man timerfd_create: From e491c51cdc9a94018fc6918f74232cfff747436a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 3 Aug 2024 01:49:53 +0000 Subject: [PATCH 0738/1170] e --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9115cb5608e..2b89344d3d9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -670,7 +670,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( { /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). /// Otherwise the function may crash when passed columns of unexpected types. - /// * Why not check this for all subexperessions rather than function arguments? + /// * Why not check this for all subexpressions rather than function arguments? /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` /// the AND's args were UInt64, but OR's args are UInt8. /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? From 8cb2e308f41638ebb6ba7fddbd4f0bf89d4d612e Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Sat, 3 Aug 2024 01:55:24 +0000 Subject: [PATCH 0739/1170] Only new analyzer --- .../03209_parameterized_view_with_non_literal_params.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql index f2c61e5cb1d..7fe84929910 100644 --- a/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql +++ b/tests/queries/0_stateless/03209_parameterized_view_with_non_literal_params.sql @@ -1,4 +1,4 @@ - +SET allow_experimental_analyzer = 1; select 'Test with Date parameter'; drop table if exists date_table_pv; From 28ec383739d9a4974549e8c6491797f0eaafaffb Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 3 Aug 2024 02:07:24 +0000 Subject: [PATCH 0740/1170] add sharedmergetree --- src/Storages/MergeTree/MergeTreeData.cpp | 5 +++-- src/Storages/StorageFactory.cpp | 8 ++++---- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a158e375ae9..e849c4b794f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3216,10 +3216,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Block the case of alter table add projection for special merge trees. if (std::any_of(commands.begin(), commands.end(), [](const AlterCommand & c) { return c.type == AlterCommand::ADD_PROJECTION; })) { - if (auto storage_name = getName(); storage_name != "MergeTree" && storage_name != "ReplicatedMergeTree" + const std::unordered_set allowed_storages{"MergeTree", "ReplicatedMergeTree", "SharedMergeTree"}; + if (auto storage_name = getName(); !allowed_storages.contains(storage_name) && settings_from_storage->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" + "Projection is fully supported in (Replictaed, Shared)MergeTree, but also allowed in non-throw mode with other" " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." " Current storage name is {}.", storage_name); } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 7360d351e8a..557f53a9ada 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -204,10 +204,10 @@ StoragePtr StorageFactory::get( /// Now let's handle the merge tree family. Note we only handle in the mode of CREATE due to backward compatibility. /// Otherwise, it would fail to start in the case of existing projections with special mergetree. - /// Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other mergetree family members. chassert(query.storage->engine); - if (std::string_view engine_name(query.storage->engine->name); mode == LoadingStrictnessLevel::CREATE - && engine_name != "MergeTree" && engine_name != "ReplicatedMergeTree") + const std::unordered_set allowed_engines{"MergeTree", "ReplicatedMergeTree", "SharedMergeTree"}; + if (auto engine_name(query.storage->engine->name); mode == LoadingStrictnessLevel::CREATE + && !allowed_engines.contains(engine_name)) { /// default throw mode in deduplicate_merge_projection_mode bool projection_allowed = false; @@ -224,7 +224,7 @@ StoragePtr StorageFactory::get( } if (!projection_allowed) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Projection is fully supported in (Replictaed)MergeTree, but also allowed in non-throw mode with other" + "Projection is fully supported in (Replictaed, Shared)MergeTree, but also allowed in non-throw mode with other" " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." " Current storage name is {}.", engine_name); } From 96e826d154e9b55cf035a3ce025ac81194455ebc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 3 Aug 2024 02:57:33 +0000 Subject: [PATCH 0741/1170] Change tactics again --- src/Storages/MergeTree/KeyCondition.cpp | 31 +++++++++++++++---------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2b89344d3d9..dfb43c4e75d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -668,23 +668,30 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } else { - /// Make sure we don't change types of function arguments (e.g. remove LowCardinality). - /// Otherwise the function may crash when passed columns of unexpected types. - /// * Why not check this for all subexpressions rather than function arguments? - /// Because types may change, e.g. in `NOT (u64 AND u64)` -> `(NOT u64 OR NOT u64)` - /// the AND's args were UInt64, but OR's args are UInt8. - /// * Why not re-resolve function overload, using FunctionFactory::instance().get(name, context)? - /// Because some functions can't be found through FunctionFactory, e.g. FunctionCapture. - /// (But maybe we could re-resolve only if argument types changed.) + /// Argument types could change slightly because of our transformations, e.g. + /// LowCardinality can be added because some subexpressions became constant + /// (in particular, sets). If that happens, re-run function overload resolver. + /// Otherwise don't re-run it because some functions may not be available + /// through FunctionFactory::get(), e.g. FunctionCapture. + bool types_changed = false; for (size_t i = 0; i < children.size(); ++i) { if (!node.children[i]->result_type->equals(*children[i]->result_type)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "KeyCondition inadvertently changed subexpression data type: '{}' -> '{}', column `{}`", - node.children[i]->result_type->getName(), children[i]->result_type->getName(), node.children[i]->result_name); + { + types_changed = true; + break; + } } - res = &inverted_dag.addFunction(node.function_base, children, ""); + if (types_changed) + { + auto function_builder = FunctionFactory::instance().get(name, context); + res = &inverted_dag.addFunction(function_builder, children, ""); + } + else + { + res = &inverted_dag.addFunction(node.function_base, children, ""); + } } } } From f97abf69949f8822d70f4b1251e1945f279dd0ec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 09:35:37 +0200 Subject: [PATCH 0742/1170] tests: avoid endless wait in 01042_system_reload_dictionary_reloads_completely Signed-off-by: Azat Khuzhin --- ...em_reload_dictionary_reloads_completely.sh | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index 03dd376f802..ebc4110332f 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -8,6 +8,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail +# Wait when the dictionary will update the value for 13 on its own: +function wait_for_dict_upate() +{ + for ((i = 0; i < 100; ++i)); do + if [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))")" != -1 ]; then + return 0 + fi + sleep 0.5 + done + return 1 +} + $CLICKHOUSE_CLIENT < ', dictGetInt64('${CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (13, 103, now())" $CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (14, 104, now() - INTERVAL 1 DAY)" -# Wait when the dictionary will update the value for 13 on its own: -while [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))")" = -1 ] -do - sleep 0.5 -done +if ! wait_for_dict_upate; then + echo "Dictionary had not been reloaded" >&2 + exit 1 +fi $CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" From 40cd5467c18d65a6624d273ac1a8fd9cc9257d8c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 09:39:33 +0200 Subject: [PATCH 0743/1170] tests: fix 01042_system_reload_dictionary_reloads_completely flakiness (increase lag) The test fails in case of INSERT takes > 1 sec: 2024.08.02 13:06:07.746869 [ 45445 ] {c9b55378-6bc5-46d5-80c1-5385a880f88b} executeQuery: (from [::1]:37208) (comment: 01042_system_reload_dictionary_reloads_completely.sh) CREATE DICTIONARY test_m4lx2bit.dict ( x Int64 DEFAULT -1, y Int64 DEFAULT -1, insert_time DateTime ) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB 'test_m4lx2bit' UPDATE_FIELD 'insert_time')) LAYOUT(FLAT()) LIFETIME(1); (stage: Complete) ... 2024.08.02 13:06:08.263832 [ 59987 ] {744da223-67b9-4e32-b12a-eb2550a92fdb} DictionaryFactory: Created dictionary source 'ClickHouse: test_m4lx2bit.table' for dictionary '5b2b98a9-9372-47c9-bda3-830794cb96e7' 2024.08.02 13:06:08.268118 [ 59987 ] {744da223-67b9-4e32-b12a-eb2550a92fdb} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table`; (stage: Complete) ... 2024.08.02 13:06:09.193190 [ 45445 ] {b6033498-4666-452f-bcf9-02ecf257ba7f} executeQuery: (from [::1]:37262) (comment: 01042_system_reload_dictionary_reloads_completely.sh) INSERT INTO test_m4lx2bit.table VALUES (stage: Complete) ... 2024.08.02 13:06:11.342119 [ 50962 ] {} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table` WHERE insert_time >= '2024-08-02 13:06:07'; (stage: Complete) ... 2024.08.02 13:06:11.832158 [ 45445 ] {b6033498-4666-452f-bcf9-02ecf257ba7f} TCPHandler: Processed in 2.642106236 sec. ... 2024.08.02 13:06:16.357448 [ 41632 ] {} executeQuery: (internal) SELECT `x`, `y`, `insert_time` FROM `test_m4lx2bit`.`table` WHERE insert_time >= '2024-08-02 13:06:10'; (stage: Complete) Signed-off-by: Azat Khuzhin --- .../01042_system_reload_dictionary_reloads_completely.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index ebc4110332f..453e1bb8f0a 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -31,7 +31,7 @@ CREATE DICTIONARY ${CLICKHOUSE_DATABASE}.dict insert_time DateTime ) PRIMARY KEY x -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB '${CLICKHOUSE_DATABASE}' UPDATE_FIELD 'insert_time')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB '${CLICKHOUSE_DATABASE}' UPDATE_FIELD 'insert_time' UPDATE_LAG 60)) LAYOUT(FLAT()) LIFETIME(1); EOF From 6ce6af0647590f4b58a6ab87ee5f29b8487e8c2f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 3 Aug 2024 14:16:24 +0200 Subject: [PATCH 0744/1170] Fix completion RESTORE ON CLUSTER. --- src/Backups/RestorerFromBackup.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 3056f9fe421..278af9d4eb3 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -222,10 +222,19 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa if (restore_coordination) { restore_coordination->setStage(new_stage, message); - if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) - restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); - else - restore_coordination->waitForStage(new_stage); + + /// The initiator of a RESTORE ON CLUSTER query waits for other hosts to complete their work (see waitForStage(Stage::COMPLETED) in BackupsWorker::doRestore), + /// but other hosts shouldn't wait for each others' completion. (That's simply unnecessary and also + /// the initiator may start cleaning up (e.g. removing restore-coordination ZooKeeper nodes) once all other hosts are in Stage::COMPLETED.) + bool need_wait = (new_stage != Stage::COMPLETED); + + if (need_wait) + { + if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) + restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); + else + restore_coordination->waitForStage(new_stage); + } } } From 73080d25a93aa715c4126ed82b89e3ada89d7a3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 15:10:05 +0200 Subject: [PATCH 0745/1170] Fix test `00002_log_and_exception_messages_formatting` --- src/Databases/DatabaseHDFS.cpp | 4 ++-- src/Storages/ObjectStorage/HDFS/Configuration.cpp | 6 +++--- .../00002_log_and_exception_messages_formatting.sql | 5 ++++- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index eccaae5f22e..f58f1b76e71 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -75,8 +75,8 @@ std::string DatabaseHDFS::getTablePath(const std::string & table_name) const return table_name; if (source.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. " - "It should have structure 'hdfs://:/path'", table_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. " + "It should have the following structure 'hdfs://:/path'", table_name); return fs::path(source) / table_name; } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index e8071be6f02..85eb29a3868 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -142,11 +142,11 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) { auto pos = url_.find("//"); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have the following structure 'hdfs://:/path'", url_); pos = url_.find('/', pos + 2); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have the following structure 'hdfs://:/path'", url_); path = url_.substr(pos + 1); if (!path.starts_with('/')) @@ -155,7 +155,7 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) url = url_.substr(0, pos); paths = {path}; - LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); + LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using URL: {}, path: {}", url, path); } void StorageHDFSConfiguration::addStructureAndFormatToArgs( diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 07c42d6d039..e916fdfc1ff 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -163,7 +163,10 @@ create temporary table known_short_messages (s String) as select * from (select '{} -> {}', '{} {}', '{}%', - '{}: {}' + '{}: {}', + 'Unknown data type family: {}', + 'Cannot load time zone {}', + 'Unknown table engine {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From 8b7a294bc7db64f675faaf9b91a40e3e9f613935 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 30 May 2024 17:15:29 +0200 Subject: [PATCH 0746/1170] 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 0747/1170] 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 0748/1170] 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 0749/1170] 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 0750/1170] 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 0751/1170] 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 0752/1170] 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 0753/1170] 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 0754/1170] 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 0755/1170] 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 0756/1170] 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 0757/1170] 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 0758/1170] 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 0759/1170] 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 0760/1170] 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 0761/1170] 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 a749223251b7e580f5d7bbcb4fc59aa6b5fffbe2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 3 Aug 2024 22:43:12 +0800 Subject: [PATCH 0762/1170] change as request --- src/Functions/printf.cpp | 39 ++++++++++++++----- .../0_stateless/03203_function_printf.sql | 7 +++- 2 files changed, 36 insertions(+), 10 deletions(-) diff --git a/src/Functions/printf.cpp b/src/Functions/printf.cpp index 3efe854a53b..3cf3efaf534 100644 --- a/src/Functions/printf.cpp +++ b/src/Functions/printf.cpp @@ -6,11 +6,10 @@ #include #include #include +#include #include #include -#include -#include #include #include #include @@ -22,6 +21,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int BAD_ARGUMENTS; } namespace @@ -52,10 +52,9 @@ private: [[maybe_unused]] String toString() const { - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() - << std::endl; - return oss.str(); + WriteBufferFromOwnString buf; + buf << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() << "\n"; + return buf.str(); } private: @@ -229,9 +228,31 @@ public: ColumnsWithTypeAndName concat_args(instructions.size()); for (size_t i = 0; i < instructions.size(); ++i) { - // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; - concat_args[i] = instructions[i].execute(); - // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + const auto & instruction = instructions[i]; + try + { + // std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl; + concat_args[i] = instruction.execute(); + // std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl; + } + catch (const fmt::v9::format_error & e) + { + if (instruction.is_literal) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Bad format {} in function {} without input argument, reason: {}", + instruction.format, + getName(), + e.what()); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Bad format {} in function {} with {} as input argument, reason: {}", + instructions[i].format, + getName(), + instruction.input.dumpStructure(), + e.what()); + } } auto res = function_concat->build(concat_args)->execute(concat_args, std::make_shared(), input_rows_count); diff --git a/tests/queries/0_stateless/03203_function_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql index c41cbf0b5e9..6ff4699c8a7 100644 --- a/tests/queries/0_stateless/03203_function_printf.sql +++ b/tests/queries/0_stateless/03203_function_printf.sql @@ -31,4 +31,9 @@ select printf('%%.2e: %.2e', 123.456) = '%.2e: 1.23e+02'; select printf('%%.2g: %.2g', 123.456) = '%.2g: 1.2e+02'; -- Testing character formats with precision -select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; \ No newline at end of file +select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; + +select printf('%%X: %X', 123.123); -- { serverError BAD_ARGUMENTS } +select printf('%%A: %A', 'abc'); -- { serverError BAD_ARGUMENTS } +select printf('%%s: %s', 100); -- { serverError BAD_ARGUMENTS } +select printf('%%n: %n', 100); -- { serverError BAD_ARGUMENTS } From abb747498d04fd6d6cac71d6097741b41bf572e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 16:54:44 +0200 Subject: [PATCH 0763/1170] Fix test `02789_reading_from_s3_with_connection_pool` --- ...89_reading_from_s3_with_connection_pool.sh | 84 +++++++++++-------- 1 file changed, 50 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh index 751b2798243..39399842db1 100755 --- a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh +++ b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh @@ -14,43 +14,59 @@ SETTINGS disk = 's3_disk', min_bytes_for_wide_part = 0; INSERT INTO test_s3 SELECT number, number FROM numbers_mt(1e7); " -query="SELECT a, b FROM test_s3" -query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" 2>&1) -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -nm --query " -SELECT ProfileEvents['DiskConnectionsPreserved'] > 0 -FROM system.query_log -WHERE type = 'QueryFinish' - AND current_database = currentDatabase() - AND query_id='$query_id'; -" + +# This (reusing connections from the pool) is not guaranteed to always happen, +# (due to random time difference between the queries and random activity in parallel) +# but should happen most of the time. + +while true +do + query="SELECT a, b FROM test_s3" + query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" 2>&1) + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + + RES=$(${CLICKHOUSE_CLIENT} -nm --query " + SELECT ProfileEvents['DiskConnectionsPreserved'] > 0 + FROM system.query_log + WHERE type = 'QueryFinish' + AND current_database = currentDatabase() + AND query_id='$query_id'; + ") + + [[ $RES -eq 1 ]] && echo "$RES" && break; +done # Test connection pool in ReadWriteBufferFromHTTP -query_id=$(${CLICKHOUSE_CLIENT} -nq " -create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; -set insert_keeper_fault_injection_probability=0; -insert into mut values (1, 2, 3), (10, 20, 30); +while true +do + query_id=$(${CLICKHOUSE_CLIENT} -nq " + create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; + set insert_keeper_fault_injection_probability=0; + insert into mut values (1, 2, 3), (10, 20, 30); -system stop merges mut; -alter table mut delete where n = 10; + system stop merges mut; + alter table mut delete where n = 10; -select queryID() from( - -- a funny way to wait for a MUTATE_PART to be assigned - select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( - 'select 1 where ''MUTATE_PART'' not in (select type from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'')' - ), 'LineAsString', 's String') - -- queryID() will be returned for each row, since the query above doesn't return anything we need to return a fake row - union all - select 1 -) limit 1 settings max_threads=1; -" 2>&1) -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -nm --query " -SELECT ProfileEvents['StorageConnectionsPreserved'] > 0 -FROM system.query_log -WHERE type = 'QueryFinish' - AND current_database = currentDatabase() - AND query_id='$query_id'; -" + select queryID() from( + -- a funny way to wait for a MUTATE_PART to be assigned + select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( + 'select 1 where ''MUTATE_PART'' not in (select type from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'')' + ), 'LineAsString', 's String') + -- queryID() will be returned for each row, since the query above doesn't return anything we need to return a fake row + union all + select 1 + ) limit 1 settings max_threads=1; + " 2>&1) + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + RES=$(${CLICKHOUSE_CLIENT} -nm --query " + SELECT ProfileEvents['StorageConnectionsPreserved'] > 0 + FROM system.query_log + WHERE type = 'QueryFinish' + AND current_database = currentDatabase() + AND query_id='$query_id'; + ") + + [[ $RES -eq 1 ]] && echo "$RES" && break; +done From fc651cc0c61feb37e9cf104612cc0ac0cd7448e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 17:57:48 +0200 Subject: [PATCH 0764/1170] Fix strange code in HostResolvePool --- src/Common/HostResolvePool.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/HostResolvePool.cpp b/src/Common/HostResolvePool.cpp index cad64ee7204..e8a05a269bc 100644 --- a/src/Common/HostResolvePool.cpp +++ b/src/Common/HostResolvePool.cpp @@ -253,18 +253,18 @@ void HostResolver::updateImpl(Poco::Timestamp now, std::vector Date: Sat, 3 Aug 2024 18:30:33 +0200 Subject: [PATCH 0765/1170] Fix typo --- 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 20db4c2773c..7800ee9ff00 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -849,7 +849,7 @@ try #endif #if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + LOG_INFO(log, "Query Profiler is disabled because it cannot work under sanitizers" " when two different stack unwinding methods will interfere with each other."); #endif From 60648e5240fecb92344ff029d2b280f542c3a86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 18:41:17 +0200 Subject: [PATCH 0766/1170] Revert "Add replication lag and recovery time metrics" --- src/Databases/DatabaseReplicated.cpp | 57 ++++------------- src/Databases/DatabaseReplicated.h | 10 +-- src/Databases/DatabaseReplicatedWorker.cpp | 21 ------- src/Databases/DatabaseReplicatedWorker.h | 5 -- src/Storages/System/StorageSystemClusters.cpp | 37 ++++------- src/Storages/System/StorageSystemClusters.h | 4 +- .../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, 27 insertions(+), 226 deletions(-) delete mode 100644 tests/integration/test_recovery_time_metric/__init__.py delete mode 100644 tests/integration/test_recovery_time_metric/configs/config.xml delete mode 100644 tests/integration/test_recovery_time_metric/test.py delete mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.reference delete mode 100644 tests/queries/0_stateless/03206_replication_lag_metric.sql diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d2dee9b5994..f127ccbc224 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include @@ -339,12 +338,9 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const return std::make_shared(getContext()->getSettingsRef(), shards, params); } -ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const +std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr & cluster_) const { - Strings paths_get, paths_exists; - - paths_get.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); - + Strings paths; 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) @@ -352,59 +348,32 @@ ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) for (const auto & replica : addresses_with_failover[shard_index]) { String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths_exists.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); - paths_get.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); + paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } try { auto current_zookeeper = getZooKeeper(); - auto get_res = current_zookeeper->get(paths_get); - auto exist_res = current_zookeeper->exists(paths_exists); - chassert(get_res.size() == exist_res.size() + 1); + auto res = current_zookeeper->exists(paths); - auto max_log_ptr_zk = get_res[0]; - if (max_log_ptr_zk.error != Coordination::Error::ZOK) - throw Coordination::Exception(max_log_ptr_zk.error); + std::vector statuses; + statuses.resize(paths.size()); - UInt32 max_log_ptr = parse(max_log_ptr_zk.data); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error == Coordination::Error::ZOK) + statuses[i] = 1; - ReplicasInfo replicas_info; - replicas_info.resize(exist_res.size()); - - 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 = exist_res[global_replica_index]; - auto replica_log_ptr = get_res[global_replica_index + 1]; - - if (replica_active.error != Coordination::Error::ZOK && replica_active.error != Coordination::Error::ZNONODE) - throw Coordination::Exception(replica_active.error); - - if (replica_log_ptr.error != Coordination::Error::ZOK) - throw Coordination::Exception(replica_log_ptr.error); - - replicas_info[global_replica_index] = ReplicaInfo{ - .is_active = replica_active.error == Coordination::Error::ZOK, - .replication_lag = max_log_ptr - parse(replica_log_ptr.data), - .recovery_time = replica.is_local ? ddl_worker->getCurrentInitializationDurationMs() : 0, - }; - - ++global_replica_index; - } - } - - return replicas_info; - } catch (...) + return statuses; + } + 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 5a1570ae2e2..27ab262d1f1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -17,14 +17,6 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; -struct ReplicaInfo -{ - bool is_active; - UInt32 replication_lag; - UInt64 recovery_time; -}; -using ReplicasInfo = std::vector; - class DatabaseReplicated : public DatabaseAtomic { public: @@ -92,7 +84,7 @@ public: static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, bool throw_if_noop); - ReplicasInfo tryGetReplicasInfo(const ClusterPtr & cluster_) const; + std::vector tryGetAreReplicasActive(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 4e7408aa96e..1ef88dc03bc 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,12 +32,6 @@ 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 @@ -75,10 +69,6 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; - { - std::lock_guard lock(initialization_duration_timer_mutex); - initialization_duration_timer.reset(); - } return true; } catch (...) @@ -88,11 +78,6 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } - { - std::lock_guard lock(initialization_duration_timer_mutex); - initialization_duration_timer.reset(); - } - return false; } @@ -474,10 +459,4 @@ 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 2309c831839..41edf2221b8 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -36,8 +36,6 @@ public: DatabaseReplicated * const database, bool committed = false); /// NOLINT UInt32 getLogPointer() const; - - UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -58,9 +56,6 @@ 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 d03b600b6ef..160c8d6270e 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -31,8 +31,6 @@ 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({ @@ -48,30 +46,31 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co writeCluster(res_columns, name_and_cluster, {}); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & [database_name, database] : databases) + for (const auto & name_and_database : databases) { - if (const auto * replicated = typeid_cast(database.get())) + if (const auto * replicated = typeid_cast(name_and_database.second.get())) { + if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {database_name, database_cluster}, - replicated->tryGetReplicasInfo(database_cluster)); + writeCluster(res_columns, {name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + database_name, database_cluster}, - replicated->tryGetReplicasInfo(database_cluster)); + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); } } } void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const ReplicasInfo & replicas_info) + const std::vector & is_active) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); - size_t global_replica_idx = 0; + size_t replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { const auto & shard_info = shards_info[shard_index]; @@ -100,24 +99,10 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); res_columns[i++]->insert(address.database_shard_name); res_columns[i++]->insert(address.database_replica_name); - if (replicas_info.empty()) - { + if (is_active.empty()) res_columns[i++]->insertDefault(); - res_columns[i++]->insertDefault(); - res_columns[i++]->insertDefault(); - } else - { - const auto & replica_info = replicas_info[global_replica_idx]; - res_columns[i++]->insert(replica_info.is_active); - res_columns[i++]->insert(replica_info.replication_lag); - if (replica_info.recovery_time != 0) - res_columns[i++]->insert(replica_info.recovery_time); - else - res_columns[i++]->insertDefault(); - } - - ++global_replica_idx; + res_columns[i++]->insert(is_active[replica_idx++]); } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index f6e08734896..0f7c792261d 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 { @@ -27,7 +27,7 @@ protected: using NameAndCluster = std::pair>; void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const ReplicasInfo & replicas_info); + static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); }; } diff --git a/tests/integration/test_recovery_time_metric/__init__.py b/tests/integration/test_recovery_time_metric/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_recovery_time_metric/configs/config.xml b/tests/integration/test_recovery_time_metric/configs/config.xml deleted file mode 100644 index bad9b1fa9ea..00000000000 --- a/tests/integration/test_recovery_time_metric/configs/config.xml +++ /dev/null @@ -1,41 +0,0 @@ - - 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 deleted file mode 100644 index 6fcf2fad423..00000000000 --- a/tests/integration/test_recovery_time_metric/test.py +++ /dev/null @@ -1,61 +0,0 @@ -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 32e8b2f4312..cfae4fee6c2 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -52,8 +52,6 @@ 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 deleted file mode 100644 index 02f4a7264b1..00000000000 --- a/tests/queries/0_stateless/03206_replication_lag_metric.reference +++ /dev/null @@ -1,4 +0,0 @@ -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 deleted file mode 100644 index 998c332a11c..00000000000 --- a/tests/queries/0_stateless/03206_replication_lag_metric.sql +++ /dev/null @@ -1,11 +0,0 @@ --- 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 3dedd8d76b1c3fdb533e9f47d537e4cdf369af5e Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 3 Aug 2024 10:40:12 +0200 Subject: [PATCH 0767/1170] CI: Minor refactoring in ci_utils --- tests/ci/bugfix_validate_check.py | 5 +- tests/ci/ci.py | 8 +- tests/ci/ci_cache.py | 4 +- tests/ci/ci_config.py | 7 +- tests/ci/ci_definitions.py | 22 ----- tests/ci/ci_settings.py | 9 +- tests/ci/ci_utils.py | 150 +++++------------------------ tests/ci/report.py | 3 +- tests/ci/run_check.py | 152 +++++++++++++++++++++++++++--- tests/ci/test_ci_config.py | 12 ++- 10 files changed, 186 insertions(+), 186 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 71b18572938..932d709a7b8 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -8,7 +8,6 @@ from pathlib import Path from typing import List, Sequence, Tuple from ci_config import CI -from ci_utils import normalize_string from env_helper import TEMP_PATH from functional_test_check import NO_CHANGES_MSG from report import ( @@ -142,7 +141,9 @@ def main(): for file in set(jr.additional_files): file_ = Path(file) file_name = file_.name - file_name = file_name.replace(".", "__" + normalize_string(job_id) + ".", 1) + file_name = file_name.replace( + ".", "__" + CI.Utils.normalize_string(job_id) + ".", 1 + ) file_ = file_.rename(file_.parent / file_name) additional_files.append(file_) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 2565c8944e4..e36f2904182 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -16,7 +16,7 @@ import upload_result_helper from build_check import get_release_or_pr from ci_config import CI from ci_metadata import CiMetadata -from ci_utils import GH, normalize_string, Utils +from ci_utils import GH, Utils from clickhouse_helper import ( CiLogsCredentials, ClickHouseHelper, @@ -296,7 +296,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): # do not set report prefix for scheduled or dispatched wf (in case it started from feature branch while # testing), otherwise reports won't be found if not (pr_info.is_scheduled or pr_info.is_dispatched): - report_prefix = normalize_string(pr_info.head_ref) + report_prefix = Utils.normalize_string(pr_info.head_ref) print( f"Use report prefix [{report_prefix}], pr_num [{pr_info.number}], head_ref [{pr_info.head_ref}]" ) @@ -718,7 +718,7 @@ def _upload_build_artifacts( ( get_release_or_pr(pr_info, get_version_from_repo())[1], pr_info.sha, - normalize_string(build_name), + Utils.normalize_string(build_name), "performance.tar.zst", ) ) @@ -1248,7 +1248,7 @@ def main() -> int: ( get_release_or_pr(pr_info, get_version_from_repo())[0], pr_info.sha, - normalize_string( + Utils.normalize_string( job_report.check_name or _get_ext_check_name(args.job_name) ), ) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 4846233ab03..a59fd3e5a29 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -7,7 +7,7 @@ from typing import Dict, Optional, Any, Union, Sequence, List, Set from ci_config import CI -from ci_utils import is_hex, GH +from ci_utils import Utils, GH from commit_status_helper import CommitStatusData from env_helper import ( TEMP_PATH, @@ -240,7 +240,7 @@ class CiCache: int(job_properties[-1]), ) - if not is_hex(job_digest): + if not Utils.is_hex(job_digest): print("ERROR: wrong record job digest") return None diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c031ca9b805..ef48466e451 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -3,7 +3,7 @@ import re from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser from typing import Dict, Optional, List -from ci_utils import normalize_string +from ci_utils import Utils from ci_definitions import * @@ -13,7 +13,6 @@ class CI: each config item in the below dicts should be an instance of JobConfig class or inherited from it """ - MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI = 5 MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI = 2 # reimport types to CI class so that they visible as CI.* and mypy is happy @@ -37,9 +36,7 @@ class CI: from ci_utils import GH as GH from ci_utils import Shell as Shell from ci_definitions import Labels as Labels - from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS from ci_definitions import WorkFlowNames as WorkFlowNames - from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL # Jobs that run for doc related updates _DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] @@ -558,7 +555,7 @@ class CI: @classmethod def get_tag_config(cls, label_name: str) -> Optional[LabelConfig]: for label, config in cls.TAG_CONFIGS.items(): - if normalize_string(label_name) == normalize_string(label): + if Utils.normalize_string(label_name) == Utils.normalize_string(label): return config return None diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index de6791acda8..795bda3d4b0 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -32,28 +32,6 @@ class Labels: AUTO_BACKPORT = {"pr-critical-bugfix"} -TRUSTED_CONTRIBUTORS = { - e.lower() - for e in [ - "amosbird", - "azat", # SEMRush - "bharatnc", # Many contributions. - "cwurm", # ClickHouse, Inc - "den-crane", # Documentation contributor - "ildus", # adjust, ex-pgpro - "nvartolomei", # Seasoned contributor, CloudFlare - "taiyang-li", - "ucasFL", # Amos Bird's friend - "thomoco", # ClickHouse, Inc - "tonickkozlov", # Cloudflare - "tylerhannan", # ClickHouse, Inc - "tsolodov", # ClickHouse, Inc - "justindeguzman", # ClickHouse, Inc - "XuJia0210", # ClickHouse, Inc - ] -} - - class WorkflowStages(metaclass=WithIter): """ Stages of GitHUb actions workflow diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index d6e9765ceb7..05929179e06 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -2,7 +2,6 @@ import re from dataclasses import dataclass, asdict from typing import Optional, List, Dict, Any, Iterable -from ci_utils import normalize_string from ci_config import CI from git_helper import Runner as GitRunner, GIT_PREFIX from pr_info import PRInfo @@ -89,14 +88,14 @@ class CiSettings: if not res.include_keywords: res.include_keywords = [] res.include_keywords.append( - normalize_string(match.removeprefix("ci_include_")) + CI.Utils.normalize_string(match.removeprefix("ci_include_")) ) elif match.startswith("ci_exclude_"): if not res.exclude_keywords: res.exclude_keywords = [] keywords = match.removeprefix("ci_exclude_").split("|") res.exclude_keywords += [ - normalize_string(keyword) for keyword in keywords + CI.Utils.normalize_string(keyword) for keyword in keywords ] elif match == CI.Tags.NO_CI_CACHE: res.no_ci_cache = True @@ -163,7 +162,7 @@ class CiSettings: # do not exclude builds if self.exclude_keywords and not CI.is_build_job(job): for keyword in self.exclude_keywords: - if keyword in normalize_string(job): + if keyword in CI.Utils.normalize_string(job): print(f"Job [{job}] matches Exclude keyword [{keyword}] - deny") return False @@ -174,7 +173,7 @@ class CiSettings: # never exclude Style Check by include keywords return True for keyword in self.include_keywords: - if keyword in normalize_string(job): + if keyword in CI.Utils.normalize_string(job): print(f"Job [{job}] matches Include keyword [{keyword}] - pass") return True to_deny = True diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index dae1520afb6..067bedb19c3 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -6,7 +6,7 @@ import sys import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Sequence, Tuple +from typing import Any, Iterator, List, Union, Optional, Sequence import requests @@ -20,41 +20,6 @@ class Envs: GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") -LABEL_CATEGORIES = { - "pr-backward-incompatible": ["Backward Incompatible Change"], - "pr-bugfix": [ - "Bug Fix", - "Bug Fix (user-visible misbehavior in an official stable release)", - "Bug Fix (user-visible misbehaviour in official stable or prestable release)", - "Bug Fix (user-visible misbehavior in official stable or prestable release)", - ], - "pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"], - "pr-build": [ - "Build/Testing/Packaging Improvement", - "Build Improvement", - "Build/Testing Improvement", - "Build", - "Packaging Improvement", - ], - "pr-documentation": [ - "Documentation (changelog entry is not required)", - "Documentation", - ], - "pr-feature": ["New Feature"], - "pr-improvement": ["Improvement"], - "pr-not-for-changelog": [ - "Not for changelog (changelog entry is not required)", - "Not for changelog", - ], - "pr-performance": ["Performance Improvement"], - "pr-ci": ["CI Fix or Improvement (changelog entry is not required)"], -} - -CATEGORY_TO_LABEL = { - c: lb for lb, categories in LABEL_CATEGORIES.items() for c in categories -} - - class WithIter(type): def __iter__(cls): return (v for k, v in cls.__dict__.items() if not k.startswith("_")) @@ -70,21 +35,6 @@ def cd(path: Union[Path, str]) -> Iterator[None]: os.chdir(oldpwd) -def is_hex(s): - try: - int(s, 16) - return True - except ValueError: - return False - - -def normalize_string(string: str) -> str: - res = string.lower() - for r in ((" ", "_"), ("(", "_"), (")", "_"), (",", "_"), ("/", "_"), ("-", "_")): - res = res.replace(*r) - return res - - class GH: class ActionsNames: RunConfig = "RunConfig" @@ -149,8 +99,8 @@ class GH: ) -> str: assert len(token) == 40 assert len(commit_sha) == 40 - assert is_hex(commit_sha) - assert not is_hex(token) + assert Utils.is_hex(commit_sha) + assert not Utils.is_hex(token) url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses?per_page={200}" headers = { "Authorization": f"token {token}", @@ -298,79 +248,23 @@ class Utils: Shell.check("sudo dmesg --clear", verbose=True) @staticmethod - def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: - """The function checks the body to being properly formatted according to - .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, - then there is an error.""" - lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) - lines = [re.sub(r"\s+", " ", line) for line in lines] + def is_hex(s): + try: + int(s, 16) + return True + except ValueError: + return False - # Check if body contains "Reverts ClickHouse/ClickHouse#36337" - if [ - True for line in lines if re.match(rf"\AReverts {repo_name}#[\d]+\Z", line) - ]: - return "", LABEL_CATEGORIES["pr-not-for-changelog"][0] - - category = "" - entry = "" - description_error = "" - - i = 0 - while i < len(lines): - if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category - # itself. Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - - # Should not have more than one category. Require empty line - # after the first found category. - if i >= len(lines): - break - if lines[i]: - second_category = re.sub(r"^[-*\s]*", "", lines[i]) - description_error = ( - "More than one changelog category specified: " - f"'{category}', '{second_category}'" - ) - return description_error, category - - elif re.match( - r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] - ): - i += 1 - # Can have one empty line between header and the entry itself. - # Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - # Don't accept changelog entries like '...'. - entry = re.sub(r"[#>*_.\- ]", "", entry) - # Don't accept changelog entries like 'Close #12345'. - entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) - else: - i += 1 - - if not category: - description_error = "Changelog category is empty" - # Filter out the PR categories that are not for changelog. - elif "(changelog entry is not required)" in category: - pass # to not check the rest of the conditions - elif category not in CATEGORY_TO_LABEL: - description_error, category = f"Category '{category}' is not valid", "" - elif not entry: - description_error = f"Changelog entry required for category '{category}'" - - return description_error, category + @staticmethod + def normalize_string(string: str) -> str: + res = string.lower() + for r in ( + (" ", "_"), + ("(", "_"), + (")", "_"), + (",", "_"), + ("/", "_"), + ("-", "_"), + ): + res = res.replace(*r) + return res diff --git a/tests/ci/report.py b/tests/ci/report.py index f50ed4c1f85..f5571939d0b 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -22,7 +22,6 @@ from typing import ( from build_download_helper import get_gh_api from ci_config import CI -from ci_utils import normalize_string from env_helper import REPORT_PATH, GITHUB_WORKSPACE logger = logging.getLogger(__name__) @@ -622,7 +621,7 @@ class BuildResult: def write_json(self, directory: Union[Path, str] = REPORT_PATH) -> Path: path = Path(directory) / self.get_report_name( - self.build_name, self.pr_number or normalize_string(self.head_ref) + self.build_name, self.pr_number or CI.Utils.normalize_string(self.head_ref) ) path.write_text( json.dumps( diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 0ad01e3accd..55a0c383812 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 import logging +import re import sys from typing import Tuple @@ -16,7 +17,6 @@ from commit_status_helper import ( from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL from get_robot_token import get_best_robot_token from ci_config import CI -from ci_utils import Utils from pr_info import PRInfo from report import FAILURE, PENDING, SUCCESS, StatusType @@ -25,12 +25,144 @@ TRUSTED_ORG_IDS = { 54801242, # clickhouse } +TRUSTED_CONTRIBUTORS = { + e.lower() + for e in [ + "amosbird", + "azat", # SEMRush + "bharatnc", # Many contributions. + "cwurm", # ClickHouse, Inc + "den-crane", # Documentation contributor + "ildus", # adjust, ex-pgpro + "nvartolomei", # Seasoned contributor, CloudFlare + "taiyang-li", + "ucasFL", # Amos Bird's friend + "thomoco", # ClickHouse, Inc + "tonickkozlov", # Cloudflare + "tylerhannan", # ClickHouse, Inc + "tsolodov", # ClickHouse, Inc + "justindeguzman", # ClickHouse, Inc + "XuJia0210", # ClickHouse, Inc + ] +} + OK_SKIP_LABELS = {CI.Labels.RELEASE, CI.Labels.PR_BACKPORT, CI.Labels.PR_CHERRYPICK} PR_CHECK = "PR Check" +LABEL_CATEGORIES = { + "pr-backward-incompatible": ["Backward Incompatible Change"], + "pr-bugfix": [ + "Bug Fix", + "Bug Fix (user-visible misbehavior in an official stable release)", + "Bug Fix (user-visible misbehaviour in official stable or prestable release)", + "Bug Fix (user-visible misbehavior in official stable or prestable release)", + ], + "pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"], + "pr-build": [ + "Build/Testing/Packaging Improvement", + "Build Improvement", + "Build/Testing Improvement", + "Build", + "Packaging Improvement", + ], + "pr-documentation": [ + "Documentation (changelog entry is not required)", + "Documentation", + ], + "pr-feature": ["New Feature"], + "pr-improvement": ["Improvement"], + "pr-not-for-changelog": [ + "Not for changelog (changelog entry is not required)", + "Not for changelog", + ], + "pr-performance": ["Performance Improvement"], + "pr-ci": ["CI Fix or Improvement (changelog entry is not required)"], +} + +CATEGORY_TO_LABEL = { + c: lb for lb, categories in LABEL_CATEGORIES.items() for c in categories +} + + +def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: + """The function checks the body to being properly formatted according to + .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, + then there is an error.""" + lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) + lines = [re.sub(r"\s+", " ", line) for line in lines] + + # Check if body contains "Reverts ClickHouse/ClickHouse#36337" + if [True for line in lines if re.match(rf"\AReverts {repo_name}#[\d]+\Z", line)]: + return "", LABEL_CATEGORIES["pr-not-for-changelog"][0] + + category = "" + entry = "" + description_error = "" + + i = 0 + while i < len(lines): + if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): + i += 1 + if i >= len(lines): + break + # Can have one empty line between header and the category + # itself. Filter it out. + if not lines[i]: + i += 1 + if i >= len(lines): + break + category = re.sub(r"^[-*\s]*", "", lines[i]) + i += 1 + + # Should not have more than one category. Require empty line + # after the first found category. + if i >= len(lines): + break + if lines[i]: + second_category = re.sub(r"^[-*\s]*", "", lines[i]) + description_error = ( + "More than one changelog category specified: " + f"'{category}', '{second_category}'" + ) + return description_error, category + + elif re.match( + r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] + ): + i += 1 + # Can have one empty line between header and the entry itself. + # Filter it out. + if i < len(lines) and not lines[i]: + i += 1 + # All following lines until empty one are the changelog entry. + entry_lines = [] + while i < len(lines) and lines[i]: + entry_lines.append(lines[i]) + i += 1 + entry = " ".join(entry_lines) + # Don't accept changelog entries like '...'. + entry = re.sub(r"[#>*_.\- ]", "", entry) + # Don't accept changelog entries like 'Close #12345'. + entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) + else: + i += 1 + + if not category: + description_error = "Changelog category is empty" + # Filter out the PR categories that are not for changelog. + elif "(changelog entry is not required)" in category: + pass # to not check the rest of the conditions + elif category not in CATEGORY_TO_LABEL: + description_error, category = f"Category '{category}' is not valid", "" + elif not entry: + description_error = f"Changelog entry required for category '{category}'" + + return description_error, category + + def pr_is_by_trusted_user(pr_user_login, pr_user_orgs): - if pr_user_login.lower() in CI.TRUSTED_CONTRIBUTORS: + if pr_user_login.lower() in TRUSTED_CONTRIBUTORS: logging.info("User '%s' is trusted", pr_user_login) return True @@ -92,22 +224,20 @@ def main(): commit = get_commit(gh, pr_info.sha) status = SUCCESS # type: StatusType - description_error, category = Utils.check_pr_description( - pr_info.body, GITHUB_REPOSITORY - ) + description_error, category = check_pr_description(pr_info.body, GITHUB_REPOSITORY) pr_labels_to_add = [] pr_labels_to_remove = [] if ( - category in CI.CATEGORY_TO_LABEL - and CI.CATEGORY_TO_LABEL[category] not in pr_info.labels + category in CATEGORY_TO_LABEL + and CATEGORY_TO_LABEL[category] not in pr_info.labels ): - pr_labels_to_add.append(CI.CATEGORY_TO_LABEL[category]) + pr_labels_to_add.append(CATEGORY_TO_LABEL[category]) for label in pr_info.labels: if ( - label in CI.CATEGORY_TO_LABEL.values() - and category in CI.CATEGORY_TO_LABEL - and label != CI.CATEGORY_TO_LABEL[category] + label in CATEGORY_TO_LABEL.values() + and category in CATEGORY_TO_LABEL + and label != CATEGORY_TO_LABEL[category] ): pr_labels_to_remove.append(label) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index f376a129e6f..6ffedfdecd4 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -9,7 +9,7 @@ from ci_settings import CiSettings from pr_info import PRInfo, EventType from s3_helper import S3Helper from ci_cache import CiCache -from ci_utils import normalize_string +from ci_utils import Utils _TEST_EVENT_JSON = {"dummy": "dummy"} @@ -55,7 +55,7 @@ class TestCIConfig(unittest.TestCase): if CI.JOB_CONFIGS[job].job_name_keyword: self.assertTrue( CI.JOB_CONFIGS[job].job_name_keyword.lower() - in normalize_string(job), + in Utils.normalize_string(job), f"Job [{job}] apparently uses wrong common config with job keyword [{CI.JOB_CONFIGS[job].job_name_keyword}]", ) @@ -291,7 +291,9 @@ class TestCIConfig(unittest.TestCase): assert tag_config set_jobs = tag_config.run_jobs for job in set_jobs: - if any(k in normalize_string(job) for k in settings.exclude_keywords): + if any( + k in Utils.normalize_string(job) for k in settings.exclude_keywords + ): continue expected_jobs_to_do.append(job) for job, config in CI.JOB_CONFIGS.items(): @@ -303,12 +305,12 @@ class TestCIConfig(unittest.TestCase): # expected to run all builds jobs expected_jobs_to_do.append(job) if not any( - keyword in normalize_string(job) + keyword in Utils.normalize_string(job) for keyword in settings.include_keywords ): continue if any( - keyword in normalize_string(job) + keyword in Utils.normalize_string(job) for keyword in settings.exclude_keywords ): continue From a19750234153e760907f3c7bc040f949100534df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:11:20 +0200 Subject: [PATCH 0768/1170] Fix test retries Should fix issues like: - 02494_zero_copy_projection_cancel_fetch - https://s3.amazonaws.com/clickhouse-test-reports/67719/40cd5467c18d65a6624d273ac1a8fd9cc9257d8c/stateless_tests__tsan__s3_storage__[4_4].html Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a29c786e998..877548e577e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2218,7 +2218,6 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) - break except TimeoutError: break finally: From 087aff87dd23b4821965904e0a760b3ec7c4f8f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:12:07 +0200 Subject: [PATCH 0769/1170] Something --- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 8 ++++---- src/Databases/DatabaseOnDisk.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index ca30ee6db15..baa6910f6a8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -52,7 +52,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { - iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) + iterateMetadataFiles([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/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 01d8867661b..82a81b0b32d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -579,14 +579,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(local_context->getPath() + getDataPath()); + (void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path); (void)fs::remove_all(getMetadataPath()); } else { try { - (void)fs::remove(local_context->getPath() + getDataPath()); + (void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path); (void)fs::remove(getMetadataPath()); } catch (const fs::filesystem_error & e) @@ -624,7 +624,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n } } -void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const { if (!fs::exists(metadata_path)) return; @@ -635,7 +635,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat 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(local_context->getPath() + getDataPath() + '/' + object_name)) + if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / 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)); diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index a8be674a4e2..0c0ecf76a26 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -83,7 +83,7 @@ protected: using IteratingFunction = std::function; - void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const; ASTPtr getCreateTableQueryImpl( const String & table_name, diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 49719c25319..dd8a3f42ea8 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables } }; - iterateMetadataFiles(local_context, process_metadata); + iterateMetadataFiles(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; From ced8c3445c929efdb62ec707a7a8af7ef1a9f541 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:14:40 +0200 Subject: [PATCH 0770/1170] Something --- src/Databases/DatabaseLazy.cpp | 1 + src/Databases/DatabaseLazy.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index baa6910f6a8..e80ee930d79 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -47,6 +47,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, : DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { + createDirectories(); } diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 4347649117d..aeac130594f 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 in memory only cache. + * Works like DatabaseOrdinary, but stores only recently accessed tables in memory. * Can be used only with *Log engines. */ class DatabaseLazy final : public DatabaseOnDisk From f06ae2f5518ff8cb610b337d4900fd6f0088190f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:27:19 +0200 Subject: [PATCH 0771/1170] Fill only selected columns from system.clusters Some of them pretty heavy, i.e. is_active for ReplicatedDatabase This should fix 02903_rmt_retriable_merge_exception flakiness [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67687/89c47df559ba23d988f8af3c342e0c8d5531f4b8/fast_test.html Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemClusters.cpp | 82 ++++++++++++------- src/Storages/System/StorageSystemClusters.h | 6 +- 2 files changed, 56 insertions(+), 32 deletions(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 160c8d6270e..9c5c07ae49f 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -40,10 +40,10 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() return description; } -void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const { for (const auto & name_and_cluster : context->getClusters()) - writeCluster(res_columns, name_and_cluster, {}); + writeCluster(res_columns, columns_mask, name_and_cluster, /* replicated= */ nullptr); const auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & name_and_database : databases) @@ -52,18 +52,15 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co { if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {name_and_database.first, database_cluster}, replicated); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, replicated); } } } -void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const std::vector & is_active) +void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; @@ -79,30 +76,55 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index) { - size_t i = 0; + size_t src_index = 0, res_index = 0; const auto & address = shard_addresses[replica_index]; - res_columns[i++]->insert(cluster_name); - res_columns[i++]->insert(shard_info.shard_num); - res_columns[i++]->insert(shard_info.weight); - res_columns[i++]->insert(shard_info.has_internal_replication); - res_columns[i++]->insert(replica_index + 1); - res_columns[i++]->insert(address.host_name); - auto resolved = address.getResolvedAddress(); - res_columns[i++]->insert(resolved ? resolved->host().toString() : String()); - res_columns[i++]->insert(address.port); - res_columns[i++]->insert(address.is_local); - res_columns[i++]->insert(address.user); - res_columns[i++]->insert(address.default_database); - res_columns[i++]->insert(pool_status[replica_index].error_count); - res_columns[i++]->insert(pool_status[replica_index].slowdown_count); - res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); - res_columns[i++]->insert(address.database_shard_name); - res_columns[i++]->insert(address.database_replica_name); - if (is_active.empty()) - res_columns[i++]->insertDefault(); - else - res_columns[i++]->insert(is_active[replica_idx++]); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(cluster_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.shard_num); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.weight); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.has_internal_replication); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(replica_index + 1); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.host_name); + if (columns_mask[src_index++]) + { + auto resolved = address.getResolvedAddress(); + res_columns[res_index++]->insert(resolved ? resolved->host().toString() : String()); + } + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.port); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.is_local); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.user); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.default_database); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].error_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].slowdown_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].estimated_recovery_time.count()); + if (columns_mask[src_index++]) + 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()) + res_columns[res_index++]->insertDefault(); + else + res_columns[res_index++]->insert(is_active[replica_idx++]); + } } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 0f7c792261d..f6adb902f43 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -10,6 +10,7 @@ namespace DB class Context; class Cluster; +class DatabaseReplicated; /** Implements system table 'clusters' * that allows to obtain information about available clusters @@ -26,8 +27,9 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; using NameAndCluster = std::pair>; - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const override; + static void writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated); + bool supportsColumnsMask() const override { return true; } }; } From 9d0e066cda8d0ccb6bd4f9e07fee36a2bfae707a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:41:10 +0200 Subject: [PATCH 0772/1170] Bump NuRaft (to properly catch thread exceptions) Refs: https://github.com/ClickHouse/NuRaft/pull/75 Refs: https://github.com/eBay/NuRaft/pull/525 Signed-off-by: Azat Khuzhin --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce From 8562a6106c286882f26383086e52e399106893be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:45:18 +0200 Subject: [PATCH 0773/1170] Better safety thresholds in `arrayWithConstant` --- src/Functions/array/arrayWithConstant.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayWithConstant.cpp b/src/Functions/array/arrayWithConstant.cpp index 48262870553..4cbc6404b9b 100644 --- a/src/Functions/array/arrayWithConstant.cpp +++ b/src/Functions/array/arrayWithConstant.cpp @@ -1,9 +1,9 @@ #include -#include #include #include #include #include +#include namespace DB @@ -15,7 +15,8 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } -/// Reasonable threshold. +/// Reasonable thresholds. +static constexpr Int64 max_array_size_in_columns_bytes = 1000000000; static constexpr size_t max_arrays_size_in_columns = 1000000000; @@ -63,12 +64,19 @@ public: auto array_size = col_num->getInt(i); if (unlikely(array_size < 0)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size cannot be negative: while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} cannot be negative: while executing function {}", array_size, getName()); + + Int64 estimated_size = 0; + if (unlikely(common::mulOverflow(array_size, col_value->byteSize(), estimated_size))) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); + + if (unlikely(estimated_size > max_array_size_in_columns_bytes)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); offset += array_size; if (unlikely(offset > max_arrays_size_in_columns)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size {} (will generate at least {} elements) while executing function {}", array_size, offset, getName()); offsets.push_back(offset); } From 7a066a6505108b14bd49da8766c1bc473a978b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:48:51 +0200 Subject: [PATCH 0774/1170] Add a test --- .../0_stateless/03216_arrayWithConstant_limits.reference | 1 + tests/queries/0_stateless/03216_arrayWithConstant_limits.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.reference create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.sql diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference new file mode 100644 index 00000000000..825319e1c5b --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference @@ -0,0 +1 @@ +10000000 diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql new file mode 100644 index 00000000000..c46524c50e6 --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql @@ -0,0 +1,3 @@ +SELECT arrayWithConstant(96142475, ['qMUF']); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT arrayWithConstant(100000000, materialize([[[[[[[[[['Hello, world!']]]]]]]]]])); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT length(arrayWithConstant(10000000, materialize([[[[[[[[[['Hello world']]]]]]]]]]))); From 6013e4b81d29f2b46bec64859132a4e2ff1bbdc9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Jul 2024 20:52:48 +0200 Subject: [PATCH 0775/1170] poco/MongoDB: Support broader UUID types (generated by python uuid.UUID) Signed-off-by: Azat Khuzhin --- base/poco/MongoDB/src/Binary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/MongoDB/src/Binary.cpp b/base/poco/MongoDB/src/Binary.cpp index ea814d6969f..47f3453630f 100644 --- a/base/poco/MongoDB/src/Binary.cpp +++ b/base/poco/MongoDB/src/Binary.cpp @@ -76,7 +76,7 @@ std::string Binary::toString(int indent) const UUID Binary::uuid() const { - if (_subtype == 0x04 && _buffer.size() == 16) + if ((_subtype == 0x04 || _subtype == 0x03) && _buffer.size() == 16) { UUID uuid; uuid.copyFrom((const char*) _buffer.begin()); From 2f00c962711e13ca00af324366421fe4593b4ce6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Jul 2024 20:34:15 +0200 Subject: [PATCH 0776/1170] Support true UUID type for MongoDB engine Signed-off-by: Azat Khuzhin --- base/poco/Foundation/include/Poco/UUID.h | 7 +++ base/poco/MongoDB/src/Binary.cpp | 2 +- src/Processors/Sources/MongoDBSource.cpp | 46 +++++++++++++++++-- .../integration/test_storage_mongodb/test.py | 33 +++++++++++++ 4 files changed, 82 insertions(+), 6 deletions(-) diff --git a/base/poco/Foundation/include/Poco/UUID.h b/base/poco/Foundation/include/Poco/UUID.h index df67ef73e4b..6466d226b2e 100644 --- a/base/poco/Foundation/include/Poco/UUID.h +++ b/base/poco/Foundation/include/Poco/UUID.h @@ -19,6 +19,7 @@ #include "Poco/Foundation.h" +#include namespace Poco @@ -135,6 +136,12 @@ public: static const UUID & x500(); /// Returns the namespace identifier for the X500 namespace. + UInt32 getTimeLow() const { return _timeLow; } + UInt16 getTimeMid() const { return _timeMid; } + UInt16 getTimeHiAndVersion() const { return _timeHiAndVersion; } + UInt16 getClockSeq() const { return _clockSeq; } + std::array getNode() const { return std::array{_node[0], _node[1], _node[2], _node[3], _node[4], _node[5]}; } + protected: UUID(UInt32 timeLow, UInt32 timeMid, UInt32 timeHiAndVersion, UInt16 clockSeq, UInt8 node[]); UUID(const char * bytes, Version version); diff --git a/base/poco/MongoDB/src/Binary.cpp b/base/poco/MongoDB/src/Binary.cpp index 47f3453630f..8b0e6baeccb 100644 --- a/base/poco/MongoDB/src/Binary.cpp +++ b/base/poco/MongoDB/src/Binary.cpp @@ -82,7 +82,7 @@ UUID Binary::uuid() const uuid.copyFrom((const char*) _buffer.begin()); return uuid; } - throw BadCastException("Invalid subtype"); + throw BadCastException("Invalid subtype: " + std::to_string(_subtype) + ", size: " + std::to_string(_buffer.size())); } diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index 0d583cf6be5..e00a541b300 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -17,6 +18,7 @@ #include #include #include +#include "base/types.h" #include #include @@ -45,8 +47,28 @@ namespace using ValueType = ExternalResultDescription::ValueType; using ObjectId = Poco::MongoDB::ObjectId; using MongoArray = Poco::MongoDB::Array; + using MongoUUID = Poco::MongoDB::Binary::Ptr; + UUID parsePocoUUID(const Poco::UUID & src) + { + UUID uuid; + + std::array src_node = src.getNode(); + UInt64 node = 0; + node |= UInt64(src_node[0]) << 40; + node |= UInt64(src_node[1]) << 32; + node |= UInt64(src_node[2]) << 24; + node |= UInt64(src_node[3]) << 16; + node |= UInt64(src_node[4]) << 8; + node |= src_node[5]; + + UUIDHelpers::getHighBytes(uuid) = UInt64(src.getTimeLow()) << 32 | UInt32(src.getTimeMid() << 16 | src.getTimeHiAndVersion()); + UUIDHelpers::getLowBytes(uuid) = UInt64(src.getClockSeq()) << 48 | node; + + return uuid; + } + template Field getNumber(const Poco::MongoDB::Element & value, const std::string & name) { @@ -149,12 +171,20 @@ namespace else if (which.isUUID()) parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String (UUID), got type id = {} for column {}", + if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + String string = static_cast &>(value).value(); + return parse(string); + } + else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); + return parsePocoUUID(poco_uuid); + } + else + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", toString(value.type()), name); - String string = static_cast &>(value).value(); - return parse(string); }; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); @@ -286,8 +316,14 @@ namespace String string = static_cast &>(value).value(); assert_cast(column).getData().push_back(parse(string)); } + else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) + { + const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); + UUID uuid = parsePocoUUID(poco_uuid); + assert_cast(column).getData().push_back(uuid); + } else - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String (UUID), got type id = {} for column {}", + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", toString(value.type()), name); break; } diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 3957afe8b29..1a1a790e8e8 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -1,4 +1,5 @@ import pymongo +from uuid import UUID import pytest from helpers.client import QueryRuntimeException @@ -72,6 +73,28 @@ def test_simple_select(started_cluster): simple_mongo_table.drop() +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_uuid(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + mongo_table = db["uuid_table"] + mongo_table.insert({"key": 0, "data": UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed")}) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE uuid_mongo_table(key UInt64, data UUID) ENGINE = MongoDB('mongo1:27017', 'test', 'uuid_table', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM uuid_mongo_table") == "1\n" + assert ( + node.query("SELECT data from uuid_mongo_table where key = 0") + == "f0e77736-91d1-48ce-8f01-15123ca1c7ed\n" + ) + node.query("DROP TABLE uuid_mongo_table") + mongo_table.drop() + + @pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) def test_simple_select_from_view(started_cluster): mongo_connection = get_mongo_connection(started_cluster) @@ -140,6 +163,10 @@ def test_arrays(started_cluster): "f0e77736-91d1-48ce-8f01-15123ca1c7ed", "93376a07-c044-4281-a76e-ad27cf6973c5", ], + "arr_mongo_uuid": [ + UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed"), + UUID("93376a07-c044-4281-a76e-ad27cf6973c5"), + ], "arr_arr_bool": [ [True, False, True], [True], @@ -174,6 +201,7 @@ def test_arrays(started_cluster): "arr_datetime Array(DateTime)," "arr_string Array(String)," "arr_uuid Array(UUID)," + "arr_mongo_uuid Array(UUID)," "arr_arr_bool Array(Array(Bool))," "arr_empty Array(UInt64)," "arr_null Array(UInt64)," @@ -222,6 +250,11 @@ def test_arrays(started_cluster): == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" ) + assert ( + node.query(f"SELECT arr_mongo_uuid FROM arrays_mongo_table WHERE key = 42") + == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" + ) + assert ( node.query(f"SELECT arr_arr_bool FROM arrays_mongo_table WHERE key = 42") == "[[true,false,true],[true],[],[],[false],[false]]\n" From a051eb2a5d1c29a61d3c2d9e2f3f0841bbb2817c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 00:24:17 +0200 Subject: [PATCH 0777/1170] Fix tests --- src/Databases/DatabaseHDFS.cpp | 2 +- src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index f58f1b76e71..7fa67a5678e 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -51,7 +51,7 @@ DatabaseHDFS::DatabaseHDFS(const String & name_, const String & source_url, Cont if (!source.empty()) { if (!re2::RE2::FullMatch(source, std::string(HDFS_HOST_REGEXP))) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs host: {}. " + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS host: {}. " "It should have structure 'hdfs://:'", source); context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(source)); diff --git a/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp index 365828bcc40..7f8727eea1c 100644 --- a/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp +++ b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp @@ -192,7 +192,7 @@ String getNameNodeCluster(const String &hdfs_url) void checkHDFSURL(const String & url) { if (!re2::RE2::FullMatch(url, std::string(HDFS_URL_REGEXP))) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. It should have structure 'hdfs://:/'", url); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad HDFS URL: {}. It should have structure 'hdfs://:/'", url); } } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 47d8f44c0b7..ccd2c7eaf11 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -157,7 +157,7 @@ def test_bad_hdfs_uri(started_cluster): ) except Exception as ex: print(ex) - assert "Bad hdfs url" in str(ex) + assert "Bad HDFS URL" in str(ex) try: node1.query( "create table BadStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs100500:9000/other_storage', 'TSV')" From 185b6a54da8f24a97f130bdebe7bb1ec2bd266c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 00:34:58 +0200 Subject: [PATCH 0778/1170] Merge with master --- src/Core/SettingsChangesHistory.cpp | 261 +--------------------------- 1 file changed, 1 insertion(+), 260 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3f07bfdb933..5b94391bade 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +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", {{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."}}}, - {"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", { } @@ -335,6 +75,7 @@ static std::initializer_list Date: Sun, 4 Aug 2024 09:02:19 +0200 Subject: [PATCH 0779/1170] Use RabbitMQ without management in tests (attempt to improve startup) Sometimes startup can take ~90 seconds [1]: 2024-08-03 23:11:38.756067+00:00 [info] <0.9.0> Time to start RabbitMQ: 94651980 us Unlike normally ~10 [2]: 2024-08-04 00:33:07.016137+00:00 [info] <0.9.0> Time to start RabbitMQ: 10082489 us [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/b4e3bbcb82158bea4f5db1d9f5c28cfb741d1d51/integration_tests__asan__old_analyzer__[4_6].html [2]: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__asan__old_analyzer__[4_6].html I've tried locally, the difference is very small, 3135665 us (+management) vs (2740747 us), but still something, and who knows how it works under pressure. Signed-off-by: Azat Khuzhin --- tests/integration/compose/docker_compose_rabbitmq.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_rabbitmq.yml b/tests/integration/compose/docker_compose_rabbitmq.yml index 61b21e0e3d9..94c7f0111c4 100644 --- a/tests/integration/compose/docker_compose_rabbitmq.yml +++ b/tests/integration/compose/docker_compose_rabbitmq.yml @@ -2,7 +2,7 @@ version: '2.3' services: rabbitmq1: - image: rabbitmq:3.12.6-management-alpine + image: rabbitmq:3.12.6-alpine hostname: rabbitmq1 expose: - ${RABBITMQ_PORT:-5672} From 264be9c598b42d91ee0a19f718c4d9a4291c7bc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:06:50 +0200 Subject: [PATCH 0780/1170] tests: increase timeout for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9259c720ff0..a1bdee33d57 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2371,7 +2371,7 @@ class ClickHouseCluster: time.sleep(0.5) raise Exception("Cannot wait PostgreSQL Java Client container") - def wait_rabbitmq_to_start(self, timeout=30): + def wait_rabbitmq_to_start(self, timeout=60): self.print_all_docker_pieces() self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) From dc527b6fd1dfdffb1d177237bffc69cd110cd2a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:07:05 +0200 Subject: [PATCH 0781/1170] tests: detailed errors for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a1bdee33d57..56d111629c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2399,7 +2399,7 @@ class ClickHouseCluster: ) rabbitmq_debuginfo(self.rabbitmq_docker_id, self.rabbitmq_cookie) except Exception as e: - logging.debug("Unable to get logs from docker.") + logging.debug(f"Unable to get logs from docker: {e}.") raise Exception("Cannot wait RabbitMQ container") def wait_nats_is_available(self, max_retries=5): From 27db715761ec66b0dfc915269270697e6cf67909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 4 Aug 2024 09:52:36 +0000 Subject: [PATCH 0782/1170] Incorporate review feedback --- src/Functions/CountSubstringsImpl.h | 16 +++++++++----- src/Functions/MatchImpl.h | 34 ++++++++++++++--------------- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 8e91bc3aeb4..b1cefae6f1d 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -38,10 +38,10 @@ struct CountSubstringsImpl const ColumnPtr & start_pos, PaddedPODArray & res, [[maybe_unused]] ColumnUInt8 * res_null, - size_t /*input_rows_count*/) + size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); const UInt8 * const begin = haystack_data.data(); const UInt8 * const end = haystack_data.data() + haystack_data.size(); @@ -81,6 +81,8 @@ struct CountSubstringsImpl } pos = begin + haystack_offsets[i]; ++i; + + chassert(i < input_rows_count); } } @@ -116,7 +118,7 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); Impl::toLowerIfNeed(haystack); Impl::toLowerIfNeed(needle); @@ -154,8 +156,10 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null, size_t input_rows_count) { + chassert(input_rows_count == haystack_offsets.size()); + /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); ColumnString::Offset prev_haystack_offset = 0; ColumnString::Offset prev_needle_offset = 0; @@ -210,8 +214,10 @@ struct CountSubstringsImpl [[maybe_unused]] ColumnUInt8 * res_null, size_t input_rows_count) { + chassert(input_rows_count == needle_offsets.size()); + /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); /// NOTE You could use haystack indexing. But this is a rare case. ColumnString::Offset prev_needle_offset = 0; diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index ceac753fe79..7dc93ba79e0 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -131,11 +131,11 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == haystack_offsets.size()); - assert(res.size() == input_rows_count); - assert(start_pos_ == nullptr); + chassert(res.size() == haystack_offsets.size()); + chassert(res.size() == input_rows_count); + chassert(start_pos_ == nullptr); if (input_rows_count == 0) return; @@ -295,10 +295,10 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == haystack.size() / N); - assert(res.size() == input_rows_count); + chassert(res.size() == haystack.size() / N); + chassert(res.size() == input_rows_count); if (input_rows_count == 0) return; @@ -468,12 +468,12 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(haystack_offsets.size() == needle_offset.size()); - assert(res.size() == haystack_offsets.size()); - assert(res.size() == input_rows_count); - assert(start_pos_ == nullptr); + chassert(haystack_offsets.size() == needle_offset.size()); + chassert(res.size() == haystack_offsets.size()); + chassert(res.size() == input_rows_count); + chassert(start_pos_ == nullptr); if (input_rows_count == 0) return; @@ -577,12 +577,12 @@ struct MatchImpl size_t input_rows_count) { /// `res_null` serves as an output parameter for implementing an XYZOrNull variant. - assert(!res_null); + chassert(!res_null); - assert(res.size() == input_rows_count); - assert(res.size() == haystack.size() / N); - assert(res.size() == needle_offset.size()); - assert(start_pos_ == nullptr); + chassert(res.size() == input_rows_count); + chassert(res.size() == haystack.size() / N); + chassert(res.size() == needle_offset.size()); + chassert(start_pos_ == nullptr); if (haystack.empty()) return; From f0aaac3bd19c21a796f54ba080fd67f92959131a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:14:28 +0200 Subject: [PATCH 0783/1170] tests: remove useless retries from test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 3b79ea7916d..48a6224347d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -76,7 +76,7 @@ def get_used_disks_for_table(node, table_name, partition=None): ) -def check_used_disks_with_retry(node, table_name, expected_disks, retries): +def check_used_disks_with_retry(node, table_name, expected_disks, retries=1): for _ in range(retries): used_disks = get_used_disks_for_table(node, table_name) if set(used_disks).issubset(expected_disks): @@ -1635,9 +1635,9 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"]), 100) + assert check_used_disks_with_retry(node1, name, set(["external"])) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"]), 50) + assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) time.sleep(5) From 47dbc5e05b12213a08c25ade9536603a3fd2b175 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:16:17 +0200 Subject: [PATCH 0784/1170] tests: add debug info into test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 48a6224347d..4ebe9a30699 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1635,9 +1635,17 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"])) + assert check_used_disks_with_retry( + node1, name, set(["external"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) + assert check_used_disks_with_retry( + node1, name, set(["jbod1", "jbod2"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) time.sleep(5) From 062490e1b40a8df8d63fca567b11e7dd26cf52ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:22:24 +0200 Subject: [PATCH 0785/1170] tests: fix test_ttl_move::test_alter_with_merge_work flakiness MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Increase timeout for TTL DELETE, since otherwise if other routines will take too long, the part will be removed when it should be still be on "external" disk: 2024.08.04 03:48:53.803032 [ 622 ] {} default.mt_test_alter_with_merge_work_1722743323 (9dc6904a-f082-4f06-be7a-efe4733e811c): Will drop empty part all_1_3_4_4 And this is how part_log looks like: SELECT event_time, event_type, rows, part_name, error, database, disk_name FROM system.part_log WHERE `table` = 'mt_test_alter_with_merge_work_1722743323' ORDER BY event_time ASC Query id: a118b3cd-e4fe-45a5-b675-d73bdd887d79 ┌──────────event_time─┬─event_type─┬─rows─┬─part_name───┬─error─┬─database─┬─disk_name─┐ 1. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_1_1_0 │ 0 │ default │ jbod1 │ 2. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_2_2_0 │ 0 │ default │ jbod2 │ 3. │ 2024-08-04 03:48:45 │ NewPart │ 2 │ all_3_3_0 │ 0 │ default │ jbod1 │ 4. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_1_1_0_4 │ 0 │ default │ jbod1 │ 5. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_2_2_0_4 │ 0 │ default │ jbod2 │ 6. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod1 │ 7. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_1_1_0_4 │ 0 │ default │ external │ 8. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod2 │ 9. │ 2024-08-04 03:48:47 │ MergeParts │ 6 │ all_1_3_1_4 │ 0 │ default │ jbod2 │ 10. │ 2024-08-04 03:48:48 │ MovePart │ 6 │ all_1_3_1_4 │ 0 │ default │ external │ 11. │ 2024-08-04 03:48:52 │ MergeParts │ 4 │ all_1_3_2_4 │ 0 │ default │ external │ 12. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_3_4 │ 0 │ default │ external │ # rows==0 13. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_4_4 │ 0 │ default │ external │ └─────────────────────┴────────────┴──────┴─────────────┴───────┴──────────┴───────────┘ CI: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__tsan__[5_6].html Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 4ebe9a30699..925bdf9baaa 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1613,7 +1613,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): ALTER TABLE {name} MODIFY TTL d1 + INTERVAL 0 SECOND TO DISK 'jbod2', d1 + INTERVAL 5 SECOND TO VOLUME 'external', - d1 + INTERVAL 10 SECOND DELETE + d1 + INTERVAL 30 SECOND DELETE """.format( name=name ) @@ -1647,7 +1647,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" ) - time.sleep(5) + time.sleep(25) optimize_table(20) From 9f31488e502c2b2c02e3058f9794829aac14f8b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 15:43:18 +0200 Subject: [PATCH 0786/1170] Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading On CI you can find that 01747_executable_pool_dictionary_implicit_key can hang [1], it is possible due to after CANNOT_SCHEDULE_TASK the async loading will hang: 2024.07.18 03:56:32.365226 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} ExternalDictionariesLoader: Will load the object 'executable_pool_simple_implicit_key' in background, force = false, loading_id = 2 2024.07.18 03:56:32.368005 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} executeQuery: Code: 439. DB::Exception: Cannot schedule a task: fault injected (threads=766, jobs=746): In scope SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)). (CANNOT_SCHEDULE_TASK) (version 24.7.1.2241) (from [::1]:56446) (comment: 01747_executable_pool_dictionary_implicit_key.sql) (in query: SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1));), Stack trace (when copying this message, always include the lines below): 0. /build/contrib/llvm-project/libcxx/include/exception:141: Poco::Exception::Exception(String const&, int) @ 0x0000000015f8a292 1. /build/src/Common/Exception.cpp:110: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c3df6b9 2. /build/contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x0000000006de714c 3. /build/contrib/llvm-project/libcxx/include/vector:438: DB::Exception::Exception(int, FormatStringHelperImpl::type, std::type_identity::type, std::type_identity::type>, String const&, unsigned long&&, unsigned long&) @ 0x000000000c4838eb 4. /build/src/Common/ThreadPool.cpp:0: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda'(String const&)::operator()(String const&) const @ 0x000000000c4832d3 5. /build/src/Common/ThreadPool.cpp:186: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool) @ 0x000000000c47e7db 6. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000000c47ec8d 7. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001114b16e 8. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::ExternalLoader::LoadingDispatcher::startLoading(DB::ExternalLoader::LoadingDispatcher::Info&, bool, unsigned long) @ 0x0000000011147733 9. /build/src/Interpreters/ExternalLoader.cpp:837: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&)::'lambda'()::operator()() const @ 0x0000000011158bf9 10. /build/contrib/llvm-project/libcxx/include/__mutex_base:397: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&) @ 0x00000000111588bc 11. /build/src/Interpreters/ExternalLoader.cpp:604: DB::ExternalLoader::LoadResult DB::ExternalLoader::LoadingDispatcher::tryLoad(String const&, std::chrono::duration>) @ 0x00000000111440bf 12. /build/src/Interpreters/ExternalLoader.cpp:1381: std::shared_ptr DB::ExternalLoader::load, void>(String const&) const @ 0x00000000111442f5 13. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:587: DB::ExternalDictionariesLoader::getDictionary(String const&, std::shared_ptr) const @ 0x0000000011141028 14. /build/src/Functions/FunctionsExternalDictionaries.h:76: DB::FunctionDictHelper::getDictionary(String const&) @ 0x00000000071d28ec ... 2024.07.18 03:58:29.000900 [ 48468 ] {8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b} executeQuery: (from [::1]:40410) (comment: 01747_executable_pool_dictionary_implicit_key.sql) SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)); (stage: Complete) # and no more rows for 8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b [1]: https://s3.amazonaws.com/clickhouse-test-reports/66495/bc029ed8207ac75e96e9cb48cb79d27a9ffa4e2f/stress_test__debug_.html The problem that it should be properly cancelled, otherwise it will not be loaded in loadImpl(), but will be waited. Signed-off-by: Azat Khuzhin --- src/Interpreters/ExternalLoader.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 96405f35f3f..511300be2e0 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -922,7 +922,16 @@ private: if (enable_async_loading) { /// Put a job to the thread pool for the loading. - auto thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + ThreadFromGlobalPool thread; + try + { + thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + } + catch (...) + { + cancelLoading(info); + throw; + } loading_threads.try_emplace(loading_id, std::move(thread)); } else From dea95e6c332392a87ec3175bb32a50770a57d65b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:06:07 +0200 Subject: [PATCH 0787/1170] Ensure that llvm-symbolizer is used for symbolizing sanitizer reports Since you don't want to fallback to addr2line: # addr2line $ time /bin/test set_flag_impl: Success set_flag_if: Success /usr/bin/addr2line: DWARF error: invalid or unhandled FORM value: 0x23 ================== WARNING: ThreadSanitizer: data race (pid=18) ... real 3m8.580s user 0m21.967s sys 0m40.628s # llvm-symbolizer $ time ./test set_flag_impl: Success set_flag_if: Success ================== WARNING: ThreadSanitizer: data race (pid=24884) real 0m0.028s user 0m0.003s sys 0m0.006s Signed-off-by: Azat Khuzhin --- docker/test/base/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index a81826ed6b5..e7ac62604dc 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -35,7 +35,9 @@ ENV UBSAN_OPTIONS='print_stacktrace=1 max_allocation_size_mb=32768' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1 max_allocation_size_mb=32768' ENV LSAN_OPTIONS='max_allocation_size_mb=32768' -# for external_symbolizer_path +# for external_symbolizer_path, and also ensure that llvm-symbolizer really +# exists (since you don't want to fallback to addr2line, it is very slow) +RUN test -f /usr/bin/llvm-symbolizer-${LLVM_VERSION} RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 From 1f2bb6676f39259ec20dd0b077381d251b591d3a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:08:09 +0200 Subject: [PATCH 0788/1170] Remove outdated comments about workaround for #64086 Signed-off-by: Azat Khuzhin --- docker/test/util/Dockerfile | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index babddbd228c..dc928ba7195 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -5,14 +5,6 @@ FROM ubuntu:22.04 ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -# FIXME: rebuild for clang 18.1.3, that contains a workaround [1] for -# sanitizers issue [2]: -# -# $ git tag --contains c2a57034eff048cd36c563c8e0051db3a70991b3 | tail -1 -# llvmorg-18.1.3 -# -# [1]: https://github.com/llvm/llvm-project/commit/c2a57034eff048cd36c563c8e0051db3a70991b3 -# [2]: https://github.com/ClickHouse/ClickHouse/issues/64086 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ From eed2edd7db5f2424d85a66bfee63281cba5a5c94 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 Jul 2024 12:23:10 +0200 Subject: [PATCH 0789/1170] Comment certificateFile/privateKeyFile/dhParamsFile in keeper config Otherwise you will get annoying messages at startup: 2024.07.02 10:03:38.331593 [ 1 ] {} CertificateReloader: Cannot obtain modification time for certificate file /etc/clickhouse-keeper/server.crt, skipping update. errno: 2, strerror: 0 2024.07.02 10:03:38.331658 [ 1 ] {} CertificateReloader: Cannot obtain modification time for key file /etc/clickhouse-keeper/server.key, skipping update. errno: 2, strerror: 0 2024.07.02 10:03:38.341085 [ 1 ] {} CertificateReloader: Poco::Exception. Code: 1000, e.code() = 0, SSL context exception: Error loading private key from file /etc/clickhouse-keeper/server.key: error:80000002:system library::No such file or directory Signed-off-by: Azat Khuzhin --- programs/keeper/keeper_config.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/keeper/keeper_config.xml b/programs/keeper/keeper_config.xml index 4cf84cffc86..efd0010d184 100644 --- a/programs/keeper/keeper_config.xml +++ b/programs/keeper/keeper_config.xml @@ -66,14 +66,14 @@ - /etc/clickhouse-keeper/server.crt - /etc/clickhouse-keeper/server.key + + - /etc/clickhouse-keeper/dhparam.pem + none true true From a75c4b9f9fe519cb74a47d88efb55216da22a64b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:07:40 +0200 Subject: [PATCH 0790/1170] Introduce `no-flaky-check` tag --- tests/clickhouse-test | 3 +++ .../00002_log_and_exception_messages_formatting.sql | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 893fded8d23..b70dd61a25a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,6 +1232,9 @@ class TestCase: ): return FailureReason.SKIP + elif "no-flaky-check" in tags and (args.test_runs > 1): + return FailureReason.SKIP + elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 07c42d6d039..32db77cd8dd 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-ubsan, no-batch +-- Tags: no-parallel, no-fasttest, no-ubsan, no-batch, no-flaky-check -- no-parallel because we want to run this test when most of the other tests already passed -- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log From 87e0cf6b7635b7249d72c6ac0e7aad75c863d01d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:25:36 +0200 Subject: [PATCH 0791/1170] Fix stack overflow in JSONMergePatch --- src/Common/JSONParsers/RapidJSONParser.h | 12 ++++++--- src/Functions/jsonMergePatch.cpp | 34 +++++++++++++++--------- 2 files changed, 29 insertions(+), 17 deletions(-) diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 6c5ea938bfe..ad7a4cbf53a 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -3,10 +3,14 @@ #include "config.h" #if USE_RAPIDJSON -# include -# include -# include -# include "ElementTypes.h" + +/// Prevent stack overflow: +#define RAPIDJSON_PARSE_DEFAULT_FLAGS (kParseIterativeFlag) + +#include +#include +#include +#include "ElementTypes.h" namespace DB { diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index a83daacdbf6..3bde415aabf 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -10,12 +10,14 @@ #if USE_RAPIDJSON -#include "rapidjson/document.h" -#include "rapidjson/writer.h" -#include "rapidjson/stringbuffer.h" -#include "rapidjson/filewritestream.h" -#include "rapidjson/prettywriter.h" -#include "rapidjson/filereadstream.h" +/// Prevent stack overflow: +#define RAPIDJSON_PARSE_DEFAULT_FLAGS (kParseIterativeFlag) + +#include +#include +#include +#include +#include namespace DB @@ -31,17 +33,17 @@ namespace ErrorCodes namespace { - // select jsonMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); + // select JSONMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); // || // \/ // ┌───────────────────────┐ // │ {"a":1,"name":"zoey"} │ // └───────────────────────┘ - class FunctionjsonMergePatch : public IFunction + class FunctionJSONMergePatch : public IFunction { public: - static constexpr auto name = "jsonMergePatch"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static constexpr auto name = "JSONMergePatch"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -98,7 +100,11 @@ namespace const char * json = str_ref.data; document.Parse(json); - if (document.HasParseError() || !document.IsObject()) + + if (document.HasParseError()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong JSON string to merge: {}", rapidjson::GetParseError_En(document.GetParseError())); + + if (!document.IsObject()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong JSON string to merge. Expected JSON object"); }; @@ -162,10 +168,12 @@ namespace } -REGISTER_FUNCTION(jsonMergePatch) +REGISTER_FUNCTION(JSONMergePatch) { - factory.registerFunction(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description="Returns the merged JSON object string, which is formed by merging multiple JSON objects."}); + + factory.registerAlias("jsonMergePatch", "JSONMergePatch"); } } From 37da0b3c307d249a078b94cb7bc53d8acdd22a98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 18:27:37 +0200 Subject: [PATCH 0792/1170] Add a test --- .../03217_json_merge_patch_stack_overflow.reference | 0 .../03217_json_merge_patch_stack_overflow.sql | 9 +++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference create mode 100644 tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql diff --git a/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql new file mode 100644 index 00000000000..4b366b08c6b --- /dev/null +++ b/tests/queries/0_stateless/03217_json_merge_patch_stack_overflow.sql @@ -0,0 +1,9 @@ +-- Tags: no-fasttest +-- Needs rapidjson library +SELECT JSONMergePatch(REPEAT('{"c":', 1000000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 100000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 10000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 1000)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 100)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 10)); -- { serverError BAD_ARGUMENTS } +SELECT JSONMergePatch(REPEAT('{"c":', 1)); -- { serverError BAD_ARGUMENTS } From a433115434ebe4c1d69f2ed200005fa93c7adcb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 19:18:00 +0200 Subject: [PATCH 0793/1170] Fix typo --- docker/test/util/process_functional_tests_result.py | 6 +++--- docs/changelogs/v20.5.1.3833-prestable.md | 2 +- docs/changelogs/v21.11.1.8636-prestable.md | 6 +++--- docs/changelogs/v21.12.1.9017-prestable.md | 10 +++++----- docs/changelogs/v21.5.1.6601-prestable.md | 2 +- docs/changelogs/v21.6.9.7-stable.md | 2 +- docs/changelogs/v21.7.9.7-stable.md | 2 +- docs/changelogs/v21.8.1.7409-prestable.md | 2 +- docs/changelogs/v21.8.5.7-lts.md | 2 +- docs/changelogs/v21.9.1.8000-prestable.md | 2 +- docs/changelogs/v22.1.1.2542-prestable.md | 4 ++-- docs/changelogs/v22.4.1.2305-prestable.md | 2 +- docs/changelogs/v23.4.1.1943-stable.md | 3 +-- docs/changelogs/v23.6.1.1524-stable.md | 2 +- docs/zh/changelog/index.md | 2 +- src/Common/SystemLogBase.cpp | 2 +- 16 files changed, 25 insertions(+), 26 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 3da1a8f3674..aa2ea686c46 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -161,11 +161,11 @@ def process_result(result_path, broken_tests): retries, test_results, ) = process_test_log(result_path, broken_tests) - is_flacky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) - logging.info("Is flaky check: %s", is_flacky_check) + is_flaky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) + logging.info("Is flaky check: %s", is_flaky_check) # If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately) # But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped. - if failed != 0 or unknown != 0 or (success == 0 and (not is_flacky_check)): + if failed != 0 or unknown != 0 or (success == 0 and (not is_flaky_check)): state = "failure" if hung: diff --git a/docs/changelogs/v20.5.1.3833-prestable.md b/docs/changelogs/v20.5.1.3833-prestable.md index 79d61fb12f9..3de263f5e37 100644 --- a/docs/changelogs/v20.5.1.3833-prestable.md +++ b/docs/changelogs/v20.5.1.3833-prestable.md @@ -331,7 +331,7 @@ * Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). * Add a test for Join table engine from @donmikel. This closes [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Repeat test in CI if `curl` invocation was timed out. It is possible due to system hangups for 10+ seconds that are typical in our CI infrastructure. This fixes [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix potentially flacky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix potentially flaky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Now clickhouse-test check the server aliveness before tests run. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). * Emit a warning if server was build in debug or with sanitizers. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Better check for hung queries in clickhouse-test. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v21.11.1.8636-prestable.md b/docs/changelogs/v21.11.1.8636-prestable.md index d6a435dd3ce..95c8580d591 100644 --- a/docs/changelogs/v21.11.1.8636-prestable.md +++ b/docs/changelogs/v21.11.1.8636-prestable.md @@ -280,7 +280,7 @@ sidebar_label: 2022 * Cleanup unbundled image [#29689](https://github.com/ClickHouse/ClickHouse/pull/29689) ([Azat Khuzhin](https://github.com/azat)). * Fix memory tracking for merges and mutations [#29691](https://github.com/ClickHouse/ClickHouse/pull/29691) ([Azat Khuzhin](https://github.com/azat)). * Fix data-race in WriteIndirectBuffer (used in DiskMemory) [#29692](https://github.com/ClickHouse/ClickHouse/pull/29692) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). * BorrowedObjectPool condition variable notify fix [#29722](https://github.com/ClickHouse/ClickHouse/pull/29722) ([Maksim Kita](https://github.com/kitaisreal)). * Better exception message for local interactive [#29737](https://github.com/ClickHouse/ClickHouse/pull/29737) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix --stage for clickhouse-local [#29745](https://github.com/ClickHouse/ClickHouse/pull/29745) ([Azat Khuzhin](https://github.com/azat)). @@ -308,7 +308,7 @@ sidebar_label: 2022 * Fix client [#29864](https://github.com/ClickHouse/ClickHouse/pull/29864) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove some more streams. [#29898](https://github.com/ClickHouse/ClickHouse/pull/29898) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Add logging in ZooKeeper client [#29901](https://github.com/ClickHouse/ClickHouse/pull/29901) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix some flacky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix some flaky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). * Grep server log even if it contains binary data [#29903](https://github.com/ClickHouse/ClickHouse/pull/29903) ([Alexander Tokmakov](https://github.com/tavplubix)). * Cosmetic refactoring of server constants. [#29913](https://github.com/ClickHouse/ClickHouse/pull/29913) ([Amos Bird](https://github.com/amosbird)). * Format improvement of AlterQuery [#29916](https://github.com/ClickHouse/ClickHouse/pull/29916) ([flynn](https://github.com/ucasfl)). @@ -465,7 +465,7 @@ sidebar_label: 2022 * Fix docs release [#30933](https://github.com/ClickHouse/ClickHouse/pull/30933) ([alesapin](https://github.com/alesapin)). * Fix style check [#30937](https://github.com/ClickHouse/ClickHouse/pull/30937) ([alesapin](https://github.com/alesapin)). * Fix file progress for clickhouse-local [#30938](https://github.com/ClickHouse/ClickHouse/pull/30938) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix reading from TinyLog [#30941](https://github.com/ClickHouse/ClickHouse/pull/30941) ([Vitaly Baranov](https://github.com/vitlibar)). * Add github to known hosts in docs release [#30947](https://github.com/ClickHouse/ClickHouse/pull/30947) ([alesapin](https://github.com/alesapin)). * Parse json from response in ci checks [#30948](https://github.com/ClickHouse/ClickHouse/pull/30948) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.12.1.9017-prestable.md b/docs/changelogs/v21.12.1.9017-prestable.md index bd84873e67a..f5416664d35 100644 --- a/docs/changelogs/v21.12.1.9017-prestable.md +++ b/docs/changelogs/v21.12.1.9017-prestable.md @@ -220,7 +220,7 @@ sidebar_label: 2022 * Fix test_backward_compatibility [#30950](https://github.com/ClickHouse/ClickHouse/pull/30950) ([Ilya Yatsishin](https://github.com/qoega)). * Add stress test to github actions [#30952](https://github.com/ClickHouse/ClickHouse/pull/30952) ([alesapin](https://github.com/alesapin)). * Try smaller blacklist of non parallel integration tests [#30963](https://github.com/ClickHouse/ClickHouse/pull/30963) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). * Move access-rights source code [#30973](https://github.com/ClickHouse/ClickHouse/pull/30973) ([Vitaly Baranov](https://github.com/vitlibar)). * Set output_format_avro_rows_in_file default to 1 [#30990](https://github.com/ClickHouse/ClickHouse/pull/30990) ([Kruglov Pavel](https://github.com/Avogar)). * Remove remaining usages of Y_IGNORE [#30993](https://github.com/ClickHouse/ClickHouse/pull/30993) ([Yuriy Chernyshov](https://github.com/georgthegreat)). @@ -353,7 +353,7 @@ sidebar_label: 2022 * Support toUInt8/toInt8 for if constant condition optimization. [#31866](https://github.com/ClickHouse/ClickHouse/pull/31866) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Added -no-sanitize=unsigned-integer-overflow build flag [#31881](https://github.com/ClickHouse/ClickHouse/pull/31881) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix typos [#31886](https://github.com/ClickHouse/ClickHouse/pull/31886) ([Anton Popov](https://github.com/CurtizJ)). -* Try to fix flacky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Try to fix flaky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Reduce the files that depend on parser headers [#31896](https://github.com/ClickHouse/ClickHouse/pull/31896) ([Raúl Marín](https://github.com/Algunenano)). * Fix magic_enum for debug helpers (fixes build w/ USE_DEBUG_HELPERS) [#31922](https://github.com/ClickHouse/ClickHouse/pull/31922) ([Azat Khuzhin](https://github.com/azat)). * Remove some trash from build [#31923](https://github.com/ClickHouse/ClickHouse/pull/31923) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -387,7 +387,7 @@ sidebar_label: 2022 * make looping in H3 funcs uniform [#32110](https://github.com/ClickHouse/ClickHouse/pull/32110) ([Bharat Nallan](https://github.com/bharatnc)). * Remove PVS check from master [#32114](https://github.com/ClickHouse/ClickHouse/pull/32114) ([alesapin](https://github.com/alesapin)). * Fix flaky keeper whitelist test [#32115](https://github.com/ClickHouse/ClickHouse/pull/32115) ([alesapin](https://github.com/alesapin)). -* Fix flacky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix flaky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). * Fix data race in `removePartAndEnqueueFetch(...)` [#32119](https://github.com/ClickHouse/ClickHouse/pull/32119) ([Alexander Tokmakov](https://github.com/tavplubix)). * Move fuzzers and unit tests to another group [#32120](https://github.com/ClickHouse/ClickHouse/pull/32120) ([alesapin](https://github.com/alesapin)). * Add a test with 20000 mutations in one query [#32122](https://github.com/ClickHouse/ClickHouse/pull/32122) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). @@ -411,11 +411,11 @@ sidebar_label: 2022 * Add test for [#32186](https://github.com/ClickHouse/ClickHouse/issues/32186) [#32203](https://github.com/ClickHouse/ClickHouse/pull/32203) ([Raúl Marín](https://github.com/Algunenano)). * Fix uncaught exception in DatabaseLazy [#32206](https://github.com/ClickHouse/ClickHouse/pull/32206) ([Alexander Tokmakov](https://github.com/tavplubix)). * Update ASTCreateQuery.cpp [#32208](https://github.com/ClickHouse/ClickHouse/pull/32208) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix jemalloc under osx [#32219](https://github.com/ClickHouse/ClickHouse/pull/32219) ([Azat Khuzhin](https://github.com/azat)). * Add missing timezones to some tests [#32222](https://github.com/ClickHouse/ClickHouse/pull/32222) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix versioning of aggregate functions (fixes performance tests) [#32236](https://github.com/ClickHouse/ClickHouse/pull/32236) ([Azat Khuzhin](https://github.com/azat)). -* Disable window view tests temporarily because still flacky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disable window view tests temporarily because still flaky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix typo in tupleToNameValuePairs doc [#32262](https://github.com/ClickHouse/ClickHouse/pull/32262) ([Vladimir C](https://github.com/vdimir)). * Fix possible Pipeline stuck in case of StrictResize processor. [#32270](https://github.com/ClickHouse/ClickHouse/pull/32270) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix possible crash in DataTypeAggregateFunction [#32287](https://github.com/ClickHouse/ClickHouse/pull/32287) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). diff --git a/docs/changelogs/v21.5.1.6601-prestable.md b/docs/changelogs/v21.5.1.6601-prestable.md index b7dd8ae87c0..69ea9cb8d0e 100644 --- a/docs/changelogs/v21.5.1.6601-prestable.md +++ b/docs/changelogs/v21.5.1.6601-prestable.md @@ -158,7 +158,7 @@ sidebar_label: 2022 * MemoryStorage sync comments and code [#22721](https://github.com/ClickHouse/ClickHouse/pull/22721) ([Maksim Kita](https://github.com/kitaisreal)). * Fix potential segfault on Keeper startup [#22743](https://github.com/ClickHouse/ClickHouse/pull/22743) ([alesapin](https://github.com/alesapin)). * Avoid using harmful function rand() [#22744](https://github.com/ClickHouse/ClickHouse/pull/22744) ([Amos Bird](https://github.com/amosbird)). -* Fix flacky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix flaky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). * add more messages when flushing the logs [#22761](https://github.com/ClickHouse/ClickHouse/pull/22761) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Moved BorrowedObjectPool to common [#22764](https://github.com/ClickHouse/ClickHouse/pull/22764) ([Maksim Kita](https://github.com/kitaisreal)). * Functions ExternalDictionaries standardize exception throw [#22821](https://github.com/ClickHouse/ClickHouse/pull/22821) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v21.6.9.7-stable.md b/docs/changelogs/v21.6.9.7-stable.md index 0a989e4d6b7..533c58badac 100644 --- a/docs/changelogs/v21.6.9.7-stable.md +++ b/docs/changelogs/v21.6.9.7-stable.md @@ -55,7 +55,7 @@ sidebar_label: 2022 * Try fix rabbitmq tests [#26826](https://github.com/ClickHouse/ClickHouse/pull/26826) ([Kseniia Sumarokova](https://github.com/kssenii)). * One more library bridge fix [#26873](https://github.com/ClickHouse/ClickHouse/pull/26873) ([Kseniia Sumarokova](https://github.com/kssenii)). * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.7.9.7-stable.md b/docs/changelogs/v21.7.9.7-stable.md index 7aaab54af6b..684d0e8995e 100644 --- a/docs/changelogs/v21.7.9.7-stable.md +++ b/docs/changelogs/v21.7.9.7-stable.md @@ -35,7 +35,7 @@ sidebar_label: 2022 #### NOT FOR CHANGELOG / INSIGNIFICANT * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.8.1.7409-prestable.md b/docs/changelogs/v21.8.1.7409-prestable.md index cb6ab82b30f..6ef2f1b50d2 100644 --- a/docs/changelogs/v21.8.1.7409-prestable.md +++ b/docs/changelogs/v21.8.1.7409-prestable.md @@ -101,7 +101,7 @@ sidebar_label: 2022 * Separate log files for separate runs in stress test [#25741](https://github.com/ClickHouse/ClickHouse/pull/25741) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix slow performance test [#25742](https://github.com/ClickHouse/ClickHouse/pull/25742) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * DatabaseAtomic EXCHANGE DICTIONARIES fix test [#25753](https://github.com/ClickHouse/ClickHouse/pull/25753) ([Maksim Kita](https://github.com/kitaisreal)). -* Try fix flacky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Try fix flaky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a test for [#13993](https://github.com/ClickHouse/ClickHouse/issues/13993) [#25758](https://github.com/ClickHouse/ClickHouse/pull/25758) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Set follow-fork-mode child for gdb in stress/fasttest/fuzzer [#25769](https://github.com/ClickHouse/ClickHouse/pull/25769) ([Azat Khuzhin](https://github.com/azat)). * Ignore TOO_DEEP_RECURSION server exception during fuzzing [#25770](https://github.com/ClickHouse/ClickHouse/pull/25770) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.5.7-lts.md b/docs/changelogs/v21.8.5.7-lts.md index fa459e093f7..4d0727e362c 100644 --- a/docs/changelogs/v21.8.5.7-lts.md +++ b/docs/changelogs/v21.8.5.7-lts.md @@ -40,7 +40,7 @@ sidebar_label: 2022 * Fix several bugs in ZooKeeper snapshots deserialization [#26127](https://github.com/ClickHouse/ClickHouse/pull/26127) ([alesapin](https://github.com/alesapin)). * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.9.1.8000-prestable.md b/docs/changelogs/v21.9.1.8000-prestable.md index bc921a68693..70ae3697e92 100644 --- a/docs/changelogs/v21.9.1.8000-prestable.md +++ b/docs/changelogs/v21.9.1.8000-prestable.md @@ -346,7 +346,7 @@ sidebar_label: 2022 * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix 01300_client_save_history_when_terminated_long [#27324](https://github.com/ClickHouse/ClickHouse/pull/27324) ([Raúl Marín](https://github.com/Algunenano)). * Try update contrib/zlib-ng [#27327](https://github.com/ClickHouse/ClickHouse/pull/27327) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add and check system.mutations for database filter [#27384](https://github.com/ClickHouse/ClickHouse/pull/27384) ([Azat Khuzhin](https://github.com/azat)). * Correct the key data type used in mapContains [#27423](https://github.com/ClickHouse/ClickHouse/pull/27423) ([Fuwang Hu](https://github.com/fuwhu)). * Fix tests for WithMergeableStateAfterAggregationAndLimit [#27424](https://github.com/ClickHouse/ClickHouse/pull/27424) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.1.1.2542-prestable.md b/docs/changelogs/v22.1.1.2542-prestable.md index cacd13c1e12..3b0422abb11 100644 --- a/docs/changelogs/v22.1.1.2542-prestable.md +++ b/docs/changelogs/v22.1.1.2542-prestable.md @@ -398,7 +398,7 @@ sidebar_label: 2022 * test for [#24410](https://github.com/ClickHouse/ClickHouse/issues/24410) [#33265](https://github.com/ClickHouse/ClickHouse/pull/33265) ([Denny Crane](https://github.com/den-crane)). * Wait for RabbitMQ container to actually start when it was restarted in test on purpose [#33266](https://github.com/ClickHouse/ClickHouse/pull/33266) ([Kseniia Sumarokova](https://github.com/kssenii)). * Mark max_alter_threads as obsolete [#33268](https://github.com/ClickHouse/ClickHouse/pull/33268) ([Denny Crane](https://github.com/den-crane)). -* Fix azure tests flackyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix azure tests flakyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). * Test for [#26920](https://github.com/ClickHouse/ClickHouse/issues/26920) [#33272](https://github.com/ClickHouse/ClickHouse/pull/33272) ([Denny Crane](https://github.com/den-crane)). * Fix test_storage_kafka failures by adjusting retention.ms [#33278](https://github.com/ClickHouse/ClickHouse/pull/33278) ([Azat Khuzhin](https://github.com/azat)). * Disable FunctionConvertFromString::canBeExecutedOnDefaultArguments [#33286](https://github.com/ClickHouse/ClickHouse/pull/33286) ([Vladimir C](https://github.com/vdimir)). @@ -447,7 +447,7 @@ sidebar_label: 2022 * Update mongodb.md [#33585](https://github.com/ClickHouse/ClickHouse/pull/33585) ([Kseniia Sumarokova](https://github.com/kssenii)). * Restore existing static builds links [#33597](https://github.com/ClickHouse/ClickHouse/pull/33597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix pylint for run_check.py [#33600](https://github.com/ClickHouse/ClickHouse/pull/33600) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix flacky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). * Make ZooKeeper client better interpret keeper server connection reject [#33602](https://github.com/ClickHouse/ClickHouse/pull/33602) ([alesapin](https://github.com/alesapin)). * Fix broken workflow dependencies [#33608](https://github.com/ClickHouse/ClickHouse/pull/33608) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Force rebuild images in CI [#33609](https://github.com/ClickHouse/ClickHouse/pull/33609) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). diff --git a/docs/changelogs/v22.4.1.2305-prestable.md b/docs/changelogs/v22.4.1.2305-prestable.md index b277137ca7e..e8304e6505f 100644 --- a/docs/changelogs/v22.4.1.2305-prestable.md +++ b/docs/changelogs/v22.4.1.2305-prestable.md @@ -410,7 +410,7 @@ sidebar_label: 2022 * Fix mongodb test with new cert [#36161](https://github.com/ClickHouse/ClickHouse/pull/36161) ([alesapin](https://github.com/alesapin)). * Some fixes for ReplicatedMergeTree [#36163](https://github.com/ClickHouse/ClickHouse/pull/36163) ([Alexander Tokmakov](https://github.com/tavplubix)). * clickhouse-client: properly cancel query in case of error during formatting data [#36164](https://github.com/ClickHouse/ClickHouse/pull/36164) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). * Revert "Fix possible mutation stuck due to race with DROP_RANGE" [#36190](https://github.com/ClickHouse/ClickHouse/pull/36190) ([Azat Khuzhin](https://github.com/azat)). * Use atomic instead of mutex + condvar in ParallelReadBuffer [#36192](https://github.com/ClickHouse/ClickHouse/pull/36192) ([Kruglov Pavel](https://github.com/Avogar)). * Follow-up to [#36138](https://github.com/ClickHouse/ClickHouse/issues/36138) [#36194](https://github.com/ClickHouse/ClickHouse/pull/36194) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v23.4.1.1943-stable.md b/docs/changelogs/v23.4.1.1943-stable.md index ea16f5856be..34590ba9d37 100644 --- a/docs/changelogs/v23.4.1.1943-stable.md +++ b/docs/changelogs/v23.4.1.1943-stable.md @@ -321,7 +321,7 @@ sidebar_label: 2023 * Add a test for [#38128](https://github.com/ClickHouse/ClickHouse/issues/38128) [#48817](https://github.com/ClickHouse/ClickHouse/pull/48817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove excessive logging [#48826](https://github.com/ClickHouse/ClickHouse/pull/48826) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * remove duplicate indentwith in clang-format [#48834](https://github.com/ClickHouse/ClickHouse/pull/48834) ([cluster](https://github.com/infdahai)). -* Try fix flacky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). +* Try fix flaky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). * fix the race wait loading parts [#48844](https://github.com/ClickHouse/ClickHouse/pull/48844) ([Sema Checherinda](https://github.com/CheSema)). * suppress assert of progress for test_system_replicated_fetches [#48856](https://github.com/ClickHouse/ClickHouse/pull/48856) ([Han Fei](https://github.com/hanfei1991)). * Fix: do not run test_store_cleanup_disk_s3 in parallel [#48863](https://github.com/ClickHouse/ClickHouse/pull/48863) ([Igor Nikonov](https://github.com/devcrafter)). @@ -372,4 +372,3 @@ sidebar_label: 2023 * suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)). * Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)). * Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)). - diff --git a/docs/changelogs/v23.6.1.1524-stable.md b/docs/changelogs/v23.6.1.1524-stable.md index b91c5340789..0de9ab37653 100644 --- a/docs/changelogs/v23.6.1.1524-stable.md +++ b/docs/changelogs/v23.6.1.1524-stable.md @@ -263,7 +263,7 @@ sidebar_label: 2023 * Fix broken labeling for `manual approve` [#51405](https://github.com/ClickHouse/ClickHouse/pull/51405) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix parts lifetime in `MergeTreeTransaction` [#51407](https://github.com/ClickHouse/ClickHouse/pull/51407) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix flaky test test_skip_empty_files [#51409](https://github.com/ClickHouse/ClickHouse/pull/51409) ([Kruglov Pavel](https://github.com/Avogar)). -* fix flacky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). +* fix flaky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). * Update README.md [#51413](https://github.com/ClickHouse/ClickHouse/pull/51413) ([Tyler Hannan](https://github.com/tylerhannan)). * Replace try/catch logic in hasTokenOrNull() by something more lightweight [#51425](https://github.com/ClickHouse/ClickHouse/pull/51425) ([Robert Schulze](https://github.com/rschu1ze)). * Add retries to `tlsv1_3` tests [#51434](https://github.com/ClickHouse/ClickHouse/pull/51434) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index cd77a8c03cf..fb50dfcee85 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -252,7 +252,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" - 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) - 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复flacky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复flaky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 清理重复的链接器标志。 确保链接器不会查找意想不到的符号。 [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([阿莫斯鸟](https://github.com/amosbird)) - 添加 `clickhouse-odbc` 驱动程序进入测试图像。 这允许通过自己的ODBC驱动程序测试ClickHouse与ClickHouse的交互。 [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) - 修复单元测试中的几个错误。 [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([阿利沙平](https://github.com/alesapin)) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index a9307c3be99..7d2c15714e2 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -65,7 +65,7 @@ void SystemLogQueue::push(LogElement&& element) /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. - /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. + /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flaky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; /// Should not log messages under mutex. From bd5022432909bcb633f31617a764ca223ad585d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 20:57:22 +0200 Subject: [PATCH 0794/1170] Whitespaces --- docs/changelogs/v23.4.1.1943-stable.md | 1 - src/Processors/Chunk.h | 2 +- tests/queries/0_stateless/02099_tsv_raw_format.sh | 7 +++---- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/changelogs/v23.4.1.1943-stable.md b/docs/changelogs/v23.4.1.1943-stable.md index ea16f5856be..6ab6d8b457a 100644 --- a/docs/changelogs/v23.4.1.1943-stable.md +++ b/docs/changelogs/v23.4.1.1943-stable.md @@ -372,4 +372,3 @@ sidebar_label: 2023 * suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)). * Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)). * Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)). - diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 1348966c0d3..f45e2c4619e 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -22,7 +22,7 @@ public: }; -template +template class ChunkInfoCloneable : public ChunkInfo { public: diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh index 16b695e4037..026607ac6d5 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -14,7 +14,7 @@ do echo $format $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 SELECT number, toString(number), toDate(number) FROM numbers(3)" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" - + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099" @@ -49,13 +49,12 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_string_02099" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()"; -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099" -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099" - From a49924b3834d577827279f6278e5df21e3065035 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 13:27:06 +0200 Subject: [PATCH 0795/1170] tests: avoid leaving leftovers after test_storage_mongodb (fixes flaky chec) CI: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- tests/integration/test_storage_mongodb/test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 1a1a790e8e8..6e2d15b03be 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -410,6 +410,7 @@ def test_no_credentials(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table_2") node.query( "create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')" ) @@ -439,10 +440,13 @@ def test_auth_source(started_cluster): simple_mongo_table.insert_many(data) node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table_fail") node.query( "create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail") + + node.query("drop table if exists simple_mongo_table_ok") node.query( "create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')" ) From 9f1753bb4f825349ac4238650bc87d7792f99cc8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 2 Aug 2024 16:58:04 +0200 Subject: [PATCH 0796/1170] 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 c200f437746ddbd03e5ce7c7ad9d9613015ce929 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 16:00:46 +0200 Subject: [PATCH 0797/1170] ci: fix stateless runner to correctly gather artifacts when server failed Right now it simply does not work due to "set -e", with it you cannot use "foo=$(false)" since bash will break execution after, rewrite it to a plain shell, with "if". Also use ZSTD everywhere (ugh) Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..5c15c05652b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -299,22 +299,22 @@ stop_logs_replication failed_to_save_logs=0 for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log do - err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes") - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 - err=$( { clickhouse-client --port 29000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client --port 19000 -q "select * from system.$table into outfile '/test_output/$table.1.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi + if ! clickhouse-client --port 29000 -q "select * from system.$table into outfile '/test_output/$table.2.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi fi if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then - err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) - echo "$err" - [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + if ! clickhouse-client --port 29000 -q "select * from system.$table into outfile '/test_output/$table.2.tsv.zst' format TSVWithNamesAndTypes"; then + failed_to_save_logs=1 + fi fi done From 43cf85ef2841190a5d12a592a65a5181bb110661 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 16:09:48 +0200 Subject: [PATCH 0798/1170] ci: collect basic issues in stateless tests (dmesg, fatal and similar) Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 9 +++++++++ docker/test/stateless/stress_tests.lib | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5c15c05652b..b352539cc1a 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -3,6 +3,12 @@ # shellcheck disable=SC1091 source /setup_export_logs.sh +# shellcheck source=../stateless/stress_tests.lib +source /stress_tests.lib + +# Avoid overlaps with previous runs +dmesg --clear + # fail on errors, verbose and export all env variables set -e -x -a @@ -420,4 +426,7 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: fi +# Grep logs for sanitizer asserts, crashes and other critical errors +check_logs_for_critical_errors + collect_core_dumps diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 682da1df837..36782101fa7 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -242,7 +242,7 @@ function check_server_start() function check_logs_for_critical_errors() { # Sanitizer asserts - sed -n '/WARNING:.*anitizer/,/^$/p' /var/log/clickhouse-server/stderr.log >> /test_output/tmp + sed -n '/WARNING:.*anitizer/,/^$/p' /var/log/clickhouse-server/stderr*.log >> /test_output/tmp rg -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ && echo -e "Sanitizer assert (in stderr.log)$FAIL$(head_escaped /test_output/tmp)" >> /test_output/test_results.tsv \ || echo -e "No sanitizer asserts$OK" >> /test_output/test_results.tsv From 83be27cdf59ec5886abc23441f5cf92740b04c57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:34:01 +0200 Subject: [PATCH 0799/1170] Fix test `02833_concurrent_sessions`, Fix test `02835_drop_user_during_session` --- .../0_stateless/02833_concurrent_sessions.sh | 18 ++++++++++++- .../02835_drop_user_during_session.sh | 26 +++++++++++++++---- 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02833_concurrent_sessions.sh b/tests/queries/0_stateless/02833_concurrent_sessions.sh index 846661cfeed..feaff3a38a3 100755 --- a/tests/queries/0_stateless/02833_concurrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrent_sessions.sh @@ -137,7 +137,23 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN ( for user in "${ALL_USERS[@]}"; do ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "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')" + + # 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 true + do + [[ 3 -eq $(${CLICKHOUSE_CLIENT} -q " + 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' + )") ]] && echo 3 && break; + sleep 0.1 + done + echo "LoginFailure" ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" done diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index 347ebd22f96..c32003a2a11 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -24,7 +24,7 @@ function http_session() ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" } -function http_with_session_id_session() +function http_with_session_id_session() { local user=$1 ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" @@ -104,11 +104,27 @@ wait ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -echo "port_0_sessions:" +echo "port_0_sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" echo "address_0_sessions:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "Corresponding LoginSuccess/Logout" + +# 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 true +do + [[ 9 -eq $(${CLICKHOUSE_CLIENT} -q " + SELECT COUNT(*) FROM ( + SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' + INTERSECT + SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' + )") ]] && echo 9 && break; + sleep 0.1 +done + echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From d5629655c77653f51e44e1bfa9f8935b000da891 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sun, 4 Aug 2024 21:52:29 +0000 Subject: [PATCH 0800/1170] 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 0bdaa57023ef69e49ab5cf9d54ed2e52c1fd2dae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:53:54 +0200 Subject: [PATCH 0801/1170] Fix diagnostics in the test script --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..bec8b9cc4d1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -260,7 +260,7 @@ function run_tests() | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e - DURATION=$((START_TIME - SECONDS)) + DURATION=$((SECONDS - START_TIME)) echo "Elapsed ${DURATION} seconds." if [[ $DURATION -ge $TIMEOUT ]] From a573b2926e13ea31c7947b0429b1b5723c7fb938 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:59:01 +0200 Subject: [PATCH 0802/1170] Fixes for the script --- docker/test/stateless/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..8b9e729970c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -72,8 +72,12 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th remove_keeper_config "latest_logs_cache_size_threshold" "[[:digit:]]\+" fi +export IS_FLAKY_CHECK=0 + # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then + export IS_FLAKY_CHECK=1 + export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 export THREAD_FUZZER_SLEEP_PROBABILITY=0.1 export THREAD_FUZZER_SLEEP_TIME_US_MAX=100000 From 538761b43dbf704d0700548e61b4034ef66c5766 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:02:32 +0200 Subject: [PATCH 0803/1170] Fix flaky check --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b70dd61a25a..38b0e99760e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,7 +1232,7 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and (args.test_runs > 1): + elif "no-flaky-check" in tags and (1 == int(os.environ.get("IS_FLAKY_CHECK", 0))): return FailureReason.SKIP elif tags: From 270dddc00fd5533ba24914e7d9ae0aebc50a3fd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:22:48 +0200 Subject: [PATCH 0804/1170] Fix test `02231_bloom_filter_sizing` --- tests/queries/0_stateless/02231_bloom_filter_sizing.reference | 4 ++-- tests/queries/0_stateless/02231_bloom_filter_sizing.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02231_bloom_filter_sizing.reference b/tests/queries/0_stateless/02231_bloom_filter_sizing.reference index bdba311c092..aabadfc92fd 100644 --- a/tests/queries/0_stateless/02231_bloom_filter_sizing.reference +++ b/tests/queries/0_stateless/02231_bloom_filter_sizing.reference @@ -1,6 +1,6 @@ Bloom filter on sort key -10000 +1000 0 Bloom filter on non-sort key -10000 +1000 0 diff --git a/tests/queries/0_stateless/02231_bloom_filter_sizing.sql b/tests/queries/0_stateless/02231_bloom_filter_sizing.sql index 233e3111067..ee896675d64 100644 --- a/tests/queries/0_stateless/02231_bloom_filter_sizing.sql +++ b/tests/queries/0_stateless/02231_bloom_filter_sizing.sql @@ -12,7 +12,7 @@ INSERT INTO bloom_filter_sizing_pk SELECT number % 100 as key, -- 100 unique keys number as value -- whatever -FROM numbers(1000 * 1000); +FROM numbers(100_000); -- -- Merge everything into a single part @@ -40,7 +40,7 @@ SELECT number % 100 as key1, -- 100 unique keys rand() % 100 as key2, -- 100 unique keys number as value -- whatever -FROM numbers(1000 * 1000); +FROM numbers(100_000); -- -- Merge everything into a single part From b2ec479cee2a07f7e1fe0a8384a36b5ffc999fda Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 4 Aug 2024 22:23:08 +0000 Subject: [PATCH 0805/1170] 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 5a860fcc3aaf4e77025cd9286da10d0587134c3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:25:56 +0200 Subject: [PATCH 0806/1170] Update 02099_tsv_raw_format.sh --- tests/queries/0_stateless/02099_tsv_raw_format.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh index 026607ac6d5..a69c96ab613 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 6897178c41286fd3d4f3064ad8e2c716d12e002f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 4 Aug 2024 22:29:03 +0000 Subject: [PATCH 0807/1170] 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 7adeaf9c28016b29453d6be2bca1aa1ef866ecdf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 4 Aug 2024 22:35:07 +0000 Subject: [PATCH 0808/1170] Automatic style fix --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 38b0e99760e..907d773337a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,7 +1232,9 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and (1 == int(os.environ.get("IS_FLAKY_CHECK", 0))): + elif "no-flaky-check" in tags and ( + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + ): return FailureReason.SKIP elif tags: From 6b1e184e12bed759487d89f54f5ac4f269dffda2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 22:50:46 +0200 Subject: [PATCH 0809/1170] Print original query for AST formatting check on CI It may be tricky to understand the root cause of the AST formatting issue in case of syntax error, so add one knob to control this - debug_ast_formatting_print_original_query. And CI contains core dumps anyway, so let's enable for CI. P.S. There was concern from @al13n321 that printing original query even in debug build is not a good idea [1], hence a knob for this. [1]: https://github.com/ClickHouse/ClickHouse/pull/63357/files#r1674809348 Signed-off-by: Azat Khuzhin --- src/Interpreters/executeQuery.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ba6fc0f14a0..ce58f7f922c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -796,10 +796,9 @@ static std::tuple executeQueryImpl( catch (const Exception & e) { if (e.code() == ErrorCodes::SYNTAX_ERROR) - /// Don't print the original query text because it may contain sensitive data. throw Exception(ErrorCodes::LOGICAL_ERROR, - "Inconsistent AST formatting: the query:\n{}\ncannot parse.", - formatted1); + "Inconsistent AST formatting: the query:\n{}\ncannot parse query back from {}", + formatted1, std::string_view(begin, end-begin)); else throw; } From 8bca80f4dd08d8ad05db0325b96365d82e6c4076 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 22:35:21 +0200 Subject: [PATCH 0810/1170] Fix REPLACE modifier formatting (forbid omitting brackets) It is too tricky to verify does brackets required or not, i.e. "SELECT * REPLACE(1/3/3 AS dummy)" will be formatted to "SELECT * REPLACE (1/3)/3 AS dummy" which is already invalid query. So let's simply always print them. Signed-off-by: Azat Khuzhin v2: move the fix into correct place ASTColumnsReplaceTransformer::formatImpl() instead of ASTColumnsReplaceTransformer::Replacement::formatImpl() --- src/Parsers/ASTColumnsTransformers.cpp | 8 ++------ .../03220_replace_formatting.reference | 16 ++++++++++++++++ .../0_stateless/03220_replace_formatting.sh | 14 ++++++++++++++ 3 files changed, 32 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03220_replace_formatting.reference create mode 100755 tests/queries/0_stateless/03220_replace_formatting.sh diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 2a61892f8cc..332ebca3bdb 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -323,9 +323,7 @@ void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, F { settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (is_strict ? " STRICT " : " ") << (settings.hilite ? hilite_none : ""); - if (children.size() > 1) - settings.ostr << "("; - + settings.ostr << "("; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { if (it != children.begin()) @@ -333,9 +331,7 @@ void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, F (*it)->formatImpl(settings, state, frame); } - - if (children.size() > 1) - settings.ostr << ")"; + settings.ostr << ")"; } void ASTColumnsReplaceTransformer::appendColumnName(WriteBuffer & ostr) const diff --git a/tests/queries/0_stateless/03220_replace_formatting.reference b/tests/queries/0_stateless/03220_replace_formatting.reference new file mode 100644 index 00000000000..cbcd63839b1 --- /dev/null +++ b/tests/queries/0_stateless/03220_replace_formatting.reference @@ -0,0 +1,16 @@ +SELECT * REPLACE ((1 / 3) / 3 AS dummy) +SELECT * REPLACE ((1 / 3) / 3 AS dummy) +SELECT * REPLACE STRICT (1 AS id, 2 AS value) +FROM +( + SELECT + 0 AS id, + 1 AS value +) +SELECT * REPLACE STRICT (1 AS id, 2 AS value) +FROM +( + SELECT + 0 AS id, + 1 AS value +) diff --git a/tests/queries/0_stateless/03220_replace_formatting.sh b/tests/queries/0_stateless/03220_replace_formatting.sh new file mode 100755 index 00000000000..1c11ed6da8d --- /dev/null +++ b/tests/queries/0_stateless/03220_replace_formatting.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +q=$($CLICKHOUSE_FORMAT <<<"SELECT * REPLACE(1/3/3 AS dummy)") +echo "$q" +$CLICKHOUSE_FORMAT <<<"$q" + +# multiple columns +q=$($CLICKHOUSE_FORMAT <<<"SELECT * REPLACE STRICT (1 AS id, 2 AS value) FROM (SELECT 0 id, 1 value)") +echo "$q" +$CLICKHOUSE_FORMAT <<<"$q" From 2a7ad3a1f979708fd152e364296c505db8926aba Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 23:21:19 +0200 Subject: [PATCH 0811/1170] Update test references for new REPLACE modifier syntax Signed-off-by: Azat Khuzhin --- .../01913_fix_column_transformer_replace_format.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference b/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference index 33be11c07d5..6fabd33c804 100644 --- a/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference +++ b/tests/queries/0_stateless/01913_fix_column_transformer_replace_format.reference @@ -1 +1 @@ -CREATE VIEW default.my_view\n(\n `Id` UInt32,\n `Object.Key` Array(UInt16),\n `Object.Value` Array(String)\n)\nAS SELECT * REPLACE arrayMap(x -> (x + 1), `Object.Key`) AS `Object.Key`\nFROM default.my_table +CREATE VIEW default.my_view\n(\n `Id` UInt32,\n `Object.Key` Array(UInt16),\n `Object.Value` Array(String)\n)\nAS SELECT * REPLACE (arrayMap(x -> (x + 1), `Object.Key`) AS `Object.Key`)\nFROM default.my_table From e5134e14ea68ff4d02bde892a7c66d00d4c1e800 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Mon, 5 Aug 2024 07:59:51 +0000 Subject: [PATCH 0812/1170] 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 0813/1170] 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 cebdc5ecf6b7e41cf3c75fd8ea9765972afa084c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 15:19:51 +0000 Subject: [PATCH 0814/1170] Bump rocksdb to v8.0.0 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 1 + src/Coordination/KeeperContext.cpp | 7 ++++--- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 17 ++++++++++------- tests/config/config.d/rocksdb.xml | 3 ++- 5 files changed, 18 insertions(+), 12 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 01e43568fa9..fdf403f5918 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 01e43568fa9f3f7bf107b2b66c00b286b456f33e +Subproject commit fdf403f5918a2b4355cf75ebe5e21d0fc22db880 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 98790158baa..8660bd0e7ba 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -309,6 +309,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc ${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc ${ROCKSDB_SOURCE_DIR}/util/crc32c.cc + ${ROCKSDB_SOURCE_DIR}/util/data_structure.cc ${ROCKSDB_SOURCE_DIR}/util/dynamic_bloom.cc ${ROCKSDB_SOURCE_DIR}/util/hash.cc ${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 1f66882ecad..dd2c1d59d56 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -23,6 +23,7 @@ #if USE_ROCKSDB #include #include +#include #include #endif @@ -88,7 +89,7 @@ static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractCo if (config.has("keeper_server.rocksdb.options")) { auto config_options = getOptionsFromConfig(config, "keeper_server.rocksdb.options"); - status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged); + status = rocksdb::GetDBOptionsFromMap({}, merged, config_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' : {}", @@ -98,7 +99,7 @@ static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractCo if (config.has("rocksdb.column_family_options")) { auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options"); - status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged); + status = rocksdb::GetColumnFamilyOptionsFromMap({}, merged, column_family_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.column_family_options' at: {}", status.ToString()); @@ -107,7 +108,7 @@ static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractCo if (config.has("rocksdb.block_based_table_options")) { auto block_based_table_options = getOptionsFromConfig(config, "rocksdb.block_based_table_options"); - status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options); + status = rocksdb::GetBlockBasedTableOptionsFromMap({}, table_options, block_based_table_options, &table_options); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.block_based_table_options' at: {}", status.ToString()); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index fafc72da04e..50f6266cb2f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -36,10 +36,12 @@ #include #include +#include +#include #include #include +#include #include -#include #include #include @@ -428,6 +430,7 @@ void StorageEmbeddedRocksDB::initDB() rocksdb::Options base; base.create_if_missing = true; + base.compression = rocksdb::CompressionType::kZSTD; base.statistics = rocksdb::CreateDBStatistics(); /// It is too verbose by default, and in fact we don't care about rocksdb logs at all. base.info_log_level = rocksdb::ERROR_LEVEL; @@ -439,7 +442,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has("rocksdb.options")) { auto config_options = getOptionsFromConfig(config, "rocksdb.options"); - status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged); + status = rocksdb::GetDBOptionsFromMap({}, merged, config_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' at: {}: {}", @@ -449,7 +452,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has("rocksdb.column_family_options")) { auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options"); - status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged); + status = rocksdb::GetColumnFamilyOptionsFromMap({}, merged, column_family_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.column_family_options' at: {}: {}", @@ -459,7 +462,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has("rocksdb.block_based_table_options")) { auto block_based_table_options = getOptionsFromConfig(config, "rocksdb.block_based_table_options"); - status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options); + status = rocksdb::GetBlockBasedTableOptionsFromMap({}, table_options, block_based_table_options, &table_options); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.block_based_table_options' at: {}: {}", @@ -484,7 +487,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has(config_key)) { auto table_config_options = getOptionsFromConfig(config, config_key); - status = rocksdb::GetDBOptionsFromMap(merged, table_config_options, &merged); + status = rocksdb::GetDBOptionsFromMap({}, merged, table_config_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from '{}' at: {}: {}", @@ -496,7 +499,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has(config_key)) { auto table_column_family_options = getOptionsFromConfig(config, config_key); - status = rocksdb::GetColumnFamilyOptionsFromMap(merged, table_column_family_options, &merged); + status = rocksdb::GetColumnFamilyOptionsFromMap({}, merged, table_column_family_options, &merged); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from '{}' at: {}: {}", @@ -508,7 +511,7 @@ void StorageEmbeddedRocksDB::initDB() if (config.has(config_key)) { auto block_based_table_options = getOptionsFromConfig(config, config_key); - status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options); + status = rocksdb::GetBlockBasedTableOptionsFromMap({}, table_options, block_based_table_options, &table_options); if (!status.ok()) { throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from '{}' at: {}: {}", diff --git a/tests/config/config.d/rocksdb.xml b/tests/config/config.d/rocksdb.xml index a3790a3dc1d..3002e008a2d 100644 --- a/tests/config/config.d/rocksdb.xml +++ b/tests/config/config.d/rocksdb.xml @@ -1,7 +1,8 @@ - DEBUG_LEVEL + + ERROR_LEVEL From d0bc728d52140b60a4c8b9d24d2c4dd4cb9582a8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 Jul 2024 19:09:45 +0000 Subject: [PATCH 0815/1170] Bump rocksdb to v8.9.1 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 23 ++++++++----------- src/Coordination/tests/gtest_coordination.cpp | 11 +++++---- 3 files changed, 17 insertions(+), 19 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index fdf403f5918..49ce8a1064d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit fdf403f5918a2b4355cf75ebe5e21d0fc22db880 +Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 8660bd0e7ba..57c056532c6 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_ROCKSDB "Enable RocksDB" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_ROCKSDB) +if (NOT ENABLE_ROCKSDB OR NO_SSE3_OR_HIGHER) # assumes SSE4.2 and PCLMUL message (STATUS "Not using RocksDB") return() endif() @@ -39,13 +39,6 @@ if(WITH_ZSTD) list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) endif() -add_definitions(-DROCKSDB_PORTABLE) - -if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) - add_definitions(-DHAVE_SSE42) - add_definitions(-DHAVE_PCLMUL) -endif() - if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") set (HAS_ARMV8_CRC 1) # the original build descriptions set specific flags for ARM. These flags are already subsumed by ClickHouse's general @@ -91,7 +84,9 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/compressed_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/secondary_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/secondary_cache_adapter.cc ${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/blob/blob_contents.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc @@ -174,9 +169,11 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc ${ROCKSDB_SOURCE_DIR}/db/wide/wide_column_serialization.cc ${ROCKSDB_SOURCE_DIR}/db/wide/wide_columns.cc + ${ROCKSDB_SOURCE_DIR}/db/wide/wide_columns_helper.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch.cc ${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc ${ROCKSDB_SOURCE_DIR}/db/write_controller.cc + ${ROCKSDB_SOURCE_DIR}/db/write_stall_stats.cc ${ROCKSDB_SOURCE_DIR}/db/write_thread.cc ${ROCKSDB_SOURCE_DIR}/env/composite_env.cc ${ROCKSDB_SOURCE_DIR}/env/env.cc @@ -229,6 +226,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/options/configurable.cc ${ROCKSDB_SOURCE_DIR}/options/customizable.cc ${ROCKSDB_SOURCE_DIR}/options/db_options.cc + ${ROCKSDB_SOURCE_DIR}/options/offpeak_time_info.cc ${ROCKSDB_SOURCE_DIR}/options/options.cc ${ROCKSDB_SOURCE_DIR}/options/options_helper.cc ${ROCKSDB_SOURCE_DIR}/options/options_parser.cc @@ -268,6 +266,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/table/get_context.cc ${ROCKSDB_SOURCE_DIR}/table/iterator.cc ${ROCKSDB_SOURCE_DIR}/table/merging_iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/compaction_merging_iterator.cc ${ROCKSDB_SOURCE_DIR}/table/meta_blocks.cc ${ROCKSDB_SOURCE_DIR}/table/persistent_cache_helper.cc ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_bloom.cc @@ -323,6 +322,8 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/string_util.cc ${ROCKSDB_SOURCE_DIR}/util/thread_local.cc ${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc + ${ROCKSDB_SOURCE_DIR}/util/udt_util.cc + ${ROCKSDB_SOURCE_DIR}/util/write_batch_util.cc ${ROCKSDB_SOURCE_DIR}/util/xxhash.cc ${ROCKSDB_SOURCE_DIR}/utilities/agg_merge/agg_merge.cc ${ROCKSDB_SOURCE_DIR}/utilities/backup/backup_engine.cc @@ -405,12 +406,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc build_version.cc) # generated by hand -if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) - set_source_files_properties( - "${ROCKSDB_SOURCE_DIR}/util/crc32c.cc" - PROPERTIES COMPILE_FLAGS "-msse4.2 -mpclmul") -endif() - if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") list(APPEND SOURCES "${ROCKSDB_SOURCE_DIR}/util/crc32c_ppc.c" diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index beae6254562..d39031773cd 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -111,10 +111,13 @@ public: } }; -using Implementation = testing::Types, - TestParam, - TestParam, - TestParam>; +using Implementation = testing::Types + ,TestParam +#if USE_ROCKSDB + ,TestParam + ,TestParam +#endif + >; TYPED_TEST_SUITE(CoordinationTest, Implementation); TYPED_TEST(CoordinationTest, RaftServerConfigParse) From b47f8a733f926c0f52a5837414bcd6ecfece9089 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 08:40:35 +0200 Subject: [PATCH 0816/1170] ci: fix basic errors collecting after stateless tests Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b352539cc1a..c582d3a982b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -393,6 +393,8 @@ do | zstd --threads=0 > "/test_output/trace-log-$trace_type-flamegraph.tsv.zst" ||: done +# Grep logs for sanitizer asserts, crashes and other critical errors +check_logs_for_critical_errors # Compressed (FIXME: remove once only github actions will be left) rm /var/log/clickhouse-server/clickhouse-server.log @@ -426,7 +428,4 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: fi -# Grep logs for sanitizer asserts, crashes and other critical errors -check_logs_for_critical_errors - collect_core_dumps From cdbc4f357324ad0b41d46b6e54475ac2cebdc630 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 08:45:20 +0200 Subject: [PATCH 0817/1170] ci: fail the test if the entrypoint script failed Signed-off-by: Azat Khuzhin --- tests/ci/functional_test_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 66db082677f..52970404d2d 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -166,6 +166,7 @@ def _get_statless_tests_to_run(pr_info: PRInfo) -> List[str]: def process_results( + ret_code: int, result_directory: Path, server_log_path: Path, ) -> Tuple[StatusType, str, TestResults, List[Path]]: @@ -192,6 +193,9 @@ def process_results( logging.info("Files in result folder %s", os.listdir(result_directory)) return ERROR, "Invalid check_status.tsv", test_results, additional_files state, description = status[0][0], status[0][1] + if ret_code != 0: + state = ERROR + description += " (but script exited with an error)" try: results_path = result_directory / "test_results.tsv" @@ -339,7 +343,7 @@ def main(): ci_logs_credentials.clean_ci_logs_from_credentials(run_log_path) state, description, test_results, additional_logs = process_results( - result_path, server_log_path + retcode, result_path, server_log_path ) else: print( From 9ce55b69b49ac8426ed0b3db16b95964e3c4db4d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 11:22:55 +0200 Subject: [PATCH 0818/1170] Fix possible CANNOT_READ_ALL_DATA during server startup in performance tests CI [1]: 2024.08.04 22:09:11.646800 [ 1052 ] {} Application: Code: 33. DB::Exception: Cannot read all data. Bytes read: 0. Bytes expected: 4.: While checking access for disk backups. (CANNOT_READ_ALL_DATA), Stack trace (when copying this message, always include the lines below): [1]: https://s3.amazonaws.com/clickhouse-test-reports/64955/6702acf6f2e4a0ee9697066e38006631fc7f69df/performance_comparison__aarch64__[2_4].html Signed-off-by: Azat Khuzhin --- tests/performance/scripts/compare.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/performance/scripts/compare.sh b/tests/performance/scripts/compare.sh index cb56ab6c5bf..da7bbf77a28 100755 --- a/tests/performance/scripts/compare.sh +++ b/tests/performance/scripts/compare.sh @@ -71,6 +71,8 @@ function configure { # Use the new config for both servers, so that we can change it in a PR. rm right/config/config.d/text_log.xml ||: + # backups disk uses absolute path, and this overlaps between servers, that could lead to errors + rm right/config/config.d/backups.xml ||: cp -rv right/config left ||: # Start a temporary server to rename the tables From a499cd25c7e12c05f2f8fa3fe546715c751ad88d Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:31:41 +0000 Subject: [PATCH 0819/1170] Fix for integers --- src/Functions/if.cpp | 16 ++++++---------- ...3215_varian_as_common_type_integers.reference | 8 ++++++++ .../03215_varian_as_common_type_integers.sql | 8 ++++++++ 3 files changed, 22 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_integers.reference create mode 100644 tests/queries/0_stateless/03215_varian_as_common_type_integers.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 64da6e95a43..8829b3c4ff1 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -632,11 +632,6 @@ private: ColumnPtr executeTuple(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - /// For different Tuples the result type can be Variant with this Tuples if use_variant_as_common_type=1. - /// In this case we should use generic implementation. - if (!isTuple(result_type)) - return nullptr; - /// Calculate function for each corresponding elements of tuples. const ColumnWithTypeAndName & arg1 = arguments[1]; @@ -682,11 +677,6 @@ private: ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - /// For different Maps the result type can be Variant with this Maps if use_variant_as_common_type=1. - /// In this case we should use generic implementation. - if (!isMap(result_type)) - return nullptr; - auto extract_kv_from_map = [](const ColumnMap * map) { const ColumnTuple & tuple = map->getNestedData(); @@ -1243,6 +1233,12 @@ public: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " "Must be ColumnUInt8 or ColumnConstUInt8.", arg_cond.column->getName(), getName()); + /// If result is Variant, always use generic implementation. + /// Using typed implementations may lead to incorrect result column type when + /// resulting Variant is created by use_variant_when_no_common_type. + if (isVariant(result_type)) + return executeGeneric(cond_col, arguments, input_rows_count, use_variant_when_no_common_type); + auto call = [&](const auto & types) -> bool { using Types = std::decay_t; diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference b/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference new file mode 100644 index 00000000000..c5edc9e9963 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_integers.reference @@ -0,0 +1,8 @@ +0 Variant(Int64, UInt64) +1 Variant(Int64, UInt64) +0 Variant(Int32, UInt64) +1 Variant(Int32, UInt64) +0 Variant(Int16, UInt64) +1 Variant(Int16, UInt64) +0 Variant(Int8, UInt64) +1 Variant(Int8, UInt64) diff --git a/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql b/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql new file mode 100644 index 00000000000..dcc69735534 --- /dev/null +++ b/tests/queries/0_stateless/03215_varian_as_common_type_integers.sql @@ -0,0 +1,8 @@ +set use_variant_as_common_type = 1; +set allow_experimental_variant_type = 1; + +SELECT if(number % 2, number::Int64, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int32, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int16, number::UInt64) as res, toTypeName(res) FROM numbers(2); +SELECT if(number % 2, number::Int8, number::UInt64) as res, toTypeName(res) FROM numbers(2); + From 2b369cccdd58902e9da3fd3947e5cb5759ba2881 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:35:56 +0000 Subject: [PATCH 0820/1170] Reduce table size in 03037_dynamic_merges_2_vertical_wide_merge_tree test --- ...3037_dynamic_merges_2_vertical_wide_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_vertical_wide_merge_tree.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index 104d6018e41..dd643f8dffd 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); From cec8a5d52b83f0c1cdcaed833aec9bf79941b2a8 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:42:22 +0000 Subject: [PATCH 0821/1170] Reduce table size in similar tests --- ...dynamic_merges_2_horizontal_compact_merge_tree.reference | 6 +++--- ...03037_dynamic_merges_2_horizontal_compact_merge_tree.sql | 6 +++--- ...37_dynamic_merges_2_horizontal_wide_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_horizontal_wide_merge_tree.sql | 6 +++--- ...7_dynamic_merges_2_vertical_compact_merge_tree.reference | 6 +++--- .../03037_dynamic_merges_2_vertical_compact_merge_tree.sql | 6 +++--- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index e133ac3001f..fa64ed2f8fd 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index d527081b763..4b8a036f166 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference index afd392002e5..253d87de5f0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.reference @@ -1,3 +1,3 @@ -1000000 Array(UInt16) -1000000 String -1000000 UInt64 +200000 Array(UInt16) +200000 String +200000 UInt64 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index ebccfb77922..a4e67de76db 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -5,9 +5,9 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; -insert into test select number, number from numbers(1000000); -insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); -insert into test select number, range(number % 10 + 1) from numbers(2000000, 1000000); +insert into test select number, number from numbers(200000); +insert into test select number, 'str_' || toString(number) from numbers(200000, 200000); +insert into test select number, range(number % 10 + 1) from numbers(400000, 200000); system start merges test; optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); From bd6c7a504d214db606b7ba41bd3f9e172df8ae68 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 5 Aug 2024 11:45:49 +0200 Subject: [PATCH 0822/1170] 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 e0362b00f5a24ea19e16b27b71963efe64a174c0 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Mon, 5 Aug 2024 11:52:07 +0200 Subject: [PATCH 0823/1170] squash! fix for parallel execution --- tests/integration/test_parquet_page_index/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_parquet_page_index/test.py b/tests/integration/test_parquet_page_index/test.py index 59dbab09be5..778b6618a61 100644 --- a/tests/integration/test_parquet_page_index/test.py +++ b/tests/integration/test_parquet_page_index/test.py @@ -35,7 +35,7 @@ def delete_if_exists(file_path): "query, expected_result", { ( - "SElECT number, number+1 FROM system.numbers LIMIT 100 " + "SELECT number, number+1 FROM system.numbers LIMIT 100 " "INTO OUTFILE '{file_name}' FORMAT Parquet " "SETTINGS output_format_parquet_use_custom_encoder = false, " "output_format_parquet_write_page_index = true;", From 0a7a67b8e0e05bc9476d4f9dd38747bf61b6bb8e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 09:57:13 +0000 Subject: [PATCH 0824/1170] Disable 03038_nested_dynamic_merges* under sanitizers because it's too slow --- ...sted_dynamic_merges_compact_horizontal.sql | 2 +- ...nested_dynamic_merges_compact_vertical.sql | 2 +- ...8_nested_dynamic_merges_small.reference.j2 | 84 +++++++++++++++++++ .../03038_nested_dynamic_merges_small.sql.j2 | 35 ++++++++ ..._nested_dynamic_merges_wide_horizontal.sql | 2 +- ...38_nested_dynamic_merges_wide_vertical.sql | 2 +- 6 files changed, 123 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 create mode 100644 tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 1d5c63dcdf1..81888946681 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index 2bffe35c577..ba58ca471a2 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 new file mode 100644 index 00000000000..ae07c164074 --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 @@ -0,0 +1,84 @@ +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None +2 Tuple(a Dynamic(max_types=3)):Date +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):String +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 UInt64:None +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 UInt64:None +2 Tuple(a Dynamic(max_types=3)):DateTime +3 Tuple(a Dynamic(max_types=3)):Array(UInt8) +5 Tuple(a Dynamic(max_types=3)):UInt64 +7 Tuple(a Dynamic(max_types=3)):String +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +13 Tuple(a Dynamic(max_types=3)):None +5 Tuple(a Dynamic(max_types=3)):UInt64 +10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) +10 UInt64:None +12 Tuple(a Dynamic(max_types=3)):String +13 Tuple(a Dynamic(max_types=3)):None diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 new file mode 100644 index 00000000000..7828c2af49c --- /dev/null +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 @@ -0,0 +1,35 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; + +drop table if exists test; + +{% for engine in ['MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', + 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1', + 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', + 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1'] -%} + +create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600; + +system stop merges test; +insert into test select number, number from numbers(10); +insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +system start merges test; +optimize table test final; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; + +drop table test; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index fb686091ebb..a53c5b0b2a5 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index ed195452d56..4256b010ec0 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From b53e757656e298bb308862a8294cde5718e37580 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Jul 2024 10:01:50 +0000 Subject: [PATCH 0825/1170] Check argument types in DataTypeAggregateFunction ctor --- src/DataTypes/DataTypeAggregateFunction.cpp | 27 +++++++++++++++++++++ src/DataTypes/DataTypeAggregateFunction.h | 8 +----- 2 files changed, 28 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 09175617bf1..ee42e4fea11 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -33,6 +33,33 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +DataTypeAggregateFunction::DataTypeAggregateFunction(AggregateFunctionPtr function_, const DataTypes & argument_types_, + const Array & parameters_, std::optional version_) + : function(std::move(function_)) + , argument_types(argument_types_) + , parameters(parameters_) + , version(version_) +{ + Strings argument_type_names; + for (const auto & argument_type : argument_types) + argument_type_names.push_back(argument_type->getName()); + + Strings function_argument_type_names; + const auto & function_argument_types = function->getArgumentTypes(); + for (const auto & argument_type : function_argument_types) + function_argument_type_names.push_back(argument_type->getName()); + + size_t argument_types_size = std::max(argument_types.size(), function_argument_types.size()); + for (size_t i = 0; i < argument_types_size; ++i) + { + if (argument_types.size() != function_argument_types.size() || !argument_types[i]->equals(*function_argument_types[i])) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Data type AggregateFunction {} got argument types different from function argument types: [{}] != [{}]", + function->getName(), fmt::join(argument_type_names, ", "), fmt::join(function_argument_type_names, ", ")); + } +} + String DataTypeAggregateFunction::getFunctionName() const { return function->getName(); diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 52ed151107e..e3a4f9726d9 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -30,13 +30,7 @@ public: static constexpr bool is_parametric = true; DataTypeAggregateFunction(AggregateFunctionPtr function_, const DataTypes & argument_types_, - const Array & parameters_, std::optional version_ = std::nullopt) - : function(std::move(function_)) - , argument_types(argument_types_) - , parameters(parameters_) - , version(version_) - { - } + const Array & parameters_, std::optional version_ = std::nullopt); size_t getVersion() const; From 55fd2e04e331a58b83516fbefa6bad921fa842a3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Jul 2024 17:00:59 +0000 Subject: [PATCH 0826/1170] wip --- src/Storages/StorageBuffer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 04e6d6676d1..4ae9e029e1b 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -271,6 +271,8 @@ void StorageBuffer::read( } else { + if (processed_stage > QueryProcessingStage::FetchColumns) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot read from Buffer table with different structure in non-initial stage of query execution"); /// There is a struct mismatch and we need to convert read blocks from the destination table. const Block header = metadata_snapshot->getSampleBlock(); Names columns_intersection = column_names; From 12d917b74268ecb6a86b032d5c00c418c8a48f4e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Jul 2024 17:01:26 +0000 Subject: [PATCH 0827/1170] wip --- src/DataTypes/DataTypeAggregateFunction.cpp | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index ee42e4fea11..a4cd3b9e511 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -41,23 +41,6 @@ DataTypeAggregateFunction::DataTypeAggregateFunction(AggregateFunctionPtr functi , parameters(parameters_) , version(version_) { - Strings argument_type_names; - for (const auto & argument_type : argument_types) - argument_type_names.push_back(argument_type->getName()); - - Strings function_argument_type_names; - const auto & function_argument_types = function->getArgumentTypes(); - for (const auto & argument_type : function_argument_types) - function_argument_type_names.push_back(argument_type->getName()); - - size_t argument_types_size = std::max(argument_types.size(), function_argument_types.size()); - for (size_t i = 0; i < argument_types_size; ++i) - { - if (argument_types.size() != function_argument_types.size() || !argument_types[i]->equals(*function_argument_types[i])) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Data type AggregateFunction {} got argument types different from function argument types: [{}] != [{}]", - function->getName(), fmt::join(argument_type_names, ", "), fmt::join(function_argument_type_names, ", ")); - } } String DataTypeAggregateFunction::getFunctionName() const From 1329b5eb0b17d5499639bbb973aab7b17c95b644 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Jul 2024 16:22:54 +0000 Subject: [PATCH 0828/1170] Fix Buffer over Distributed --- src/Storages/StorageBuffer.cpp | 46 ++++++++++++-- src/Storages/StorageBuffer.h | 1 + ...r_over_distributed_type_mismatch.reference | 18 ++++++ ..._buffer_over_distributed_type_mismatch.sql | 60 +++++++++++++++++++ 4 files changed, 119 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.reference create mode 100644 tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4ae9e029e1b..f753d369d2d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1,3 +1,7 @@ +#include + +#include +#include #include #include #include @@ -23,7 +27,6 @@ #include #include #include -#include #include #include #include @@ -232,6 +235,12 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage( return QueryProcessingStage::FetchColumns; } +bool StorageBuffer::isRemote() const +{ + auto destination = getDestinationTable(); + return destination && destination->isRemote(); +} + void StorageBuffer::read( QueryPlan & query_plan, const Names & column_names, @@ -242,6 +251,29 @@ void StorageBuffer::read( size_t max_block_size, size_t num_streams) { + bool allow_experimental_analyzer = local_context->getSettingsRef().allow_experimental_analyzer; + + if (allow_experimental_analyzer && processed_stage > QueryProcessingStage::FetchColumns) + { + /** For query processing stages after FetchColumns, we do not allow using the same table more than once in the query. + * For example: SELECT * FROM buffer t1 JOIN buffer t2 USING (column) + * In that case, we will execute this query separately for the destination table and for the buffer, resulting in incorrect results. + */ + const auto & current_storage_id = getStorageID(); + auto table_nodes = extractAllTableReferences(query_info.query_tree); + size_t count_of_current_storage = 0; + for (const auto & node : table_nodes) + { + const auto & table_node = node->as(); + if (table_node.getStorageID().getFullNameNotQuoted() == current_storage_id.getFullNameNotQuoted()) + { + count_of_current_storage++; + if (count_of_current_storage > 1) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageBuffer over Distributed does not support using the same table more than once in the query"); + } + } + } + const auto & metadata_snapshot = storage_snapshot->metadata; if (auto destination = getDestinationTable()) @@ -271,8 +303,6 @@ void StorageBuffer::read( } else { - if (processed_stage > QueryProcessingStage::FetchColumns) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot read from Buffer table with different structure in non-initial stage of query execution"); /// There is a struct mismatch and we need to convert read blocks from the destination table. const Block header = metadata_snapshot->getSampleBlock(); Names columns_intersection = column_names; @@ -330,13 +360,17 @@ void StorageBuffer::read( } } + src_table_query_info.merge_storage_snapshot = storage_snapshot; destination->read( query_plan, columns_intersection, destination_snapshot, src_table_query_info, local_context, processed_stage, max_block_size, num_streams); - if (query_plan.isInitialized()) + if (query_plan.isInitialized() && processed_stage <= QueryProcessingStage::FetchColumns) { - + /** The code below converts columns from metadata_snapshot to columns from destination_metadata_snapshot. + * This conversion is not applicable for processed_stage > FetchColumns. + * Instead, we rely on the converting actions at the end of this function. + */ auto actions = addMissingDefaults( query_plan.getCurrentDataStream().header, header_after_adding_defaults.getNamesAndTypesList(), @@ -399,7 +433,7 @@ void StorageBuffer::read( /// TODO: Find a way to support projections for StorageBuffer if (processed_stage > QueryProcessingStage::FetchColumns) { - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (allow_experimental_analyzer) { auto storage = std::make_shared( getStorageID(), diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index cd6dd7b933f..02376f286b1 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -84,6 +84,7 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; + bool isRemote() const override; bool supportsParallelInsert() const override { return true; } diff --git a/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.reference b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.reference new file mode 100644 index 00000000000..1dc3acfeccb --- /dev/null +++ b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.reference @@ -0,0 +1,18 @@ +100 +100 +101 +101 +101 +102 +101 +101 +102 +100 +100 +101 +101 +101 +102 +101 +101 +102 diff --git a/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql new file mode 100644 index 00000000000..5a7c89074cf --- /dev/null +++ b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql @@ -0,0 +1,60 @@ + +DROP TABLE IF EXISTS realtimedrep; +CREATE TABLE realtimedrep (`amount` Int32) ENGINE = MergeTree() ORDER BY tuple(); +INSERT INTO realtimedrep FORMAT Values (100); + +DROP TABLE IF EXISTS realtimedistributed; +CREATE TABLE realtimedistributed (`amount` Int32) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), realtimedrep, rand()); + +DROP TABLE IF EXISTS realtimebuff__fuzz_19; +CREATE TABLE realtimebuff__fuzz_19 (`amount` UInt32) ENGINE = Buffer(currentDatabase(), 'realtimedistributed', 16, 3600, 36000, 10000, 1000000, 10000000, 100000000); +INSERT INTO realtimebuff__fuzz_19 FORMAT Values (101); + +DROP TABLE IF EXISTS realtimebuff__fuzz_20; +CREATE TABLE realtimebuff__fuzz_20 (`amount` Nullable(Int32)) ENGINE = Buffer(currentDatabase(), 'realtimedistributed', 16, 3600, 36000, 10000, 1000000, 10000000, 100000000); +INSERT INTO realtimebuff__fuzz_20 FORMAT Values (101); + +SELECT amount FROM realtimebuff__fuzz_19 t1 ORDER BY ALL; +SELECT amount + 1 FROM realtimebuff__fuzz_19 t1 ORDER BY ALL; +SELECT amount + 1 FROM realtimebuff__fuzz_20 t1 ORDER BY ALL; +SELECT sum(amount) = 100 FROM realtimebuff__fuzz_19 ORDER BY ALL; -- { serverError CANNOT_CONVERT_TYPE } +SELECT sum(amount) = 100 FROM realtimebuff__fuzz_20 ORDER BY ALL; -- { serverError CANNOT_CONVERT_TYPE } + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN (SELECT number :: UInt32 AS amount FROM numbers(3) ) t2 ON t1.amount = t2.amount +ORDER BY ALL +SETTINGS allow_experimental_analyzer = 0; -- { serverError UNKNOWN_IDENTIFIER } + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN (SELECT number :: UInt32 AS amount FROM numbers(3) ) t2 ON t1.amount = t2.amount +ORDER BY ALL +SETTINGS allow_experimental_analyzer = 1; + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN (SELECT number :: UInt32 AS amount FROM numbers(300) ) t2 ON t1.amount = t2.amount +ORDER BY ALL +SETTINGS allow_experimental_analyzer = 0; -- { serverError UNKNOWN_IDENTIFIER } + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN (SELECT number :: UInt32 AS amount FROM numbers(300) ) t2 ON t1.amount = t2.amount +ORDER BY ALL +SETTINGS allow_experimental_analyzer = 1; + +SELECT t2.amount + 1 FROM (SELECT number :: UInt32 AS amount FROM numbers(300) ) t1 +JOIN realtimebuff__fuzz_19 t2 USING (amount) +ORDER BY ALL +; + +SELECT t2.amount + 1 FROM (SELECT number :: UInt32 AS amount FROM numbers(300) ) t1 +JOIN realtimebuff__fuzz_19 t2 ON t1.amount = t2.amount +ORDER BY ALL +; + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN realtimebuff__fuzz_19 t2 ON t1.amount = t2.amount +; -- { serverError NOT_IMPLEMENTED,UNKNOWN_IDENTIFIER } + +SELECT amount FROM realtimebuff__fuzz_19 t1 +JOIN realtimebuff__fuzz_19 t2 ON t1.amount = t2.amount +JOIN realtimebuff__fuzz_19 t3 ON t1.amount = t3.amount +; -- { serverError NOT_IMPLEMENTED,AMBIGUOUS_COLUMN_NAME } From 65c0efb2d50dc4ac37750505cde1b8d26729b871 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 10:29:49 +0000 Subject: [PATCH 0829/1170] Revert "Merge pull request #66510 from canhld94/fix_trivial_count_non_deterministic_func" This reverts commit bf595ca374af503c087e2eb0f80f79490e5b8faa, reversing changes made to b6b1a7a7790fcce40d2de67c62998a228246e729. --- 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, 9 insertions(+), 30 deletions(-) delete mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference delete 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 ce27ad24e10..2286530aa83 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); if (!filter_dag) return {}; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 90c2c7f93c1..ba1f4488005 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -275,8 +275,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) + ActionsDAG::Nodes & additional_nodes) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -285,14 +284,8 @@ 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)) 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; @@ -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); !child) return nullptr; return &node_copy; @@ -332,7 +325,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, allow_non_deterministic_functions)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -366,13 +359,13 @@ 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) { 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); if (!res) return {}; @@ -381,7 +374,7 @@ std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); + auto dag = splitFilterDagForAllowedInputs(predicate, &block); if (dag) filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 73b7908b75c..919513b3b38 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -33,15 +33,7 @@ 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 -/// 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); /// 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 deleted file mode 100644 index 6ed281c757a..00000000000 --- a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference +++ /dev/null @@ -1,2 +0,0 @@ -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 deleted file mode 100644 index bb3269da597..00000000000 --- a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql +++ /dev/null @@ -1,4 +0,0 @@ -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 3802b1ed6c7174b0d95bf1c89d339187fe6dc69d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 12:42:57 +0200 Subject: [PATCH 0830/1170] Update comment --- src/Interpreters/Cache/FileCache.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 4c17afb79be..aff4e48d01d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -326,6 +326,8 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// ^ ^ /// right offset aligned_right_offset /// [_________] <-- last cached file segment, e.g. we have uncovered suffix of the requested range + /// ^ + /// last_file_segment_right_offset /// [________________] /// size /// [____________________________________] @@ -335,8 +337,9 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// and get something like this: /// /// [________________________] - /// ^ ^ - /// right_offset right_offset + max_file_segment_size + /// ^ ^ + /// | last_file_segment_right_offset + max_file_segment_size + /// last_file_segment_right_offset /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. /// Because its left offset would be bigger than right_offset. /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. From 6a0c0e7b1d4922b616d706820d9000ffe8040d63 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 12:48:21 +0200 Subject: [PATCH 0831/1170] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index aff4e48d01d..ed91e41db17 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -340,7 +340,7 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// ^ ^ /// | last_file_segment_right_offset + max_file_segment_size /// last_file_segment_right_offset - /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. + /// e.g. there is no need to create sub-segment for range (last_file_segment_right_offset + max_file_segment_size, aligned_right_offset]. /// Because its left offset would be bigger than right_offset. /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. From 67fe443133c277f78ecf1a11d542c61b1d805a59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 10:48:47 +0000 Subject: [PATCH 0832/1170] Fix build --- 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 2286530aa83..49888596fbb 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); if (!filter_dag) return {}; 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 0833/1170] 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 0834/1170] 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 67bddde6287f21a702d2e134921a9f00073959f6 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 3 Aug 2024 21:26:05 +0200 Subject: [PATCH 0835/1170] move Check Descriptions to commit_status_helper --- .yamllint | 6 - tests/ci/ci_config.py | 2 - tests/ci/ci_definitions.py | 187 +--------------------------- tests/ci/commit_status_helper.py | 203 +++++++++++++++++++++++++++++-- 4 files changed, 195 insertions(+), 203 deletions(-) diff --git a/.yamllint b/.yamllint index 7fb741ec9f4..b8f7c93e246 100644 --- a/.yamllint +++ b/.yamllint @@ -5,12 +5,6 @@ rules: indentation: level: warning indent-sequences: consistent - line-length: - # there are: - # - bash -c "", so this is OK - # - yaml in tests - max: 1000 - level: warning comments: min-spaces-from-content: 1 document-start: disable diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ef48466e451..8cb587a1062 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -20,12 +20,10 @@ class CI: from ci_definitions import BuildConfig as BuildConfig from ci_definitions import DigestConfig as DigestConfig from ci_definitions import JobConfig as JobConfig - from ci_definitions import CheckDescription as CheckDescription from ci_definitions import Tags as Tags from ci_definitions import JobNames as JobNames from ci_definitions import BuildNames as BuildNames from ci_definitions import StatusNames as StatusNames - from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS from ci_definitions import REQUIRED_CHECKS as REQUIRED_CHECKS from ci_definitions import SyncState as SyncState from ci_definitions import MQ_JOBS as MQ_JOBS diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 795bda3d4b0..48847b0d7a6 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -1,7 +1,7 @@ import copy from dataclasses import dataclass, field from pathlib import Path -from typing import Callable, List, Union, Iterable, Optional, Literal, Any +from typing import List, Union, Iterable, Optional, Literal, Any from ci_utils import WithIter from integration_test_images import IMAGES @@ -646,188 +646,3 @@ MQ_JOBS = [ BuildNames.BINARY_RELEASE, JobNames.UNIT_TEST, ] - - -@dataclass -class CheckDescription: - name: str - description: str # the check descriptions, will be put into the status table - match_func: Callable[[str], bool] # the function to check vs the commit status - - def __hash__(self) -> int: - return hash(self.name + self.description) - - -CHECK_DESCRIPTIONS = [ - CheckDescription( - StatusNames.PR_CHECK, - "Checks correctness of the PR's body", - lambda x: x == "PR Check", - ), - CheckDescription( - StatusNames.SYNC, - "If it fails, ask a maintainer for help", - lambda x: x == StatusNames.SYNC, - ), - CheckDescription( - "AST fuzzer", - "Runs randomly generated queries to catch program errors. " - "The build type is optionally given in parenthesis. " - "If it fails, ask a maintainer for help", - lambda x: x.startswith("AST fuzzer"), - ), - CheckDescription( - JobNames.BUGFIX_VALIDATE, - "Checks that either a new test (functional or integration) or there " - "some changed tests that fail with the binary built on master branch", - lambda x: x == JobNames.BUGFIX_VALIDATE, - ), - CheckDescription( - StatusNames.CI, - "A meta-check that indicates the running CI. Normally, it's in success or " - "pending state. The failed status indicates some problems with the PR", - lambda x: x == "CI running", - ), - CheckDescription( - "Builds", - "Builds ClickHouse in various configurations for use in further steps. " - "You have to fix the builds that fail. Build logs often has enough " - "information to fix the error, but you might have to reproduce the failure " - "locally. The cmake options can be found in the build log, grepping for " - 'cmake. Use these options and follow the
general build process', - lambda x: x.startswith("ClickHouse") and x.endswith("build check"), - ), - CheckDescription( - "Compatibility check", - "Checks that clickhouse binary runs on distributions with old libc " - "versions. If it fails, ask a maintainer for help", - lambda x: x.startswith("Compatibility check"), - ), - CheckDescription( - JobNames.DOCKER_SERVER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker server"), - ), - CheckDescription( - JobNames.DOCKER_KEEPER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker keeper"), - ), - CheckDescription( - JobNames.DOCS_CHECK, - "Builds and tests the documentation", - lambda x: x == JobNames.DOCS_CHECK, - ), - CheckDescription( - JobNames.FAST_TEST, - "Normally this is the first check that is ran for a PR. It builds ClickHouse " - 'and runs most of stateless functional tests, ' - "omitting some. If it fails, further checks are not started until it is fixed. " - "Look at the report to see which tests fail, then reproduce the failure " - 'locally as described here', - lambda x: x == JobNames.FAST_TEST, - ), - CheckDescription( - "Flaky tests", - "Checks if new added or modified tests are flaky by running them repeatedly, " - "in parallel, with more randomization. Functional tests are run 100 times " - "with address sanitizer, and additional randomization of thread scheduling. " - "Integration tests are run up to 10 times. If at least once a new test has " - "failed, or was too long, this check will be red. We don't allow flaky tests, " - 'read the doc', - lambda x: "tests flaky check" in x, - ), - CheckDescription( - "Install packages", - "Checks that the built packages are installable in a clear environment", - lambda x: x.startswith("Install packages ("), - ), - CheckDescription( - "Integration tests", - "The integration tests report. In parenthesis the package type is given, " - "and in square brackets are the optional part/total tests", - lambda x: x.startswith("Integration tests ("), - ), - CheckDescription( - StatusNames.MERGEABLE, - "Checks if all other necessary checks are successful", - lambda x: x == StatusNames.MERGEABLE, - ), - CheckDescription( - "Performance Comparison", - "Measure changes in query performance. The performance test report is " - 'described in detail here. ' - "In square brackets are the optional part/total tests", - lambda x: x.startswith("Performance Comparison"), - ), - CheckDescription( - "Push to Dockerhub", - "The check for building and pushing the CI related docker images to docker hub", - lambda x: x.startswith("Push") and "to Dockerhub" in x, - ), - CheckDescription( - "Sqllogic", - "Run clickhouse on the " - 'sqllogic ' - "test set against sqlite and checks that all statements are passed", - lambda x: x.startswith("Sqllogic test"), - ), - CheckDescription( - "SQLancer", - "Fuzzing tests that detect logical bugs with " - 'SQLancer tool', - lambda x: x.startswith("SQLancer"), - ), - CheckDescription( - "Stateful tests", - "Runs stateful functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateful tests ("), - ), - CheckDescription( - "Stateless tests", - "Runs stateless functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateless tests ("), - ), - CheckDescription( - "Stress test", - "Runs stateless functional tests concurrently from several clients to detect " - "concurrency-related errors", - lambda x: x.startswith("Stress test ("), - ), - CheckDescription( - JobNames.STYLE_CHECK, - "Runs a set of checks to keep the code style clean. If some of tests failed, " - "see the related log from the report", - lambda x: x == JobNames.STYLE_CHECK, - ), - CheckDescription( - "Unit tests", - "Runs the unit tests for different release types", - lambda x: x.startswith("Unit tests ("), - ), - CheckDescription( - "Upgrade check", - "Runs stress tests on server version from last release and then tries to " - "upgrade it to the version from the PR. It checks if the new server can " - "successfully startup without any errors, crashes or sanitizer asserts", - lambda x: x.startswith("Upgrade check ("), - ), - CheckDescription( - "ClickBench", - "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", - lambda x: x.startswith("ClickBench"), - ), - CheckDescription( - "Fallback for unknown", - "There's no description for the check yet, please add it to " - "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", - lambda x: True, - ), -] diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index fdc9c002b66..8967d453622 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -7,7 +7,7 @@ import time from collections import defaultdict from dataclasses import asdict, dataclass from pathlib import Path -from typing import Dict, List, Optional, Union +from typing import Dict, List, Optional, Union, Callable from github import Github from github.Commit import Commit @@ -176,7 +176,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: if not [status for status in statuses if status.context == CI.StatusNames.CI]: # This is the case, when some statuses already exist for the check, - # but not the StatusNames.CI. We should create it as pending. + # but not the CI.StatusNames.CI. We should create it as pending. # W/o pr_info to avoid recursion, and yes, one extra create_ci_report post_commit_status( commit, @@ -226,20 +226,20 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: f"\n" ) # group checks by the name to get the worst one per each - grouped_statuses = {} # type: Dict[CI.CheckDescription, CommitStatuses] + grouped_statuses = {} # type: Dict[CheckDescription, CommitStatuses] for status in statuses: cd = None - for c in CI.CHECK_DESCRIPTIONS: + for c in CHECK_DESCRIPTIONS: if c.match_func(status.context): cd = c break - if cd is None or cd == CI.CHECK_DESCRIPTIONS[-1]: + if cd is None or cd == CHECK_DESCRIPTIONS[-1]: # This is the case for either non-found description or a fallback - cd = CI.CheckDescription( + cd = CheckDescription( status.context, - CI.CHECK_DESCRIPTIONS[-1].description, - CI.CHECK_DESCRIPTIONS[-1].match_func, + CHECK_DESCRIPTIONS[-1].description, + CHECK_DESCRIPTIONS[-1].match_func, ) if cd in grouped_statuses: @@ -459,7 +459,7 @@ def trigger_mergeable_check( set_from_sync: bool = False, workflow_failed: bool = False, ) -> StatusType: - """calculate and update StatusNames.MERGEABLE""" + """calculate and update CI.StatusNames.MERGEABLE""" required_checks = [status for status in statuses if CI.is_required(status.context)] mergeable_status = None @@ -536,3 +536,188 @@ def update_upstream_sync_status( get_commit_filtered_statuses(last_synced_upstream_commit), set_from_sync=True, ) + + +@dataclass +class CheckDescription: + name: str + description: str # the check descriptions, will be put into the status table + match_func: Callable[[str], bool] # the function to check vs the commit status + + def __hash__(self) -> int: + return hash(self.name + self.description) + + +CHECK_DESCRIPTIONS = [ + CheckDescription( + CI.StatusNames.PR_CHECK, + "Checks correctness of the PR's body", + lambda x: x == "PR Check", + ), + CheckDescription( + CI.StatusNames.SYNC, + "If it fails, ask a maintainer for help", + lambda x: x == CI.StatusNames.SYNC, + ), + CheckDescription( + "AST fuzzer", + "Runs randomly generated queries to catch program errors. " + "The build type is optionally given in parenthesis. " + "If it fails, ask a maintainer for help", + lambda x: x.startswith("AST fuzzer"), + ), + CheckDescription( + CI.JobNames.BUGFIX_VALIDATE, + "Checks that either a new test (functional or integration) or there " + "some changed tests that fail with the binary built on master branch", + lambda x: x == CI.JobNames.BUGFIX_VALIDATE, + ), + CheckDescription( + CI.StatusNames.CI, + "A meta-check that indicates the running CI. Normally, it's in success or " + "pending state. The failed status indicates some problems with the PR", + lambda x: x == "CI running", + ), + CheckDescription( + "Builds", + "Builds ClickHouse in various configurations for use in further steps. " + "You have to fix the builds that fail. Build logs often has enough " + "information to fix the error, but you might have to reproduce the failure " + "locally. The cmake options can be found in the build log, grepping for " + 'cmake. Use these options and follow the general build process', + lambda x: x.startswith("ClickHouse") and x.endswith("build check"), + ), + CheckDescription( + "Compatibility check", + "Checks that clickhouse binary runs on distributions with old libc " + "versions. If it fails, ask a maintainer for help", + lambda x: x.startswith("Compatibility check"), + ), + CheckDescription( + CI.JobNames.DOCKER_SERVER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker server"), + ), + CheckDescription( + CI.JobNames.DOCKER_KEEPER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker keeper"), + ), + CheckDescription( + CI.JobNames.DOCS_CHECK, + "Builds and tests the documentation", + lambda x: x == CI.JobNames.DOCS_CHECK, + ), + CheckDescription( + CI.JobNames.FAST_TEST, + "Normally this is the first check that is ran for a PR. It builds ClickHouse " + 'and runs most of stateless functional tests, ' + "omitting some. If it fails, further checks are not started until it is fixed. " + "Look at the report to see which tests fail, then reproduce the failure " + 'locally as described here', + lambda x: x == CI.JobNames.FAST_TEST, + ), + CheckDescription( + "Flaky tests", + "Checks if new added or modified tests are flaky by running them repeatedly, " + "in parallel, with more randomization. Functional tests are run 100 times " + "with address sanitizer, and additional randomization of thread scheduling. " + "Integration tests are run up to 10 times. If at least once a new test has " + "failed, or was too long, this check will be red. We don't allow flaky tests, " + 'read the doc', + lambda x: "tests flaky check" in x, + ), + CheckDescription( + "Install packages", + "Checks that the built packages are installable in a clear environment", + lambda x: x.startswith("Install packages ("), + ), + CheckDescription( + "Integration tests", + "The integration tests report. In parenthesis the package type is given, " + "and in square brackets are the optional part/total tests", + lambda x: x.startswith("Integration tests ("), + ), + CheckDescription( + CI.StatusNames.MERGEABLE, + "Checks if all other necessary checks are successful", + lambda x: x == CI.StatusNames.MERGEABLE, + ), + CheckDescription( + "Performance Comparison", + "Measure changes in query performance. The performance test report is " + 'described in detail here. ' + "In square brackets are the optional part/total tests", + lambda x: x.startswith("Performance Comparison"), + ), + CheckDescription( + "Push to Dockerhub", + "The check for building and pushing the CI related docker images to docker hub", + lambda x: x.startswith("Push") and "to Dockerhub" in x, + ), + CheckDescription( + "Sqllogic", + "Run clickhouse on the " + 'sqllogic ' + "test set against sqlite and checks that all statements are passed", + lambda x: x.startswith("Sqllogic test"), + ), + CheckDescription( + "SQLancer", + "Fuzzing tests that detect logical bugs with " + 'SQLancer tool', + lambda x: x.startswith("SQLancer"), + ), + CheckDescription( + "Stateful tests", + "Runs stateful functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateful tests ("), + ), + CheckDescription( + "Stateless tests", + "Runs stateless functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateless tests ("), + ), + CheckDescription( + "Stress test", + "Runs stateless functional tests concurrently from several clients to detect " + "concurrency-related errors", + lambda x: x.startswith("Stress test ("), + ), + CheckDescription( + CI.JobNames.STYLE_CHECK, + "Runs a set of checks to keep the code style clean. If some of tests failed, " + "see the related log from the report", + lambda x: x == CI.JobNames.STYLE_CHECK, + ), + CheckDescription( + "Unit tests", + "Runs the unit tests for different release types", + lambda x: x.startswith("Unit tests ("), + ), + CheckDescription( + "Upgrade check", + "Runs stress tests on server version from last release and then tries to " + "upgrade it to the version from the PR. It checks if the new server can " + "successfully startup without any errors, crashes or sanitizer asserts", + lambda x: x.startswith("Upgrade check ("), + ), + CheckDescription( + "ClickBench", + "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", + lambda x: x.startswith("ClickBench"), + ), + CheckDescription( + "Fallback for unknown", + "There's no description for the check yet, please add it to " + "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", + lambda x: True, + ), +] From 2d92cd71a83ed07e6f22c2ba37d39926837a9df0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 5 Aug 2024 09:36:09 +0000 Subject: [PATCH 0836/1170] add some comments --- src/Interpreters/inplaceBlockConversions.cpp | 9 +++++++-- src/Storages/MergeTree/IMergeTreeReader.cpp | 1 + 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 429d467ffbf..68254768a7d 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -309,6 +309,7 @@ static bool hasDefault(const StorageMetadataPtr & metadata_snapshot, const NameA static String removeTupleElementsFromSubcolumn(String subcolumn_name, const Names & tuple_elements) { + /// Add a dot to the end of name for convenience. subcolumn_name += "."; for (const auto & elem : tuple_elements) { @@ -396,19 +397,23 @@ void fillMissingColumns( Names tuple_elements; auto serialization = IDataType::getSerialization(*requested_column); + /// For Nested columns collect names of tuple elements and skip them while getting the base type of array. IDataType::forEachSubcolumn([&](const auto & path, const auto &, const auto &) { if (path.back().type == ISerialization::Substream::TupleElement) tuple_elements.push_back(path.back().name_of_substream); }, ISerialization::SubstreamData(serialization)); + /// The number of dimensions that belongs to the array itself but not shared in Nested column. + /// For example for column "n Nested(a UInt64, b Array(UInt64))" this value is 0 for `n.a` and 1 for `n.b`. size_t num_empty_dimensions = num_dimensions - current_offsets.size(); + auto base_type = getBaseTypeOfArray(requested_column->getTypeInStorage(), tuple_elements); auto scalar_type = createArrayOfType(base_type, num_empty_dimensions); - size_t data_size = assert_cast(*current_offsets.back()).getData().back(); - auto subcolumn_name = removeTupleElementsFromSubcolumn(requested_column->getSubcolumnName(), tuple_elements); + /// Remove names of tuple elements because they are already processed by 'getBaseTypeOfArray'. + auto subcolumn_name = removeTupleElementsFromSubcolumn(requested_column->getSubcolumnName(), tuple_elements); res_columns[i] = createColumnWithDefaultValue(*scalar_type, subcolumn_name, data_size); for (auto it = current_offsets.rbegin(); it != current_offsets.rend(); ++it) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 05a0b5a7dbc..e0b2710c61f 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -150,6 +150,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns NamesAndTypesList full_requested_columns; /// Convert columns list to block. And convert subcolumns to full columns. + /// Defaults should be executed on full columns to get correct values for subcolumns. /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. auto it = original_requested_columns.begin(); From 3244002cae58da99c8b088888376c0fda7f3f1f4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 13:22:10 +0200 Subject: [PATCH 0837/1170] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ed91e41db17..1a15efa7cf8 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -560,7 +560,7 @@ FileCache::getOrSet( FileSegment::Range initial_range(offset, offset + size - 1); /// result_range is initial range, which will be adjusted according to - /// 1. aligned offset, alighed_end_offset + /// 1. aligned_offset, aligned_end_offset /// 2. max_file_segments_limit FileSegment::Range result_range = initial_range; From a296beb39084492a6825879ff5258f4377ead75b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 5 Aug 2024 13:31:53 +0200 Subject: [PATCH 0838/1170] Unit test: Mark as FAILURE if retcode != 0 --- tests/ci/unit_tests_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 6430fa78801..9cc8ec379bf 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -197,6 +197,11 @@ def main(): subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {TEMP_PATH}", shell=True) state, description, test_results = process_results(test_output) + if retcode != 0 and state == SUCCESS: + # The process might have failed without reporting it in the test_output (e.g. LeakSanitizer) + state = FAILURE + description = "Invalid return code. Check run.log" + additional_files = [run_log_path] + [ p for p in test_output.iterdir() if not p.is_dir() ] From 07cfcdeaaec0289d78b033260dd657bebad70674 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 5 Aug 2024 11:31:46 +0000 Subject: [PATCH 0839/1170] Fix filter pushdown for aggregation without keys --- .../Optimizations/filterPushDown.cpp | 9 +++++++ .../03217_fliter_pushdown_no_keys.reference | 6 +++++ .../03217_fliter_pushdown_no_keys.sql | 26 +++++++++++++++++++ 3 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03217_fliter_pushdown_no_keys.reference create mode 100644 tests/queries/0_stateless/03217_fliter_pushdown_no_keys.sql diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 73314f005b6..b71326ff75b 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -442,6 +442,15 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes const auto & params = aggregating->getParams(); const auto & keys = params.keys; + /** The filter is applied either to aggregation keys or aggregation result + * (columns under aggregation is not available in outer scope, so we can't have a filter for them). + * The filter for the aggregation result is not pushed down, so the only valid case is filtering aggregation keys. + * In case keys are empty, do not push down the filter. + * Also with empty keys we can have an issue with `empty_result_for_aggregation_by_empty_set`, + * since we can gen a result row when everything is filtered. + */ + if (keys.empty()) + return 0; const bool filter_column_is_not_among_aggregation_keys = std::find(keys.begin(), keys.end(), filter->getFilterColumnName()) == keys.end(); diff --git a/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.reference b/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.reference new file mode 100644 index 00000000000..9838dd1b936 --- /dev/null +++ b/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.reference @@ -0,0 +1,6 @@ +--- +1 1 +--- +3 3 +--- +--- diff --git a/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.sql b/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.sql new file mode 100644 index 00000000000..cb8bf59e790 --- /dev/null +++ b/tests/queries/0_stateless/03217_fliter_pushdown_no_keys.sql @@ -0,0 +1,26 @@ + + + +select * from ( select sum(last_seen) as dates_seen, materialize(1) as last_seen ) where last_seen > 2; +select * from ( select sum(last_seen) as dates_seen, materialize(2) as last_seen ) where last_seen < 2; +select * from ( select sum(last_seen) as dates_seen, materialize(2) as last_seen GROUP BY 'a' ) where last_seen < 2; + +select '---'; +select * from ( select sum(last_seen) as dates_seen, 1 as last_seen UNION ALL select sum(last_seen) as dates_seen, 3 as last_seen ) where last_seen < 2; + +select '---'; +select * from ( select sum(last_seen) as dates_seen, 1 as last_seen UNION ALL select sum(last_seen) as dates_seen, 3 as last_seen ) where last_seen > 2; + +select '---'; +with activity as ( + select + groupUniqArrayState(toDate('2025-01-01 01:00:00')) as dates_seen, + toDateTime('2025-01-01 01:00:00') as last_seen + union all + select + groupUniqArrayState(toDate('2023-11-11 11:11:11')) as dates_seen, + toDateTime('2023-11-11 11:11:11') as last_seen +) +select last_seen from activity +where last_seen < toDateTime('2020-01-01 00:00:00'); +select '---'; From 23190c30cf696075017f09a4997a7969c1d2f651 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 14:32:08 +0200 Subject: [PATCH 0840/1170] Fix bad test `03032_redundant_equals` --- tests/queries/0_stateless/03032_redundant_equals.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03032_redundant_equals.sql b/tests/queries/0_stateless/03032_redundant_equals.sql index de85ec5cf00..eee2be4ebf0 100644 --- a/tests/queries/0_stateless/03032_redundant_equals.sql +++ b/tests/queries/0_stateless/03032_redundant_equals.sql @@ -5,9 +5,9 @@ CREATE TABLE test_table k UInt64, ) ENGINE = MergeTree -ORDER BY k; +ORDER BY k SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO test_table SELECT number FROM numbers(10000000); +INSERT INTO test_table SELECT number FROM numbers(100000); SET allow_experimental_analyzer = 1; @@ -25,7 +25,7 @@ SELECT * FROM test_table WHERE (NOT ((k not in (100) = 0) OR (k in (100) = 1))) SELECT * FROM test_table WHERE (NOT ((k in (101) = 0) OR (k in (100) = 1))) = 1; SELECT * FROM test_table WHERE ((k not in (101) = 0) OR (k in (100) = 1)) = 1; SELECT * FROM test_table WHERE ((k not in (99) = 1) AND (k in (100) = 1)) = 1; --- we skip optimizing queries with toNullable(0 or 1) but lets make sure they still work +-- we skip optimizing queries with toNullable(0 or 1) but lets make sure they still work SELECT * FROM test_table WHERE (k = 101) = toLowCardinality(toNullable(1)); SELECT * FROM test_table WHERE (k = 101) = toNullable(1); SELECT * FROM test_table WHERE (k = 101) = toLowCardinality(1); From ff0b8889ab8bf347343e18b4e9a407b3cab8264b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 14:36:24 +0200 Subject: [PATCH 0841/1170] Update 02789_reading_from_s3_with_connection_pool.sh --- .../0_stateless/02789_reading_from_s3_with_connection_pool.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh index 39399842db1..5a37d51233d 100755 --- a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh +++ b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-random-settings, no-replicated-database +# Tags: no-fasttest, no-random-settings, no-replicated-database, no-distributed-cache CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c21b97672c86a18b693c8c60335271248c279fdd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Aug 2024 14:36:54 +0200 Subject: [PATCH 0842/1170] Fix trailing whitespace --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1a15efa7cf8..217ae614c22 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -337,7 +337,7 @@ std::vector FileCache::splitRange(size_t offset, size_t size /// and get something like this: /// /// [________________________] - /// ^ ^ + /// ^ ^ /// | last_file_segment_right_offset + max_file_segment_size /// last_file_segment_right_offset /// e.g. there is no need to create sub-segment for range (last_file_segment_right_offset + max_file_segment_size, aligned_right_offset]. 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 0843/1170] 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 8393f7f54737c0ff5c0845e9481784b5d3764bfc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 2 Aug 2024 16:26:14 +0100 Subject: [PATCH 0844/1170] impl --- .../0_stateless/02313_filesystem_cache_seeks.sh | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh b/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh index b54e3d7f805..fc91f3f1448 100755 --- a/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh +++ b/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh @@ -8,13 +8,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh +client_opts=( + --distributed_ddl_output_mode 'null_status_on_timeout' +) + for STORAGE_POLICY in 's3_cache' 'local_cache' 's3_cache_multi' 'azure_cache'; do echo "Using storage policy: $STORAGE_POLICY" $CLICKHOUSE_CLIENT --query "SYSTEM DROP FILESYSTEM CACHE" - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_02313" + $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE IF EXISTS test_02313" - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_02313 (id Int32, val String) + $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "CREATE TABLE test_02313 (id Int32, val String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS storage_policy = '$STORAGE_POLICY'" @@ -32,6 +36,6 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 's3_cache_multi' 'azure_cache'; d $CLICKHOUSE_CLIENT --query "SELECT * FROM test_02313 WHERE val LIKE concat('%', randomPrintableASCII(3), '%') FORMAT Null" $CLICKHOUSE_CLIENT --query "SELECT * FROM test_02313 WHERE val LIKE concat('%', randomPrintableASCII(3), '%') FORMAT Null" - $CLICKHOUSE_CLIENT --query "DROP TABLE test_02313" + $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE test_02313" done 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 0845/1170] 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: Mon, 5 Aug 2024 15:38:15 +0200 Subject: [PATCH 0846/1170] Update tags --- tests/queries/0_stateless/02995_index_1.sh | 4 ++-- tests/queries/0_stateless/02995_index_10.sh | 4 ++-- tests/queries/0_stateless/02995_index_5.sh | 4 ++-- tests/queries/0_stateless/02995_index_6.sh | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02995_index_1.sh b/tests/queries/0_stateless/02995_index_1.sh index 128697fd0fe..76be6341d53 100755 --- a/tests/queries/0_stateless/02995_index_1.sh +++ b/tests/queries/0_stateless/02995_index_1.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} +done | ${CLICKHOUSE_CLIENT} ${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_10.sh b/tests/queries/0_stateless/02995_index_10.sh index c15ba00fd05..813cc49cbd8 100755 --- a/tests/queries/0_stateless/02995_index_10.sh +++ b/tests/queries/0_stateless/02995_index_10.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} +done | ${CLICKHOUSE_CLIENT} ${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_5.sh b/tests/queries/0_stateless/02995_index_5.sh index 80f75a532e3..58e53a2c481 100755 --- a/tests/queries/0_stateless/02995_index_5.sh +++ b/tests/queries/0_stateless/02995_index_5.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -39,6 +39,6 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String HAVING count() > 0; " -done | ${CLICKHOUSE_CLIENT} +done | ${CLICKHOUSE_CLIENT} ${CLICKHOUSE_CLIENT} "DROP TABLE test" diff --git a/tests/queries/0_stateless/02995_index_6.sh b/tests/queries/0_stateless/02995_index_6.sh index e90387c7c0c..ef35f0e1126 100755 --- a/tests/queries/0_stateless/02995_index_6.sh +++ b/tests/queries/0_stateless/02995_index_6.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 41dfec0fe76a14bde54711de1cd8781b5d00a83e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Aug 2024 14:58:40 +0100 Subject: [PATCH 0847/1170] fix test --- tests/queries/0_stateless/02313_filesystem_cache_seeks.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh b/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh index fc91f3f1448..b7adde6fcbb 100755 --- a/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh +++ b/tests/queries/0_stateless/02313_filesystem_cache_seeks.sh @@ -16,12 +16,12 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 's3_cache_multi' 'azure_cache'; d echo "Using storage policy: $STORAGE_POLICY" $CLICKHOUSE_CLIENT --query "SYSTEM DROP FILESYSTEM CACHE" - $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE IF EXISTS test_02313" + $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE IF EXISTS test_02313" > /dev/null $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "CREATE TABLE test_02313 (id Int32, val String) ENGINE = MergeTree() ORDER BY tuple() - SETTINGS storage_policy = '$STORAGE_POLICY'" + SETTINGS storage_policy = '$STORAGE_POLICY'" > /dev/null $CLICKHOUSE_CLIENT --enable_filesystem_cache_on_write_operations=0 -n --query "INSERT INTO test_02313 SELECT * FROM @@ -36,6 +36,6 @@ for STORAGE_POLICY in 's3_cache' 'local_cache' 's3_cache_multi' 'azure_cache'; d $CLICKHOUSE_CLIENT --query "SELECT * FROM test_02313 WHERE val LIKE concat('%', randomPrintableASCII(3), '%') FORMAT Null" $CLICKHOUSE_CLIENT --query "SELECT * FROM test_02313 WHERE val LIKE concat('%', randomPrintableASCII(3), '%') FORMAT Null" - $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE test_02313" + $CLICKHOUSE_CLIENT "${client_opts[@]}" --query "DROP TABLE test_02313" > /dev/null done From 462fe8b7ffa98743ecc01c98eb0bfae840b05dca Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 18:23:56 +0200 Subject: [PATCH 0848/1170] 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 0849/1170] 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 e9506202d699c24e9b7a3e5b643bc9871680e5dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Aug 2024 16:40:54 +0200 Subject: [PATCH 0850/1170] Add debug logging --- src/Storages/WindowView/StorageWindowView.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 65bf6768b1b..bf934ed00d9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1055,6 +1055,8 @@ void StorageWindowView::threadFuncFireProc() /// TODO: consider using time_t instead (for every timestamp in this class) UInt32 timestamp_now = now(); + LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}", timestamp_now, next_fire_signal, max_watermark); + while (next_fire_signal <= timestamp_now) { try @@ -1072,6 +1074,9 @@ void StorageWindowView::threadFuncFireProc() if (slide_kind > IntervalKind::Kind::Day) slide_interval *= 86400; next_fire_signal += slide_interval; + + LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}", + timestamp_now, next_fire_signal, max_watermark, max_fired_watermark, slide_interval); } if (max_watermark >= timestamp_now) @@ -1433,16 +1438,19 @@ void StorageWindowView::writeIntoWindowView( while (window_view.modifying_query) std::this_thread::sleep_for(std::chrono::milliseconds(100)); - if (!window_view.is_proctime && window_view.max_watermark == 0 && block.rows() > 0) + const size_t block_rows = block.rows(); + if (!window_view.is_proctime && window_view.max_watermark == 0 && block_rows > 0) { std::lock_guard lock(window_view.fire_signal_mutex); const auto & window_column = block.getByName(window_view.timestamp_column_name); const ColumnUInt32::Container & window_end_data = static_cast(*window_column.column).getData(); UInt32 first_record_timestamp = window_end_data[0]; window_view.max_watermark = window_view.getWindowUpperBound(first_record_timestamp); + + LOG_TRACE(window_view.log, "New max watermark: {}", window_view.max_watermark); } - Pipe pipe(std::make_shared(block)); + Pipe pipe(std::make_shared(std::move(block))); UInt32 lateness_bound = 0; UInt32 t_max_watermark = 0; @@ -1649,6 +1657,8 @@ void StorageWindowView::writeIntoWindowView( auto executor = builder.execute(); executor->execute(builder.getNumThreads(), local_context->getSettingsRef().use_concurrency_control); + + LOG_TRACE(window_view.log, "Wrote {} rows into inner table ({})", block_rows, inner_table->getStorageID().getFullTableName()); } void StorageWindowView::startup() 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 0851/1170] 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 0852/1170] 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 0853/1170] 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 b531f6b78c080605f1ab91cb45c4a11b4b6aafb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Aug 2024 15:00:38 +0000 Subject: [PATCH 0854/1170] Fixing tests. --- src/Interpreters/ActionsDAG.cpp | 91 ++++++++++++++++++++------------- 1 file changed, 55 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 7bf65ee2416..df1c0aa1f2a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -567,49 +567,68 @@ void ActionsDAG::removeUnusedActions(const std::unordered_set & us std::stack stack; - for (const auto * root : roots) + enum class VisitStage { NonDeterministic, Required }; + + for (auto stage : {VisitStage::NonDeterministic, VisitStage::Required}) { - if (!required_nodes.contains(root)) - { - required_nodes.insert(root); - stack.push({.node = root}); - } + required_nodes.clear(); - while (!stack.empty()) + for (const auto * root : roots) { - auto & frame = stack.top(); - auto * node = const_cast(frame.node); - - while (frame.next_child_to_visit < node->children.size()) + if (!required_nodes.contains(root)) { - const auto * child = node->children[frame.next_child_to_visit]; - ++frame.next_child_to_visit; - - if (!required_nodes.contains(child)) - { - required_nodes.insert(child); - stack.push({.node = child}); - break; - } - - if (non_deterministic_nodes.contains(child)) - non_deterministic_nodes.insert(node); + required_nodes.insert(root); + stack.push({.node = root}); } - if (stack.top().node != node) - continue; - - if (!node->isDeterministic()) - non_deterministic_nodes.insert(node); - - stack.pop(); - - /// Constant folding. - if (allow_constant_folding && !node->children.empty() - && node->column && isColumnConst(*node->column) && !non_deterministic_nodes.contains(node)) + while (!stack.empty()) { - node->type = ActionsDAG::ActionType::COLUMN; - node->children.clear(); + auto & frame = stack.top(); + auto * node = const_cast(frame.node); + + while (frame.next_child_to_visit < node->children.size()) + { + const auto * child = node->children[frame.next_child_to_visit]; + ++frame.next_child_to_visit; + + if (!required_nodes.contains(child)) + { + required_nodes.insert(child); + stack.push({.node = child}); + break; + } + } + + if (stack.top().node != node) + continue; + + stack.pop(); + + if (stage == VisitStage::Required) + continue; + + if (!node->isDeterministic()) + non_deterministic_nodes.insert(node); + else + { + for (const auto * child : node->children) + { + if (non_deterministic_nodes.contains(child)) + { + non_deterministic_nodes.insert(node); + break; + } + } + } + + /// Constant folding. + if (allow_constant_folding && !node->children.empty() + && node->column && isColumnConst(*node->column)) + { + node->type = ActionsDAG::ActionType::COLUMN; + node->children.clear(); + node->is_deterministic_constant = !non_deterministic_nodes.contains(node); + } } } } From 59bffda9328a9d78c6a4422431f1014a16dd9cc2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 5 Aug 2024 17:06:28 +0200 Subject: [PATCH 0855/1170] Revert "Revert "Slightly better calculation of primary index"" --- .../MergeTree/IMergeTreeDataPartWriter.cpp | 19 +++++- .../MergeTreeDataPartWriterOnDisk.cpp | 65 ++++++++++--------- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 9 +-- .../02993_lazy_index_loading.reference | 2 +- ..._system_unload_primary_key_table.reference | 8 +-- .../03128_system_unload_primary_key.reference | 4 +- 6 files changed, 62 insertions(+), 45 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 6152da78395..c87f66b64f3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -71,9 +72,21 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( Columns IMergeTreeDataPartWriter::releaseIndexColumns() { - return Columns( - std::make_move_iterator(index_columns.begin()), - std::make_move_iterator(index_columns.end())); + /// The memory for index was allocated without thread memory tracker. + /// We need to deallocate it in shrinkToFit without memory tracker as well. + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + + Columns result; + result.reserve(index_columns.size()); + + for (auto & column : index_columns) + { + column->shrinkToFit(); + result.push_back(std::move(column)); + } + + index_columns.clear(); + return result; } SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 46dd766139a..6dc7e649b06 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -255,6 +255,12 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() index_compressor_stream = std::make_unique(*index_file_hashing_stream, primary_key_compression_codec, settings.primary_key_compress_block_size); index_source_hashing_stream = std::make_unique(*index_compressor_stream); } + + const auto & primary_key_types = metadata_snapshot->getPrimaryKey().data_types; + index_serializations.reserve(primary_key_types.size()); + + for (const auto & type : primary_key_types) + index_serializations.push_back(type->getDefaultSerialization()); } } @@ -300,22 +306,33 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() store = std::make_shared(stream_name, data_part_storage, data_part_storage, storage_settings->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; } + skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings)); skip_index_accumulated_marks.push_back(0); } } +void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndexRow(const Block & index_block, size_t row) +{ + chassert(index_block.columns() == index_serializations.size()); + auto & index_stream = compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream; + + for (size_t i = 0; i < index_block.columns(); ++i) + { + const auto & column = index_block.getByPosition(i).column; + + index_columns[i]->insertFrom(*column, row); + index_serializations[i]->serializeBinary(*column, row, index_stream, {}); + } +} + void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Block & primary_index_block, const Granules & granules_to_write) { - size_t primary_columns_num = primary_index_block.columns(); + if (!metadata_snapshot->hasPrimaryKey()) + return; + if (index_columns.empty()) - { - index_types = primary_index_block.getDataTypes(); - index_columns.resize(primary_columns_num); - last_block_index_columns.resize(primary_columns_num); - for (size_t i = 0; i < primary_columns_num; ++i) - index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty(); - } + index_columns = primary_index_block.cloneEmptyColumns(); { /** While filling index (index_columns), disable memory tracker. @@ -329,22 +346,14 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc /// Write index. The index contains Primary Key value for each `index_granularity` row. for (const auto & granule : granules_to_write) { - if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start) - { - for (size_t j = 0; j < primary_columns_num; ++j) - { - const auto & primary_column = primary_index_block.getByPosition(j); - index_columns[j]->insertFrom(*primary_column.column, granule.start_row); - primary_column.type->getDefaultSerialization()->serializeBinary( - *primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); - } - } + if (granule.mark_on_start) + calculateAndSerializePrimaryIndexRow(primary_index_block, granule.start_row); } } - /// store last index row to write final mark at the end of column - for (size_t j = 0; j < primary_columns_num; ++j) - last_block_index_columns[j] = primary_index_block.getByPosition(j).column; + /// Store block with last index row to write final mark at the end of column + if (with_final_mark) + last_index_block = primary_index_block; } void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block & block) @@ -421,17 +430,11 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat if (index_file_hashing_stream) { - if (write_final_mark) + if (write_final_mark && last_index_block) { - for (size_t j = 0; j < index_columns.size(); ++j) - { - const auto & column = *last_block_index_columns[j]; - size_t last_row_number = column.size() - 1; - index_columns[j]->insertFrom(column, last_row_number); - index_types[j]->getDefaultSerialization()->serializeBinary( - column, last_row_number, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); - } - last_block_index_columns.clear(); + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + calculateAndSerializePrimaryIndexRow(last_index_block, last_index_block.rows() - 1); + last_index_block.clear(); } if (compress_primary_key) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index bdf0fdb7f32..8d84442981e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -173,10 +173,10 @@ protected: std::unique_ptr index_source_hashing_stream; bool compress_primary_key; - DataTypes index_types; - /// Index columns from the last block - /// It's written to index file in the `writeSuffixAndFinalizePart` method - Columns last_block_index_columns; + /// Last block with index columns. + /// It's written to index file in the `writeSuffixAndFinalizePart` method. + Block last_index_block; + Serializations index_serializations; bool data_written = false; @@ -193,6 +193,7 @@ private: void initStatistics(); virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) = 0; + void calculateAndSerializePrimaryIndexRow(const Block & index_block, size_t row); struct ExecutionStatistics { diff --git a/tests/queries/0_stateless/02993_lazy_index_loading.reference b/tests/queries/0_stateless/02993_lazy_index_loading.reference index 5bc329ae4eb..08f07a92815 100644 --- a/tests/queries/0_stateless/02993_lazy_index_loading.reference +++ b/tests/queries/0_stateless/02993_lazy_index_loading.reference @@ -1,4 +1,4 @@ -100000000 140000000 +100000000 100000000 0 0 1 100000000 100000000 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference index 3ac6127fb21..2d33f7f6683 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference +++ b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference @@ -1,8 +1,8 @@ -100000000 140000000 -100000000 140000000 -100000000 140000000 +100000000 100000000 +100000000 100000000 +100000000 100000000 0 0 -100000000 140000000 +100000000 100000000 0 0 0 0 1 diff --git a/tests/queries/0_stateless/03128_system_unload_primary_key.reference b/tests/queries/0_stateless/03128_system_unload_primary_key.reference index c7b40ae5b06..2646dc7247f 100644 --- a/tests/queries/0_stateless/03128_system_unload_primary_key.reference +++ b/tests/queries/0_stateless/03128_system_unload_primary_key.reference @@ -1,4 +1,4 @@ -100000000 140000000 -100000000 140000000 +100000000 100000000 +100000000 100000000 0 0 0 0 From 5448bf7b86a27f7061e8fe2a02ef5b64ea4b2cc2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 5 Aug 2024 15:07:55 +0000 Subject: [PATCH 0856/1170] fix memory leak during index calculation --- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 9 +++++---- .../03217_primary_index_memory_leak.reference | 1 + .../03217_primary_index_memory_leak.sql | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03217_primary_index_memory_leak.reference create mode 100644 tests/queries/0_stateless/03217_primary_index_memory_leak.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 6dc7e649b06..b0e70e94b73 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -331,9 +331,6 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc if (!metadata_snapshot->hasPrimaryKey()) return; - if (index_columns.empty()) - index_columns = primary_index_block.cloneEmptyColumns(); - { /** While filling index (index_columns), disable memory tracker. * Because memory is allocated here (maybe in context of INSERT query), @@ -343,6 +340,9 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc */ MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + if (index_columns.empty()) + index_columns = primary_index_block.cloneEmptyColumns(); + /// Write index. The index contains Primary Key value for each `index_granularity` row. for (const auto & granule : granules_to_write) { @@ -434,9 +434,10 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat { MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; calculateAndSerializePrimaryIndexRow(last_index_block, last_index_block.rows() - 1); - last_index_block.clear(); } + last_index_block.clear(); + if (compress_primary_key) { index_source_hashing_stream->finalize(); diff --git a/tests/queries/0_stateless/03217_primary_index_memory_leak.reference b/tests/queries/0_stateless/03217_primary_index_memory_leak.reference new file mode 100644 index 00000000000..4913dd5e690 --- /dev/null +++ b/tests/queries/0_stateless/03217_primary_index_memory_leak.reference @@ -0,0 +1 @@ +150000 diff --git a/tests/queries/0_stateless/03217_primary_index_memory_leak.sql b/tests/queries/0_stateless/03217_primary_index_memory_leak.sql new file mode 100644 index 00000000000..d5a553c7d72 --- /dev/null +++ b/tests/queries/0_stateless/03217_primary_index_memory_leak.sql @@ -0,0 +1,15 @@ +-- Tags: no-debug, no-tsan, no-msan, no-asan, no-random-settings, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_primary_index_memory; +CREATE TABLE t_primary_index_memory (s String) ENGINE = MergeTree +ORDER BY s SETTINGS index_granularity = 1; + +INSERT INTO t_primary_index_memory SELECT repeat('a', 10000) FROM numbers(150000) +SETTINGS + max_block_size = 32, + max_memory_usage = '100M', + max_insert_block_size = 1024, + min_insert_block_size_rows = 1024; + +SELECT count() FROM t_primary_index_memory; +DROP TABLE t_primary_index_memory; From 146b8afce61ca2db2cdfd0621b0378746e90de59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 5 Aug 2024 17:20:41 +0200 Subject: [PATCH 0857/1170] Disable bad tests --- .../Nodes/tests/gtest_throttler_constraint.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 6cfccb252fa..363e286c91c 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -10,7 +10,9 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) +/// Test disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -40,7 +42,7 @@ TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) t.consumed("A", 10); } -TEST(SchedulerThrottlerConstraint, Unlimited) +TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -57,7 +59,7 @@ TEST(SchedulerThrottlerConstraint, Unlimited) } } -TEST(SchedulerThrottlerConstraint, Pacing) +TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -77,7 +79,7 @@ TEST(SchedulerThrottlerConstraint, Pacing) } } -TEST(SchedulerThrottlerConstraint, BucketFilling) +TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -111,7 +113,7 @@ TEST(SchedulerThrottlerConstraint, BucketFilling) t.consumed("A", 3); } -TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) +TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -139,7 +141,7 @@ TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) } } -TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness) +TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); From 4e9c3baa25cc7f02a8ba9ad5be6d68964b6a04f5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Jul 2024 12:49:26 +0000 Subject: [PATCH 0858/1170] Move analyzer to Beta stage --- .../test/fuzzer/query-fuzzer-tweaks-users.xml | 4 +- docker/test/stateless/stress_tests.lib | 4 +- docs/en/development/architecture.md | 2 +- docs/en/operations/analyzer.md | 8 +- docs/en/operations/settings/settings.md | 2 +- docs/ru/development/architecture.md | 2 +- src/Client/HedgedConnections.cpp | 4 +- src/Client/MultiplexedConnections.cpp | 4 +- src/Core/Settings.h | 5 +- src/Core/SettingsChangesHistory.cpp | 259 ++++++++++++++++++ src/Interpreters/ActionsVisitor.cpp | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 4 +- .../ClusterProxy/executeQuery.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 10 +- src/Interpreters/InterpreterFactory.cpp | 6 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Interpreters/executeQuery.cpp | 6 +- .../getHeaderForProcessingStage.cpp | 2 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 4 +- src/Server/TCPHandler.cpp | 6 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/IStorageCluster.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 18 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 6 +- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageMerge.cpp | 20 +- src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageView.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +- src/TableFunctions/TableFunctionView.cpp | 2 +- .../TableFunctionViewIfPermitted.cpp | 2 +- tests/config/users.d/analyzer.xml | 2 +- .../helpers/0_common_enable_old_analyzer.xml | 2 +- tests/integration/helpers/cluster.py | 2 +- .../test_analyzer_compatibility/test.py | 8 +- .../test_distributed_type_object/test.py | 2 +- .../configs/enable_parallel_replicas.xml | 2 +- .../integration/test_settings_profile/test.py | 6 +- .../performance/storage_join_direct_join.xml | 4 +- tests/performance/uniq_to_count.xml | 4 +- .../queries/0_stateless/00116_storage_set.sql | 4 +- .../queries/0_stateless/00202_cross_join.sql | 3 +- ..._shard_no_aggregates_and_constant_keys.sql | 2 +- .../00313_const_totals_extremes.sh | 12 +- ...inal_and_prewhere_condition_ver_column.sql | 4 +- .../00370_duplicate_columns_in_subqueries.sql | 2 +- .../00378_json_quote_64bit_integers.sql | 2 +- .../0_stateless/00445_join_nullable_keys.sql | 2 +- .../queries/0_stateless/00490_with_select.sql | 2 +- .../00597_push_down_predicate_long.reference | 10 +- .../00597_push_down_predicate_long.sql | 30 +- ...00621_regression_for_in_operator.reference | 4 +- .../00621_regression_for_in_operator.sql | 8 +- .../0_stateless/00674_join_on_syntax.sql | 2 +- .../0_stateless/00700_decimal_compare.sql | 2 +- .../queries/0_stateless/00722_inner_join.sql | 2 +- .../00736_disjunction_optimisation.reference | 18 +- .../00736_disjunction_optimisation.sql | 36 +-- .../0_stateless/00757_enum_defaults_const.sql | 2 +- .../00757_enum_defaults_const_analyzer.sql | 2 +- .../00800_low_cardinality_join.sql | 2 +- .../00800_low_cardinality_merge_join.sql.j2 | 2 +- .../0_stateless/00818_alias_bug_4110.sql | 2 +- .../0_stateless/00818_inner_join_bug_3567.sql | 2 +- ...00819_full_join_wrong_columns_in_block.sql | 2 +- .../0_stateless/00820_multiple_joins.sql | 2 +- ...multiple_joins_subquery_requires_alias.sql | 2 +- .../0_stateless/00830_join_overwrite.sql | 2 +- .../0_stateless/00835_if_generic_case.sql | 2 +- .../00848_join_use_nulls_segfault.sql | 2 +- .../00849_multiple_comma_join_2.sql | 34 +-- .../00855_join_with_array_join.sql | 2 +- .../queries/0_stateless/00858_issue_4756.sql | 2 +- tests/queries/0_stateless/00897_flatten.sql | 2 +- ...0940_order_by_read_in_order_query_plan.sql | 2 +- .../01000_subquery_requires_alias.sql | 2 +- .../01013_totals_without_aggregation.sql | 2 +- .../0_stateless/01018_ambiguous_column.sql | 2 +- .../01047_window_view_parser_inner_table.sql | 2 +- .../0_stateless/01048_window_view_parser.sql | 2 +- .../01049_join_low_card_bug_long.reference.j2 | 64 ++--- .../01049_join_low_card_bug_long.sql.j2 | 4 +- .../01050_window_view_parser_tumble.sql | 2 +- .../01051_window_view_parser_hop.sql | 2 +- .../01052_window_view_proc_tumble_to_now.sh | 2 +- .../01053_window_view_proc_hop_to_now.sh | 2 +- .../01054_window_view_proc_tumble_to.sh | 2 +- .../01055_window_view_proc_hop_to.sh | 2 +- .../01056_window_view_proc_hop_watch.py | 4 +- ..._window_view_event_tumble_to_strict_asc.sh | 2 +- ...058_window_view_event_hop_to_strict_asc.sh | 2 +- ..._window_view_event_hop_watch_strict_asc.py | 4 +- .../01060_window_view_event_tumble_to_asc.sh | 2 +- .../01061_window_view_event_hop_to_asc.sh | 2 +- .../01062_window_view_event_hop_watch_asc.py | 4 +- ...063_window_view_event_tumble_to_bounded.sh | 2 +- .../01064_window_view_event_hop_to_bounded.sh | 2 +- ...065_window_view_event_hop_watch_bounded.py | 4 +- ...iew_event_tumble_to_strict_asc_lateness.sh | 2 +- ...indow_view_event_tumble_to_asc_lateness.sh | 2 +- ...w_view_event_tumble_to_bounded_lateness.sh | 2 +- .../01069_window_view_proc_tumble_watch.py | 4 +- .../01070_window_view_watch_events.py | 4 +- ...01071_window_view_event_tumble_asc_join.sh | 2 +- ...72_window_view_multiple_columns_groupby.sh | 2 +- ...indow_view_event_tumble_to_asc_populate.sh | 2 +- ...dow_view_event_tumble_asc_join_populate.sh | 2 +- ...window_view_proc_tumble_to_now_populate.sh | 2 +- .../01076_window_view_alter_query_to.sh | 2 +- ...indow_view_alter_query_to_modify_source.sh | 2 +- .../01078_window_view_alter_query_watch.py | 6 +- ...9_window_view_inner_table_memory_tumble.sh | 2 +- ...1080_window_view_inner_table_memory_hop.sh | 2 +- .../01081_window_view_target_table_engine.sh | 2 +- .../01082_window_view_watch_limit.py | 4 +- .../0_stateless/01083_window_view_select.sh | 2 +- ...01084_window_view_with_table_identifier.sh | 2 +- .../0_stateless/01085_window_view_attach.sql | 2 +- .../0_stateless/01086_window_view_cleanup.sh | 2 +- .../01087_window_view_alter_query.sh | 2 +- .../01088_window_view_default_column.sh | 2 +- .../01142_join_lc_and_nullable_in_key.sql | 28 +- ...1142_merge_join_lc_and_nullable_in_key.sql | 24 +- tests/queries/0_stateless/01232_untuple.sql | 2 +- ...01300_group_by_other_keys_having.reference | 6 +- .../01300_group_by_other_keys_having.sql | 8 +- ..._redundant_functions_in_order_by.reference | 16 +- .../01323_redundant_functions_in_order_by.sql | 28 +- .../01353_low_cardinality_join_types.sql | 4 +- ...BY_injective_elimination_dictGet.reference | 2 +- ...GROUP_BY_injective_elimination_dictGet.sql | 2 +- .../0_stateless/01428_nullable_asof_join.sql | 12 +- .../01455_opentelemetry_distributed.reference | 4 +- .../01455_opentelemetry_distributed.sh | 6 +- .../01476_right_full_join_switch.sql | 4 +- .../01477_lc_in_merge_join_left_key.sql.j2 | 6 +- .../0_stateless/01479_cross_join_9855.sql | 4 +- .../0_stateless/01508_explain_header.sql | 2 +- .../01556_explain_select_with_union_query.sql | 2 +- .../01561_clickhouse_client_stage.reference | 10 +- .../01561_clickhouse_client_stage.sh | 2 +- .../0_stateless/01591_window_functions.sql | 2 +- .../0_stateless/01600_detach_permanently.sh | 4 +- ..._constraints_simple_optimization.reference | 6 +- .../01622_constraints_simple_optimization.sql | 6 +- ...2_constraints_where_optimization.reference | 12 +- .../01622_constraints_where_optimization.sql | 12 +- .../01623_constraints_column_swap.reference | 26 +- .../01623_constraints_column_swap.sql | 26 +- .../0_stateless/01646_rewrite_sum_if.sql | 2 +- .../0_stateless/01651_bugs_from_15889.sql | 2 +- .../0_stateless/01655_plan_optimizations.sh | 48 ++-- ...01655_plan_optimizations_merge_filters.sql | 4 +- ...ns_optimize_read_in_window_order.reference | 8 +- ...mizations_optimize_read_in_window_order.sh | 24 +- .../01671_merge_join_and_constants.sql | 2 +- .../01721_join_implicit_cast_long.sql.j2 | 12 +- .../0_stateless/01739_index_hint.reference | 4 +- .../queries/0_stateless/01739_index_hint.sql | 4 +- ...ze_skip_unused_shards_rewrite_in.reference | 4 +- ...optimize_skip_unused_shards_rewrite_in.sql | 4 +- ...1757_optimize_skip_unused_shards_limit.sql | 6 +- .../01763_filter_push_down_bugs.sql | 4 +- .../0_stateless/01786_explain_merge_tree.sh | 2 +- .../queries/0_stateless/01823_explain_json.sh | 2 +- .../01852_multiple_joins_with_union_join.sql | 2 +- ...01872_functions_to_subcolumns_analyzer.sql | 2 +- .../01890_cross_join_explain_crash.sql | 2 +- .../01913_names_of_tuple_literal.sql | 2 +- .../01925_join_materialized_columns.sql | 8 +- ...25_test_storage_merge_aliases_analyzer.sql | 2 +- ...hree_parts_identifiers_in_wrong_places.sql | 2 +- .../01942_dateTimeToSnowflakeID.sql | 2 +- .../01942_snowflakeIDToDateTime.sql | 2 +- ...istributed_group_by_sharding_key.reference | 2 +- ...mize_distributed_group_by_sharding_key.sql | 4 +- .../0_stateless/02000_join_on_const.reference | 12 +- .../0_stateless/02000_join_on_const.sql | 41 ++- .../0_stateless/02030_tuple_filter.sql | 2 +- .../02048_clickhouse_local_stage.reference | 10 +- .../02048_clickhouse_local_stage.sh | 2 +- .../02115_map_contains_analyzer.sql | 2 +- .../02116_tuple_element_analyzer.sql | 2 +- .../02125_query_views_log_window_function.sql | 2 +- ...window_functions_disable_optimizations.sql | 2 +- .../02136_scalar_read_rows_json.sh | 2 +- .../02149_read_in_order_fixed_prefix.sql | 12 +- .../02151_hash_table_sizes_stats_joins.sh | 4 +- .../02154_dictionary_get_http_json.sh | 2 +- .../02156_storage_merge_prewhere.sql | 4 +- .../0_stateless/02174_cte_scalar_cache.sql | 4 +- .../0_stateless/02174_cte_scalar_cache_mv.sql | 12 +- .../02184_hash_functions_and_ip_types.sql | 2 +- .../02226_analyzer_or_like_combine.reference | 4 +- .../02226_analyzer_or_like_combine.sql | 14 +- .../0_stateless/02227_union_match_by_name.sql | 2 +- .../02233_with_total_empty_chunk.sql | 2 +- .../02234_clickhouse_local_test_mode.sh | 5 +- .../02267_join_dup_columns_issue36199.sql | 4 +- ..._column_matcher_and_column_transformer.sql | 2 +- .../0_stateless/02303_query_kind.reference | 8 +- tests/queries/0_stateless/02303_query_kind.sh | 2 +- .../02315_grouping_constant_folding.reference | 2 +- .../02315_grouping_constant_folding.sql | 2 +- ..._distinct_in_order_optimization_explain.sh | 4 +- .../02337_analyzer_columns_basic.sql | 2 +- .../02337_multiple_joins_original_names.sql | 2 +- .../02338_analyzer_constants_basic.sql | 2 +- .../02339_analyzer_matcher_basic.sql | 2 +- .../0_stateless/02340_analyzer_functions.sql | 2 +- .../02341_analyzer_aliases_basics.sql | 2 +- .../02341_global_join_cte.reference | 4 +- .../0_stateless/02341_global_join_cte.sql | 4 +- .../02342_analyzer_compound_types.sql | 2 +- .../02342_window_view_different_struct.sql | 2 +- ...43_analyzer_column_transformers_strict.sql | 2 +- .../0_stateless/02343_analyzer_lambdas.sql | 2 +- .../02343_analyzer_lambdas_issue_28083.sql | 6 +- .../02343_analyzer_lambdas_issue_36677.sql | 2 +- ...alyzer_multiple_aliases_for_expression.sql | 2 +- .../0_stateless/02345_analyzer_subqueries.sql | 2 +- .../02346_fulltext_index_bug52019.sql | 6 +- .../02346_fulltext_index_match_predicate.sql | 12 +- .../0_stateless/02354_vector_search_bugs.sql | 2 +- .../0_stateless/02364_window_view_segfault.sh | 2 +- .../0_stateless/02366_explain_query_tree.sql | 2 +- .../02367_analyzer_table_alias_columns.sql | 2 +- .../02368_analyzer_table_functions.sql | 2 +- .../02369_analyzer_array_join_function.sql | 2 +- .../02370_analyzer_in_function.sql | 2 +- .../0_stateless/02371_analyzer_join_cross.sql | 2 +- .../0_stateless/02372_analyzer_join.sql.j2 | 2 +- .../02373_analyzer_join_use_nulls.sql | 2 +- .../0_stateless/02374_analyzer_array_join.sql | 2 +- .../02374_analyzer_join_using.sql.j2 | 2 +- ...2374_combine_multi_if_and_count_if_opt.sql | 2 +- .../0_stateless/02375_analyzer_union.sql | 2 +- .../02376_analyzer_in_function_subquery.sql | 2 +- .../02377_analyzer_in_function_set.sql | 2 +- ...ting_by_input_stream_properties_explain.sh | 4 +- .../02378_analyzer_projection_names.sql | 2 +- .../02379_analyzer_subquery_depth.sql | 2 +- .../02380_analyzer_join_sample.sql | 2 +- .../0_stateless/02381_analyzer_join_final.sql | 2 +- .../02381_join_dup_columns_in_plan.sql.j2 | 2 +- .../02382_analyzer_matcher_join_using.sql | 2 +- .../02383_analyzer_merge_tree_self_join.sql | 2 +- .../02384_analyzer_dict_get_join_get.sql | 2 +- ...5_analyzer_aliases_compound_expression.sql | 2 +- ...analyzer_in_function_nested_subqueries.sql | 2 +- .../0_stateless/02387_analyzer_cte.sql | 2 +- .../02388_analyzer_recursive_lambda.sql | 2 +- .../02389_analyzer_nested_lambda.sql | 2 +- .../02420_final_setting_analyzer.reference | 2 +- .../02420_final_setting_analyzer.sql | 2 +- ...02421_decimal_in_precision_issue_41125.sql | 2 +- .../0_stateless/02421_explain_subquery.sql | 4 +- ...2428_decimal_in_floating_point_literal.sql | 2 +- .../0_stateless/02428_parameterized_view.sh | 4 +- .../0_stateless/02451_order_by_monotonic.sh | 2 +- .../0_stateless/02459_group_by_all.sql | 2 +- ..._subqueries_table_expression_modifiers.sql | 2 +- .../02475_analyzer_join_tree_subquery.sql | 2 +- ..._analyzer_subquery_compound_expression.sql | 2 +- ...2475_or_function_alias_and_const_where.sql | 2 +- .../02476_analyzer_identifier_hints.sh | 90 +++--- ...2476_analyzer_join_with_unused_columns.sql | 2 +- .../0_stateless/02476_fuse_sum_count.sql | 2 +- .../02477_analyzer_array_join_with_join.sql | 2 +- ...02477_analyzer_ast_key_condition_crash.sql | 2 +- .../02477_analyzer_function_hints.sh | 10 +- .../0_stateless/02477_exists_fuzz_43478.sql | 2 +- .../0_stateless/02477_fuse_quantiles.sql | 2 +- ...ssions_optimizer_low_cardinality.reference | 12 +- ..._expressions_optimizer_low_cardinality.sql | 12 +- ...2478_analyzer_table_expression_aliases.sql | 2 +- .../02478_window_frame_type_groups.sql | 4 +- .../02479_analyzer_aggregation_crash.sql | 2 +- ...er_aggregation_totals_rollup_crash_fix.sql | 2 +- .../02479_analyzer_join_with_constants.sql | 2 +- .../02479_mysql_connect_to_self.sql | 2 +- .../02480_analyzer_alias_nullptr.sql | 2 +- .../0_stateless/02480_tlp_nan.reference | 20 +- tests/queries/0_stateless/02480_tlp_nan.sql | 20 +- .../02481_aggregation_in_order_plan.sql | 4 +- ...er_join_alias_unknown_identifier_crash.sql | 2 +- ...lyzer_optimize_aggregation_arithmetics.sql | 2 +- ...1_analyzer_optimize_grouping_sets_keys.sql | 2 +- .../02483_cuturlparameter_with_arrays.sql | 2 +- .../queries/0_stateless/02483_elapsed_time.sh | 2 +- .../0_stateless/02489_analyzer_indexes.sql | 3 +- .../02493_analyzer_sum_if_to_count_if.sql | 2 +- ...02493_analyzer_table_functions_untuple.sql | 2 +- ...r_uniq_injective_functions_elimination.sql | 2 +- ...analyzer_compound_expression_crash_fix.sql | 2 +- .../0_stateless/02494_query_cache_explain.sql | 2 +- .../02494_query_cache_nested_query_bug.sh | 4 +- .../02495_analyzer_storage_join.sql | 2 +- .../02495_sum_if_to_count_if_bug.sql | 5 +- .../02496_remove_redundant_sorting.sh | 4 +- ...nalyzer_sum_if_count_if_pass_crash_fix.sql | 2 +- ..._having_without_actual_aggregation_bug.sql | 2 +- .../02497_if_transform_strings_to_enum.sql | 2 +- .../02497_storage_join_right_assert.sql | 4 +- ...nctions_arithmetic_operations_pass_fix.sql | 2 +- .../02498_analyzer_settings_push_down.sql | 2 +- .../02498_storage_join_key_positions.sql.j2 | 2 +- ...er_aggregate_function_lambda_crash_fix.sql | 2 +- .../0_stateless/02499_analyzer_set_index.sql | 2 +- .../02500_analyzer_storage_view_crash_fix.sql | 2 +- .../02500_remove_redundant_distinct.sh | 4 +- ...501_analyzer_expired_context_crash_fix.sql | 2 +- ...02502_analyzer_insert_select_crash_fix.sql | 2 +- .../02503_join_switch_alias_fuzz.sql | 2 +- ...513_analyzer_duplicate_alias_crash_fix.sql | 2 +- .../0_stateless/02513_analyzer_sort_msan.sql | 2 +- .../02514_analyzer_drop_join_on.sql | 2 +- .../02515_analyzer_null_for_empty.sql | 2 +- ...6_join_with_totals_and_subquery_bug.sql.j2 | 4 +- .../02516_projections_and_context.sql | 4 +- ...518_rewrite_aggregate_function_with_if.sql | 2 +- ...21_analyzer_aggregation_without_column.sql | 2 +- .../02521_analyzer_array_join_crash.sql | 2 +- .../02525_analyzer_function_in_crash_fix.sql | 2 +- ...02532_analyzer_aggregation_with_rollup.sql | 2 +- .../02534_analyzer_grouping_function.sql | 2 +- .../02535_analyzer_group_by_use_nulls.sql | 2 +- .../02535_analyzer_limit_offset.sql | 2 +- .../02538_analyzer_create_table_as_select.sql | 2 +- ...zer_matcher_alias_materialized_columns.sql | 2 +- ...json_ignore_unknown_keys_in_named_tuple.sh | 2 +- ...02541_analyzer_grouping_sets_crash_fix.sql | 2 +- ..._optimize_group_by_function_keys_crash.sql | 2 +- .../02553_type_object_analyzer.sql | 2 +- ..._fix_grouping_sets_predicate_push_down.sql | 8 +- .../02560_analyzer_materialized_view.sql | 2 +- .../0_stateless/02563_analyzer_merge.sql | 2 +- .../02564_analyzer_cross_to_inner.sql | 2 +- .../02565_analyzer_limit_settings.sql | 2 +- ...66_analyzer_limit_settings_distributed.sql | 2 +- .../02567_and_consistency.reference | 2 +- .../0_stateless/02567_and_consistency.sql | 4 +- .../02576_predicate_push_down_sorting_fix.sql | 2 +- .../02576_rewrite_array_exists_to_has.sql | 4 +- .../02577_analyzer_array_join_calc_twice.sql | 2 +- .../0_stateless/02579_fill_empty_chunk.sql | 2 +- .../02579_fill_empty_chunk_analyzer.sql | 2 +- ...alyzer_join_subquery_empty_column_list.sql | 4 +- .../02661_quantile_approx.reference | 12 +- .../0_stateless/02661_quantile_approx.sql | 14 +- .../02662_first_last_value.reference | 2 +- .../0_stateless/02662_first_last_value.sql | 2 +- ...al_optimizer_removing_redundant_checks.sql | 2 +- .../02674_trivial_count_analyzer.reference | 2 +- .../02674_trivial_count_analyzer.sql | 2 +- ...75_predicate_push_down_filled_join_fix.sql | 2 +- .../02676_analyzer_limit_offset.sql | 2 +- ...676_distinct_reading_in_order_analyzer.sql | 2 +- .../02677_analyzer_bitmap_has_any.sql | 5 +- .../02677_analyzer_compound_expressions.sql | 2 +- .../02677_get_subcolumn_array_of_tuples.sql | 2 +- ...explain_merge_tree_prewhere_row_policy.sql | 4 +- .../02699_polygons_sym_difference_rollup.sql | 4 +- .../02699_polygons_sym_difference_total.sql | 2 +- ...polygons_sym_difference_total_analyzer.sql | 2 +- .../02701_invalid_having_NOT_AN_AGGREGATE.sql | 2 +- .../02702_logical_optimizer_with_nulls.sql | 2 +- ...ry_tree_is_forbidden_with_old_analyzer.sql | 2 +- ...2704_storage_merge_explain_graph_crash.sql | 2 +- .../02707_analyzer_nested_lambdas_types.sql | 10 +- .../02722_matcher_join_use_nulls.sql.j2 | 2 +- .../0_stateless/02725_cnf_large_check.sql | 8 +- .../02731_analyzer_join_resolve_nested.sql.j2 | 2 +- ..._parallel_replicas_join_subquery.reference | 8 +- .../02731_parallel_replicas_join_subquery.sql | 12 +- .../0_stateless/02734_optimize_group_by.sql | 4 +- ...s_with_subqueries_profile_events.reference | 24 +- ..._queries_with_subqueries_profile_events.sh | 26 +- .../02767_into_outfile_extensions_msan.sh | 2 +- .../02771_ignore_data_skipping_indices.sql | 4 +- ...02771_parallel_replicas_analyzer.reference | 2 +- .../02771_parallel_replicas_analyzer.sql | 2 +- .../02771_semi_join_use_nulls.sql.j2 | 4 +- .../02783_date_predicate_optimizations.sql | 100 +++---- ...lel_replicas_trivial_count_optimization.sh | 8 +- ...l_conditions_to_prewhere_analyzer_asan.sql | 2 +- ...mizations_ast_query_tree_rewrite.reference | 56 ++-- ...e_optimizations_ast_query_tree_rewrite.sql | 56 ++-- .../02803_remote_cannot_clone_block.sql | 2 +- .../02812_bug_with_unused_join_columns.sql | 2 +- ...r_aggregate_functions_of_group_by_keys.sql | 2 +- .../02815_join_algorithm_setting.sql | 4 +- ...834_analyzer_with_statement_references.sql | 2 +- .../0_stateless/02835_join_step_explain.sql | 2 +- .../02840_merge__table_or_filter.reference | 32 +-- .../02840_merge__table_or_filter.sql.j2 | 8 +- ...41_valid_json_and_xml_on_http_exception.sh | 2 +- .../0_stateless/02841_with_clause_resolve.sql | 48 ++-- .../02861_filter_pushdown_const_bug.sql | 2 +- .../02864_statistics_materialize_in_merge.sql | 2 +- .../02866_size_of_marks_skip_idx_explain.sql | 2 +- ...8_distinct_to_count_optimization.reference | 32 +-- .../02868_distinct_to_count_optimization.sql | 64 ++--- ..._key_index_in_function_different_types.sql | 4 +- .../02890_named_tuple_functions.sql | 2 +- .../02890_untuple_column_names.reference | 2 +- .../02890_untuple_column_names.sql | 44 +-- .../02911_analyzer_explain_estimate.sql | 2 +- ...yzer_order_by_read_in_order_query_plan.sql | 2 +- ...lyzer_remove_unused_projection_columns.sql | 2 +- .../02911_join_on_nullsafe_optimization.sql | 2 +- .../02911_support_alias_column_in_indices.sql | 8 +- .../0_stateless/02915_analyzer_fuzz_1.sql | 2 +- .../0_stateless/02915_analyzer_fuzz_2.sql | 3 +- .../0_stateless/02915_analyzer_fuzz_5.sql | 2 +- .../0_stateless/02915_analyzer_fuzz_6.sql | 2 +- .../0_stateless/02918_join_pm_lc_crash.sql | 9 +- .../02918_optimize_count_for_merge_tables.sql | 4 +- .../02922_respect_nulls_parser.sql | 2 +- ...alyzer_rewrite_sum_column_and_constant.sql | 2 +- .../02932_parallel_replicas_fuzzer.sql | 2 +- .../queries/0_stateless/02933_paste_join.sql | 6 +- .../0_stateless/02943_order_by_all.sql | 32 +-- ...ngrambf_indexes_support_match_function.sql | 24 +- .../02944_variant_as_common_type_analyzer.sql | 3 +- .../02952_conjunction_optimization.sql | 2 +- .../02954_analyzer_fuzz_i57086.sql | 2 +- ...nalyzer_using_functional_args.reference.j2 | 20 +- ...2955_analyzer_using_functional_args.sql.j2 | 28 +- .../02955_sparkBar_alias_sparkbar.sql | 3 +- .../02962_join_using_bug_57894.sql | 4 +- .../0_stateless/02967_analyzer_fuzz.sql | 4 +- ...allel_replicas_join_algo_and_analyzer_3.sh | 20 +- ...llel_replicas_joins_and_analyzer.reference | 72 ++--- ...arallel_replicas_joins_and_analyzer.sql.j2 | 36 +-- ...analyzer_eliminate_injective_functions.sql | 2 +- .../02969_functions_to_subcolumns_if_null.sql | 8 +- .../0_stateless/02971_analyzer_remote_id.sh | 2 +- ...1_functions_to_subcolumns_column_names.sql | 2 +- .../02971_functions_to_subcolumns_map.sql | 2 +- .../02971_functions_to_subcolumns_variant.sql | 2 +- .../02972_parallel_replicas_cte.sql | 6 +- .../02974_analyzer_array_join_subcolumn.sql | 10 +- ..._logical_optimizer_pass_lowcardinality.sql | 2 +- .../02989_join_using_parent_scope.reference | 32 +-- .../02989_join_using_parent_scope.sql | 32 +-- .../02991_count_rewrite_analyzer.sql | 2 +- .../02992_analyzer_group_by_const.sql | 2 +- .../02996_analyzer_prewhere_projection.sql | 2 +- .../02998_analyzer_prewhere_report.sql | 2 +- ...8_analyzer_secret_args_tree_node.reference | 10 +- .../02998_analyzer_secret_args_tree_node.sql | 2 +- ...rojection_after_attach_partition.reference | 4 +- ...2998_projection_after_attach_partition.sql | 4 +- .../02999_analyzer_preimage_null.sql | 2 +- .../03001_analyzer_nullable_nothing.sql | 2 +- .../0_stateless/03002_analyzer_prewhere.sql | 2 +- .../0_stateless/03003_analyzer_setting.sql | 8 +- .../03003_functions_to_subcolumns_final.sql | 2 +- .../03006_join_on_inequal_expression_2.sql.j2 | 2 +- .../03006_join_on_inequal_expression_3.sql.j2 | 2 +- .../03006_join_on_inequal_expression_4.sql.j2 | 2 +- ...006_join_on_inequal_expression_fast.sql.j2 | 2 +- ...llel_replicas_cte_explain_syntax_crash.sql | 2 +- ...007_column_nullable_uninitialzed_value.sql | 2 +- .../03010_sum_to_to_count_if_nullable.sql | 4 +- ...se_nulls_with_materialize_and_analyzer.sql | 3 +- .../03014_analyzer_groupby_fuzz_60317.sql | 4 +- ...nulls_injective_functions_and_analyzer.sql | 3 +- .../03015_analyzer_groupby_fuzz_60772.sql | 4 +- .../03016_analyzer_groupby_fuzz_59796.sql | 2 +- .../03017_analyzer_groupby_fuzz_61600.sql | 4 +- ...mize_group_by_function_keys_with_nulls.sql | 3 +- ...23_group_by_use_nulls_analyzer_crashes.sql | 4 +- .../03023_remove_unused_column_distinct.sql | 2 +- .../03031_filter_float64_logical_error.sql | 4 +- .../03031_tuple_elimination_analyzer.sql | 2 +- .../0_stateless/03032_redundant_equals.sql | 4 +- ...analyzer_merge_engine_filter_push_down.sql | 3 +- .../03033_analyzer_query_parameters.sh | 4 +- .../0_stateless/03033_cte_numbers_memory.sql | 2 +- .../03033_recursive_cte_basic.reference | 2 +- .../0_stateless/03033_recursive_cte_basic.sql | 2 +- .../03033_with_fill_interpolate.sql | 2 +- .../0_stateless/03034_normalized_ast.sql | 2 +- .../0_stateless/03034_recursive_cte_tree.sql | 2 +- ...3034_recursive_cte_tree_fuzz_crash_fix.sql | 2 +- ...34_recursive_cte_tree_merge_tree.reference | 2 +- .../03034_recursive_cte_tree_merge_tree.sql | 2 +- .../03035_alias_column_bug_distributed.sql | 2 +- .../03035_internal_functions_direct_call.sql | 2 +- .../03035_recursive_cte_postgres_1.reference | 2 +- .../03035_recursive_cte_postgres_1.sql | 2 +- ..._join_filter_push_down_equivalent_sets.sql | 2 +- .../03036_recursive_cte_postgres_2.reference | 2 +- .../03036_recursive_cte_postgres_2.sql | 2 +- .../0_stateless/03036_with_numbers.sql | 2 +- .../03037_recursive_cte_postgres_3.reference | 2 +- .../03037_recursive_cte_postgres_3.sql | 2 +- .../queries/0_stateless/03037_union_view.sql | 4 +- .../0_stateless/03038_ambiguous_column.sql | 2 +- .../03038_recursive_cte_postgres_4.reference | 2 +- .../03038_recursive_cte_postgres_4.sql | 2 +- .../03039_recursive_cte_postgres_5.reference | 2 +- .../03039_recursive_cte_postgres_5.sql | 2 +- ...039_unknown_identifier_window_function.sql | 2 +- .../0_stateless/03040_alias_column_join.sql | 2 +- .../0_stateless/03040_array_sum_and_join.sql | 2 +- .../03040_recursive_cte_postgres_6.reference | 2 +- .../03040_recursive_cte_postgres_6.sql | 2 +- .../03041_analyzer_gigachad_join.sql | 2 +- .../03041_recursive_cte_postgres_7.reference | 2 +- .../03041_recursive_cte_postgres_7.sql | 2 +- .../03041_select_with_query_result.sql | 2 +- .../0_stateless/03042_analyzer_alias_join.sql | 2 +- .../0_stateless/03042_not_found_column_c1.sql | 2 +- .../03043_group_array_result_is_expected.sql | 2 +- .../0_stateless/03044_analyzer_alias_join.sql | 2 +- ...044_array_join_columns_in_nested_table.sql | 2 +- .../03045_analyzer_alias_join_with_if.sql | 2 +- ..._unknown_identifier_alias_substitution.sql | 2 +- .../03046_column_in_block_array_join.sql | 2 +- .../0_stateless/03047_analyzer_alias_join.sql | 2 +- ..._group_by_field_identified_aggregation.sql | 2 +- .../03048_not_found_column_xxx_in_block.sql | 2 +- .../03049_analyzer_group_by_alias.sql | 2 +- ...unknown_identifier_materialized_column.sql | 2 +- .../0_stateless/03050_select_one_one_one.sql | 2 +- tests/queries/0_stateless/03051_many_ctes.sql | 2 +- .../03052_query_hash_includes_aliases.sql | 2 +- .../0_stateless/03053_analyzer_join_alias.sql | 2 +- .../0_stateless/03054_analyzer_join_alias.sql | 2 +- .../03055_analyzer_subquery_group_array.sql | 2 +- .../03057_analyzer_subquery_alias_join.sql | 2 +- .../03058_analyzer_ambiguous_columns.sql | 3 +- ...59_analyzer_join_engine_missing_column.sql | 2 +- .../03060_analyzer_regular_view_alias.sql | 2 +- ...61_analyzer_alias_as_right_key_in_join.sql | 2 +- ...62_analyzer_join_engine_missing_column.sql | 2 +- ...lyzer_multi_join_wrong_table_specifier.sql | 3 +- .../03064_analyzer_named_subqueries.sql | 2 +- ...065_analyzer_cross_join_and_array_join.sql | 2 +- .../03066_analyzer_global_with_statement.sql | 2 +- .../03067_analyzer_complex_alias_join.sql | 2 +- .../03068_analyzer_distributed_join.sql | 2 +- ...3069_analyzer_with_alias_in_array_join.sql | 2 +- .../03070_analyzer_CTE_scalar_as_numbers.sql | 2 +- ...array_join_forbid_non_existing_columns.sql | 2 +- .../03071_fix_short_circuit_logic.sql | 2 +- ...analyzer_missing_columns_from_subquery.sql | 2 +- .../03073_analyzer_alias_as_column_name.sql | 2 +- .../03074_analyzer_alias_column_in_view.sql | 2 +- .../03075_analyzer_subquery_alias.sql | 2 +- .../03076_analyzer_multiple_joins_alias.sql | 3 +- ...analyzer_multi_scalar_subquery_aliases.sql | 2 +- ...analyzer_multi_scalar_subquery_aliases.sql | 2 +- ...lyzer_numeric_literals_as_column_names.sql | 2 +- ..._column_name_to_alias__virtual_columns.sql | 2 +- .../03080_incorrect_join_with_merge.sql | 2 +- .../03081_analyzer_agg_func_CTE.sql | 2 +- ...3082_analyzer_left_join_correct_column.sql | 2 +- .../03084_analyzer_join_column_alias.sql | 2 +- .../03085_analyzer_alias_column_group_by.sql | 2 +- ..._analyzer_window_func_part_of_group_by.sql | 2 +- .../03087_analyzer_subquery_with_alias.sql | 2 +- ...8_analyzer_ambiguous_column_multi_call.sql | 2 +- .../03089_analyzer_alias_replacement.sql | 2 +- ...090_analyzer_multiple_using_statements.sql | 2 +- ...same_table_name_in_different_databases.sql | 2 +- ...same_table_name_in_different_databases.sql | 2 +- .../03093_analyzer_column_alias.sql | 2 +- .../0_stateless/03093_analyzer_miel_test.sql | 2 +- .../03093_bug37909_query_does_not_finish.sql | 2 +- .../03094_analyzer_fiddle_multiif.sql | 2 +- .../03094_named_tuple_bug24607.sql | 2 +- .../0_stateless/03094_one_thousand_joins.sql | 2 +- .../03095_window_functions_qualify.sql | 2 +- ..._text_log_format_string_args_not_empty.sql | 2 +- .../03097_query_log_join_processes.sql | 2 +- .../03098_prefer_column_to_alias_subquery.sql | 4 +- .../0_stateless/03099_analyzer_multi_join.sql | 2 +- .../03100_analyzer_constants_in_multiif.sql | 2 +- .../03101_analyzer_identifiers_1.sql | 2 +- .../03101_analyzer_identifiers_2.sql | 2 +- .../03101_analyzer_identifiers_3.sql | 2 +- .../03101_analyzer_identifiers_4.sql | 2 +- .../03101_analyzer_invalid_join_on.sql | 14 +- .../03102_prefer_column_name_to_alias.sql | 2 +- .../03103_positional_arguments.sql | 2 +- .../0_stateless/03104_create_view_join.sql | 8 +- ...ill_formed_select_in_materialized_view.sql | 4 +- .../0_stateless/03108_describe_union_all.sql | 4 +- .../queries/0_stateless/03109_ast_too_big.sql | 4 +- .../0_stateless/03110_unicode_alias.sql | 2 +- .../0_stateless/03111_inner_join_group_by.sql | 4 +- ...112_analyzer_not_found_column_in_block.sql | 4 +- ...3_analyzer_not_found_column_in_block_2.sql | 6 +- .../03114_analyzer_cte_with_join.sql | 2 +- .../0_stateless/03115_alias_exists_column.sql | 2 +- ...analyzer_explicit_alias_as_column_name.sql | 2 +- ...3117_analyzer_same_column_name_as_func.sql | 2 +- .../03118_analyzer_multi_join_prewhere.sql | 2 +- ..._analyzer_window_function_in_CTE_alias.sql | 2 +- .../0_stateless/03120_analyzer_dist_join.sql | 2 +- .../03120_analyzer_param_in_CTE_alias.sql | 2 +- ...nalyzer_filed_redefenition_in_subquery.sql | 2 +- ...22_analyzer_collate_in_window_function.sql | 2 +- .../03123_analyzer_dist_join_CTE.sql | 2 +- .../03124_analyzer_nested_CTE_dist_in.sql | 2 +- .../03125_analyzer_CTE_two_joins.sql | 2 +- .../03126_column_not_under_group_by.sql | 3 +- .../0_stateless/03129_cte_with_final.sql | 2 +- .../03130_analyzer_self_join_group_by.sql | 2 +- ...03130_convert_outer_join_to_inner_join.sql | 2 +- ...gregate_function_with_if_implicit_cast.sql | 2 +- .../0_stateless/03132_sqlancer_union_all.sql | 2 +- .../0_stateless/03142_untuple_crash.sql | 2 +- .../03142_window_function_limit_by.sql | 7 +- tests/queries/0_stateless/03143_cte_scope.sql | 2 +- .../03143_group_by_constant_secondary.sql | 2 +- .../03143_parallel_replicas_mat_view_bug.sql | 2 +- ..._aggregate_states_with_different_types.sql | 2 +- .../0_stateless/03144_invalid_filter.sql | 2 +- tests/queries/0_stateless/03146_bug47862.sql | 2 +- .../0_stateless/03146_tpc_ds_grouping.sql | 2 +- .../03148_query_log_used_dictionaries.sql | 8 +- ..._streams_to_max_threads_ratio_overflow.sql | 4 +- ...03150_grouping_sets_use_nulls_pushdown.sql | 6 +- ...lyzer_view_read_only_necessary_columns.sql | 2 +- ...in_filter_push_down_equivalent_columns.sql | 2 +- .../03154_recursive_cte_distributed.sql | 2 +- .../03155_analyzer_interpolate.sql | 2 +- .../03155_in_nested_subselects.sql | 8 +- .../0_stateless/03161_cnf_reduction.reference | 4 +- .../0_stateless/03161_cnf_reduction.sql | 12 +- .../03164_analyzer_global_in_alias.sql | 2 +- .../03164_early_constant_folding_analyzer.sql | 2 +- .../03164_materialize_skip_index.sql | 2 +- .../03165_order_by_duplicate.reference | 2 +- .../0_stateless/03165_order_by_duplicate.sql | 2 +- ...03166_mv_prewhere_duplicating_name_bug.sql | 4 +- .../03166_skip_indexes_vertical_merge_1.sql | 2 +- .../03167_parametrized_view_with_cte.sql | 2 +- ..._injective_functions_inside_uniq_crash.sql | 4 +- .../03170_part_offset_as_table_column.sql | 4 +- .../0_stateless/03171_condition_pushdown.sql | 2 +- .../03171_function_to_subcolumns_fuzzer.sql | 2 +- .../0_stateless/03173_forbid_qualify.sql | 4 +- .../03173_parallel_replicas_join_bug.sh | 2 +- .../0_stateless/03174_merge_join_bug.sql | 2 +- .../03199_join_with_materialized_column.sql | 2 +- .../03199_queries_with_new_analyzer.sql | 3 +- .../03200_memory_engine_alter_dynamic.sql | 5 +- .../03200_subcolumns_join_use_nulls.sql | 2 +- .../03201_sumIf_to_countIf_return_type.sql | 2 +- .../00081_group_by_without_key_and_totals.sql | 3 +- .../00172_early_constant_folding.sql | 4 +- .../00173_group_by_use_nulls.reference | 2 +- .../1_stateful/00173_group_by_use_nulls.sql | 2 +- 668 files changed, 1903 insertions(+), 1667 deletions(-) diff --git a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml index d5b876a4c85..476464e9cc2 100644 --- a/docker/test/fuzzer/query-fuzzer-tweaks-users.xml +++ b/docker/test/fuzzer/query-fuzzer-tweaks-users.xml @@ -28,9 +28,9 @@ - + - + diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 36782101fa7..73e0376d95a 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -139,9 +139,9 @@ EOL - + - + diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index 6428c0e90d5..c5d13ab63a5 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -118,7 +118,7 @@ And the result of interpreting the `INSERT SELECT` query is a "completed" `Query `InterpreterSelectQuery` uses `ExpressionAnalyzer` and `ExpressionActions` machinery for query analysis and transformations. This is where most rule-based query optimizations are performed. `ExpressionAnalyzer` is quite messy and should be rewritten: various query transformations and optimizations should be extracted into separate classes to allow for modular transformations of the query. -To address current problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` is being developed. It is a new version of `InterpreterSelectQuery` that does not use `ExpressionAnalyzer` and introduces an additional abstraction level between `AST` and `QueryPipeline` called `QueryTree`. It is not production-ready yet, but it can be tested with the `allow_experimental_analyzer` flag. +To address current problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` is being developed. It is a new version of `InterpreterSelectQuery` that does not use `ExpressionAnalyzer` and introduces an additional abstraction level between `AST` and `QueryPipeline` called `QueryTree`. It is not production-ready yet, but it can be tested with the `enable_analyzer` flag. ## Functions {#functions} diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 298c6dacd06..c9b3c30d30d 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -123,7 +123,7 @@ To ensure consistent and expected results, especially when migrating old queries In the new version of the analyzer, the rules for determining the common supertype for columns specified in the `USING` clause have been standardized to produce more predictable outcomes, especially when dealing with type modifiers like `LowCardinality` and `Nullable`. - `LowCardinality(T)` and `T`: When a column of type `LowCardinality(T)` is joined with a column of type `T`, the resulting common supertype will be `T`, effectively discarding the `LowCardinality` modifier. - + - `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. **Example:** @@ -144,7 +144,7 @@ During projection names computation, aliases are not substituted. SELECT 1 + 1 AS x, x + 1 -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 FORMAT PrettyCompact ┌─x─┬─plus(plus(1, 1), 1)─┐ @@ -154,7 +154,7 @@ FORMAT PrettyCompact SELECT 1 + 1 AS x, x + 1 -SETTINGS allow_experimental_analyzer = 1 +SETTINGS enable_analyzer = 1 FORMAT PrettyCompact ┌─x─┬─plus(x, 1)─┐ @@ -177,7 +177,7 @@ SELECT toTypeName(if(0, [2, 3, 4], 'String')) ### Heterogeneous clusters -The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. +The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `enable_analyzer` setting values. ### Mutations are interpreted by previous analyzer diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 143ce836beb..35547c3a9a6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4051,7 +4051,7 @@ Rewrite aggregate functions with if expression as argument when logically equiva For example, `avg(if(cond, col, null))` can be rewritten to `avgOrNullIf(cond, col)`. It may improve performance. :::note -Supported only with experimental analyzer (`allow_experimental_analyzer = 1`). +Supported only with experimental analyzer (`enable_analyzer = 1`). ::: ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} diff --git a/docs/ru/development/architecture.md b/docs/ru/development/architecture.md index 575799cccc4..0701c8f4a51 100644 --- a/docs/ru/development/architecture.md +++ b/docs/ru/development/architecture.md @@ -115,7 +115,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны `InterpreterSelectQuery` использует `ExpressionAnalyzer` и `ExpressionActions` механизмы для анализа запросов и преобразований. Именно здесь выполняется большинство оптимизаций запросов на основе правил. `ExpressionAnalyzer` написан довольно грязно и должен быть переписан: различные преобразования запросов и оптимизации должны быть извлечены в отдельные классы, чтобы позволить модульные преобразования или запросы. -Для решения текущих проблем, существующих в интерпретаторах, разрабатывается новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он еще не готов к использованию в продакшене, но его можно протестировать с помощью флага `allow_experimental_analyzer`. +Для решения текущих проблем, существующих в интерпретаторах, разрабатывается новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он еще не готов к использованию в продакшене, но его можно протестировать с помощью флага `enable_analyzer`. ## Функции {#functions} diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index dd8348ea04f..1c7f222aa78 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -196,11 +196,11 @@ void HedgedConnections::sendQuery( modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; } - /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. + /// FIXME: Remove once we will make `enable_analyzer` obsolete setting. /// Make the analyzer being set, so it will be effectively applied on the remote server. /// In other words, the initiator always controls whether the analyzer enabled or not for /// all servers involved in the distributed query processing. - modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); + modified_settings.set("enable_analyzer", static_cast(modified_settings.enable_analyzer)); replica.connection->sendQuery( timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 244eccf1ed9..7ca22ae4c81 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -150,11 +150,11 @@ void MultiplexedConnections::sendQuery( client_info.number_of_current_replica = replica_info->number_of_current_replica; } - /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. + /// FIXME: Remove once we will make `enable_analyzer` obsolete setting. /// Make the analyzer being set, so it will be effectively applied on the remote server. /// In other words, the initiator always controls whether the analyzer enabled or not for /// all servers involved in the distributed query processing. - modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); + modified_settings.set("enable_analyzer", static_cast(modified_settings.enable_analyzer)); const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d498ce7699..d6c0dc223b2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -638,7 +638,7 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ - M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `allow_experimental_analyzer`", 0) \ + M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `enable_analyzer`", 0) \ M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \ M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \ M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ @@ -943,8 +943,7 @@ class IColumn; \ M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \ \ - /* Analyzer: It's not experimental anymore (WIP) */ \ - M(Bool, allow_experimental_analyzer, true, "Allow new query analyzer.", IMPORTANT) \ + M(Bool, enable_analyzer, true, "Allow new query analyzer.", IMPORTANT) ALIAS(allow_experimental_analyzer) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ \ M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5b94391bade..08fb6dc3301 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -500,6 +500,265 @@ static std::initializer_list 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"}}}, }; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e1b7e92ee5d..b8d70e5d5dd 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1411,7 +1411,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return {}; PreparedSets::Hash set_key; - if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) + if (data.getContext()->getSettingsRef().enable_analyzer && !identifier) { /// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter. /// This is a hacky way to allow reusing cache for prepared sets. diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index e35d31d2350..0948f24eca0 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -68,7 +68,7 @@ ASTPtr rewriteSelectQuery( // are written into the query context and will be sent by the query pipeline. select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {}); - if (!context->getSettingsRef().allow_experimental_analyzer) + if (!context->getSettingsRef().enable_analyzer) { if (table_function_ptr) select_query.addTableFunction(table_function_ptr); @@ -165,7 +165,7 @@ void SelectStreamFactory::createForShardImpl( auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) { Block shard_header; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze()); else shard_header = header; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d04a73e384e..6c8ab11bfc9 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -300,7 +300,7 @@ void executeQuery( const size_t shards = cluster->getShardCount(); - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) { @@ -581,7 +581,7 @@ void executeQueryWithParallelReplicasCustomKey( /// Return directly (with correct header) if no shard to query. if (query_info.getCluster()->getShardsInfo().empty()) { - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) return; Pipe pipe(std::make_shared(header)); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 971f90bd3cd..ea631ef01d5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -834,7 +834,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti Block as_select_sample; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + if (getContext()->getSettingsRef().enable_analyzer) { as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); } @@ -1327,7 +1327,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { Block input_block; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + if (getContext()->getSettingsRef().enable_analyzer) { input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 39fc85a5e23..4a061f02c2b 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -129,7 +129,7 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression auto select_query = table_expression.subquery->children.at(0); auto current_context = getContext(); - if (settings.allow_experimental_analyzer) + if (settings.enable_analyzer) { SelectQueryOptions select_query_options; sample_block = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock(); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index bedd9cb4a80..2fbfbf3a809 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -394,9 +394,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl() } case ASTExplainQuery::QueryTree: { - if (!getContext()->getSettingsRef().allow_experimental_analyzer) + if (!getContext()->getSettingsRef().enable_analyzer) throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "EXPLAIN QUERY TREE is only supported with a new analyzer. Set allow_experimental_analyzer = 1."); + "EXPLAIN QUERY TREE is only supported with a new analyzer. Set enable_analyzer = 1."); if (ast.getExplainedQuery()->as() == nullptr) throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERY TREE query"); @@ -453,7 +453,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() ContextPtr context; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + if (getContext()->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options); context = interpreter.getContext(); @@ -499,7 +499,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() QueryPlan plan; ContextPtr context; - if (getContext()->getSettingsRef().allow_experimental_analyzer) + if (getContext()->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options); context = interpreter.getContext(); @@ -558,7 +558,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() QueryPlan plan; ContextPtr context = getContext(); - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); context = interpreter.getContext(); diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 12b3b510098..a909c4e602d 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -118,7 +118,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte if (query->as()) { - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) interpreter_name = "InterpreterSelectQueryAnalyzer"; /// This is internal part of ASTSelectWithUnionQuery. /// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child. @@ -129,7 +129,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { ProfileEvents::increment(ProfileEvents::SelectQuery); - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) interpreter_name = "InterpreterSelectQueryAnalyzer"; else interpreter_name = "InterpreterSelectWithUnionQuery"; @@ -222,7 +222,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { const auto kind = query->as()->getKind(); if (kind == ASTExplainQuery::ParsedAST || kind == ASTExplainQuery::AnalyzedSyntax) - context->setSetting("allow_experimental_analyzer", false); + context->setSetting("enable_analyzer", false); interpreter_name = "InterpreterExplainQuery"; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index c97593a1781..0213e2a2c42 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -95,7 +95,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) Block header_block; auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - if (current_context->getSettingsRef().allow_experimental_analyzer) + if (current_context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter_select(query.select, current_context, select_query_options); header_block = interpreter_select.getSampleBlock(); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 57ad5caa4c7..c049dbc9cc1 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -189,7 +189,7 @@ bool isStorageTouchedByMutations( std::optional interpreter_select_query; BlockIO io; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits()); @@ -415,9 +415,9 @@ MutationsInterpreter::MutationsInterpreter( , logger(getLogger("MutationsInterpreter(" + source.getStorage()->getStorageID().getFullTableName() + ")")) { auto new_context = Context::createCopy(context_); - if (new_context->getSettingsRef().allow_experimental_analyzer) + if (new_context->getSettingsRef().enable_analyzer) { - new_context->setSetting("allow_experimental_analyzer", false); + new_context->setSetting("enable_analyzer", false); LOG_DEBUG(logger, "Will use old analyzer to prepare mutation"); } context = std::move(new_context); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ce58f7f922c..7476915ab8a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -685,10 +685,10 @@ void validateAnalyzerSettings(ASTPtr ast, bool context_value) if (auto * set_query = node->as()) { - if (auto * value = set_query->changes.tryGet("allow_experimental_analyzer")) + if (auto * value = set_query->changes.tryGet("enable_analyzer")) { if (top_level != value->safeGet()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'enable_analyzer' is changed in the subquery. Top level value: {}", top_level); } } @@ -912,7 +912,7 @@ static std::tuple executeQueryImpl( /// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter), /// to allow settings to take effect. InterpreterSetQuery::applySettingsFromQuery(ast, context); - validateAnalyzerSettings(ast, context->getSettingsRef().allow_experimental_analyzer); + validateAnalyzerSettings(ast, context->getSettingsRef().enable_analyzer); if (auto * insert_query = ast->as()) insert_query->tail = istr; diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index cf18cbbb54a..c4a791e85e1 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -141,7 +141,7 @@ Block getHeaderForProcessingStage( Block result; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { auto storage = std::make_shared(storage_snapshot->storage.getStorageID(), storage_snapshot->getAllColumnsDescription(), diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index d8624a1c99b..dc4b7fd733b 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -65,7 +65,7 @@ std::unique_ptr createLocalPlan( .setShardInfo(static_cast(shard_num), static_cast(shard_count)) .ignoreASTOptimizations(); - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 98d66ed77c3..a2d5ec5d1cb 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -319,7 +319,7 @@ std::optional generateViewChain( Block header; /// Get list of columns we get from select query. - if (select_context->getSettingsRef().allow_experimental_analyzer) + if (select_context->getSettingsRef().enable_analyzer) header = InterpreterSelectQueryAnalyzer::getSampleBlock(query, select_context); else header = InterpreterSelectQuery(query, select_context, SelectQueryOptions()).getSampleBlock(); @@ -613,7 +613,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat QueryPipelineBuilder pipeline; - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(view.query, local_context, local_context->getViewSource(), SelectQueryOptions().ignoreAccessCheck()); pipeline = interpreter.buildQueryPipeline(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c5dfe3e6e5f..4262716b406 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1904,14 +1904,14 @@ void TCPHandler::receiveQuery() /// Settings /// - /// FIXME: Remove when allow_experimental_analyzer will become obsolete. + /// FIXME: Remove when enable_analyzer will become obsolete. /// Analyzer became Beta in 24.3 and started to be enabled by default. /// We have to disable it for ourselves to make sure we don't have different settings on /// different servers. if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && client_info.getVersionNumber() < VersionNumber(23, 3, 0) - && !passed_settings.allow_experimental_analyzer.changed) - passed_settings.set("allow_experimental_analyzer", false); + && !passed_settings.enable_analyzer.changed) + passed_settings.set("enable_analyzer", false); auto settings_changes = passed_settings.changes(); query_kind = query_context->getClientInfo().query_kind; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..2843ff5a14e 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -806,7 +806,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) metadata.select = SelectQueryDescription::getSelectQueryFromASTForMatView(select, metadata.refresh != nullptr, context); Block as_select_sample; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(select->clone(), context); } diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 63467603d16..b485ab9cbb5 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -125,7 +125,7 @@ void IStorageCluster::read( Block sample_block; ASTPtr query_to_send = query_info.query; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 71b1a0a73c9..c93da7ca512 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -378,7 +378,7 @@ void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Ch QueryPipelineBuilder builder; - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) { auto select_description = buildSelectQueryTreeDescription(select_query_description.inner_query, local_context); if (select_description.dependent_table_node) @@ -475,7 +475,7 @@ Block StorageLiveView::getHeader() const if (!sample_block) { - if (live_view_context->getSettingsRef().allow_experimental_analyzer) + if (live_view_context->getSettingsRef().enable_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(select_query_description.select_query, live_view_context, @@ -519,7 +519,7 @@ ASTPtr StorageLiveView::getInnerBlocksQuery() auto & select_with_union_query = select_query_description.select_query->as(); auto blocks_query = select_with_union_query.list_of_selects->children.at(0)->clone(); - if (!live_view_context->getSettingsRef().allow_experimental_analyzer) + if (!live_view_context->getSettingsRef().enable_analyzer) { /// Rewrite inner query with right aliases for JOIN. /// It cannot be done in constructor or startup() because InterpreterSelectQuery may access table, @@ -543,7 +543,7 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont QueryPipelineBuilder builder; - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(select_query_description.inner_query, local_context, @@ -599,7 +599,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) QueryPipelineBuilder builder; - if (block_context->getSettingsRef().allow_experimental_analyzer) + if (block_context->getSettingsRef().enable_analyzer) { auto select_description = buildSelectQueryTreeDescription(select_query_description.select_query, block_context); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ce27ad24e10..01ef0a409b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7097,7 +7097,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( SelectQueryInfo &) const { /// with new analyzer, Planner make decision regarding parallel replicas usage, and so about processing stage on reading - if (!query_context->getSettingsRef().allow_experimental_analyzer) + if (!query_context->getSettingsRef().enable_analyzer) { const auto & settings = query_context->getSettingsRef(); if (query_context->canUseParallelReplicasCustomKey()) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 6e963066f39..fccb20c2b0a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes namespace { -void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool allow_experimental_analyzer, bool legacy = false) +void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool enable_analyzer, bool legacy = false) { switch (node.type) { @@ -45,18 +45,18 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o /// If it was created from ASTLiteral, then result_name can be an alias. /// We need to convert value back to string here. const auto * column_const = typeid_cast(node.column.get()); - if (column_const && !allow_experimental_analyzer) + if (column_const && !enable_analyzer) writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out); else writeString(node.result_name, out); break; } case ActionsDAG::ActionType::ALIAS: - appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy); break; case ActionsDAG::ActionType::ARRAY_JOIN: writeCString("arrayJoin(", out); - appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy); writeChar(')', out); break; case ActionsDAG::ActionType::FUNCTION: @@ -75,17 +75,17 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o writeCString(", ", out); first = false; - appendColumnNameWithoutAlias(*arg, out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(*arg, out, enable_analyzer, legacy); } writeChar(')', out); } } } -String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool allow_experimental_analyzer, bool legacy = false) +String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool enable_analyzer, bool legacy = false) { WriteBufferFromOwnString out; - appendColumnNameWithoutAlias(node, out, allow_experimental_analyzer, legacy); + appendColumnNameWithoutAlias(node, out, enable_analyzer, legacy); return std::move(out.str()); } @@ -131,7 +131,7 @@ std::string RPNBuilderTreeNode::getColumnName() const if (ast_node) return ast_node->getColumnNameWithoutAlias(); else - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer); } std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const @@ -144,7 +144,7 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const } else { - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer, true /*legacy*/); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer, true /*legacy*/); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 04e6d6676d1..da427ca4a6a 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -397,7 +397,7 @@ void StorageBuffer::read( /// TODO: Find a way to support projections for StorageBuffer if (processed_stage > QueryProcessingStage::FetchColumns) { - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) { auto storage = std::make_shared( getStorageID(), diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3e38ddf830a..f1fe70b4594 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -833,7 +833,7 @@ void StorageDistributed::read( const auto & settings = local_context->getSettingsRef(); - if (settings.allow_experimental_analyzer) + if (settings.enable_analyzer) { StorageID remote_storage_id = StorageID::createEmpty(); if (!remote_table_function_ptr) @@ -1057,7 +1057,7 @@ static std::optional getFilterFromQuery(const ASTPtr & ast, ContextP QueryPlan plan; SelectQueryOptions options; options.only_analyze = true; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast, context, options); plan = std::move(interpreter).extractQueryPlan(); @@ -1611,7 +1611,7 @@ ClusterPtr StorageDistributed::skipUnusedShards( const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) return skipUnusedShardsWithAnalyzer(cluster, query_info, storage_snapshot, local_context); const auto & select = query_info.query->as(); diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 0094723e3fd..27bfa6f854c 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -150,7 +150,7 @@ void StorageExecutable::read( for (auto & input_query : input_queries) { QueryPipelineBuilder builder; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) builder = InterpreterSelectQueryAnalyzer(input_query, context, {}).buildQueryPipeline(); else builder = InterpreterSelectWithUnionQuery(input_query, context, {}).buildQueryPipeline(); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7c268d36a7b..613317b2564 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -590,7 +590,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ auto modified_query_info = getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases); - if (!context->getSettingsRef().allow_experimental_analyzer) + if (!context->getSettingsRef().enable_analyzer) { auto storage_columns = storage_metadata_snapshot->getColumns(); auto syntax_result = TreeRewriter(context).analyzeSelect( @@ -1047,13 +1047,13 @@ void ReadFromMerge::addVirtualColumns( const StorageWithLockAndName & storage_with_lock) const { const auto & [database_name, _, storage, table_name] = storage_with_lock; - bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; + bool enable_analyzer = context->getSettingsRef().enable_analyzer; /// Add virtual columns if we don't already have them. Block plan_header = child.plan.getCurrentDataStream().header; - if (allow_experimental_analyzer) + if (enable_analyzer) { String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); @@ -1133,8 +1133,8 @@ QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( if (!builder->initialized()) return builder; - bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; - if (processed_stage > child.stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) + bool enable_analyzer = context->getSettingsRef().enable_analyzer; + if (processed_stage > child.stage || (enable_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, @@ -1168,7 +1168,7 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( modified_select.setFinal(); } - bool allow_experimental_analyzer = modified_context->getSettingsRef().allow_experimental_analyzer; + bool enable_analyzer = modified_context->getSettingsRef().enable_analyzer; auto storage_stage = storage->getQueryProcessingStage(modified_context, processed_stage, @@ -1201,13 +1201,13 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( row_policy_data_opt->addStorageFilter(source_step_with_filter); } } - else if (processed_stage > storage_stage || allow_experimental_analyzer) + else if (processed_stage > storage_stage || enable_analyzer) { /// Maximum permissible parallelism is streams_num modified_context->setSetting("max_threads", streams_num); modified_context->setSetting("max_streams_to_max_threads_ratio", 1); - if (allow_experimental_analyzer) + if (enable_analyzer) { /// Converting query to AST because types might be different in the source table. /// Need to resolve types again. @@ -1479,7 +1479,7 @@ void ReadFromMerge::convertAndFilterSourceStream( auto storage_sample_block = snapshot->metadata->getSampleBlock(); auto pipe_columns = before_block_header.getNamesAndTypesList(); - if (local_context->getSettingsRef().allow_experimental_analyzer) + if (local_context->getSettingsRef().enable_analyzer) { for (const auto & alias : aliases) { @@ -1522,7 +1522,7 @@ void ReadFromMerge::convertAndFilterSourceStream( ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; - if (local_context->getSettingsRef().allow_experimental_analyzer + if (local_context->getSettingsRef().enable_analyzer && (child.stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f55f672fe5e..ebc88993ee4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -208,7 +208,7 @@ void StorageMergeTree::read( const auto & settings = local_context->getSettingsRef(); /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree - && !settings.allow_experimental_analyzer) + && !settings.enable_analyzer) { ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); @@ -216,7 +216,7 @@ void StorageMergeTree::read( } if (local_context->canUseParallelReplicasCustomKey() && settings.parallel_replicas_for_non_replicated_merge_tree - && !settings.allow_experimental_analyzer && local_context->getClientInfo().distributed_depth == 0) + && !settings.enable_analyzer && local_context->getClientInfo().distributed_depth == 0) { if (auto cluster = local_context->getClusterForParallelReplicas(); local_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) @@ -244,7 +244,7 @@ void StorageMergeTree::read( const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree - && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); + && (!local_context->getSettingsRef().enable_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); if (auto plan = reader.read( column_names, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2d826c6c2df..a3965e7a6d4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5480,13 +5480,13 @@ void StorageReplicatedMergeTree::read( return; } /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here - if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) + if (local_context->canUseParallelReplicasOnInitiator() && !settings.enable_analyzer) { readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); return; } - if (local_context->canUseParallelReplicasCustomKey() && !settings.allow_experimental_analyzer + if (local_context->canUseParallelReplicasCustomKey() && !settings.enable_analyzer && local_context->getClientInfo().distributed_depth == 0) { if (auto cluster = local_context->getClusterForParallelReplicas(); @@ -5555,7 +5555,7 @@ void StorageReplicatedMergeTree::readLocalImpl( const size_t num_streams) { const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() - && (!local_context->getSettingsRef().allow_experimental_analyzer + && (!local_context->getSettingsRef().enable_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); auto plan = reader.read( diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 878998ebf12..dcb5ef2ae77 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -164,7 +164,7 @@ void StorageView::read( auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done); - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { InterpreterSelectQueryAnalyzer interpreter(current_inner_query, getViewContext(context, storage_snapshot), options, column_names); interpreter.addStorageLimits(*query_info.storage_limits); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index d674f054632..16eccfd7343 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -172,7 +172,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType /// with subqueries it's possible that new analyzer will be enabled in ::read method /// of underlying storage when all other parts of infra are not ready for it /// (built with old analyzer). - context_copy->setSetting("allow_experimental_analyzer", false); + context_copy->setSetting("enable_analyzer", false); auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 65bf6768b1b..a2b1704f24b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1197,7 +1197,7 @@ StorageWindowView::StorageWindowView( , fire_signal_timeout_s(context_->getSettingsRef().wait_for_window_view_fire_signal_timeout.totalSeconds()) , clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds()) { - if (context_->getSettingsRef().allow_experimental_analyzer) + if (context_->getSettingsRef().enable_analyzer) disabled_due_to_analyzer = true; if (mode <= LoadingStrictnessLevel::CREATE) @@ -1753,9 +1753,9 @@ StoragePtr StorageWindowView::getTargetTable() const void StorageWindowView::throwIfWindowViewIsDisabled(ContextPtr local_context) const { - if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().allow_experimental_analyzer)) + if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().enable_analyzer)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Experimental WINDOW VIEW feature is not supported " - "in the current infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); + "in the current infrastructure for query analysis (the setting 'enable_analyzer')"); } void registerStorageWindowView(StorageFactory & factory) diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 57501df6d4d..02a278cf590 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -50,7 +50,7 @@ ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context Block sample_block; - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context); else sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context); diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index 935be6c1987..7bae2731525 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -114,7 +114,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const try { - if (context->getSettingsRef().allow_experimental_analyzer) + if (context->getSettingsRef().enable_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context); } diff --git a/tests/config/users.d/analyzer.xml b/tests/config/users.d/analyzer.xml index 4b9764526fa..edba8b8578e 100644 --- a/tests/config/users.d/analyzer.xml +++ b/tests/config/users.d/analyzer.xml @@ -1,7 +1,7 @@ - 0 + 0 diff --git a/tests/integration/helpers/0_common_enable_old_analyzer.xml b/tests/integration/helpers/0_common_enable_old_analyzer.xml index 4b9764526fa..edba8b8578e 100644 --- a/tests/integration/helpers/0_common_enable_old_analyzer.xml +++ b/tests/integration/helpers/0_common_enable_old_analyzer.xml @@ -1,7 +1,7 @@ - 0 + 0 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 673fc07fe94..5e0352df617 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4484,7 +4484,7 @@ class ClickHouseInstance: use_old_analyzer = os.environ.get("CLICKHOUSE_USE_OLD_ANALYZER") is not None # If specific version was used there can be no - # allow_experimental_analyzer setting, so do this only if it was + # enable_analyzer setting, so do this only if it was # explicitly requested. if self.tag: use_old_analyzer = False diff --git a/tests/integration/test_analyzer_compatibility/test.py b/tests/integration/test_analyzer_compatibility/test.py index d4ded420c61..2c840154eb5 100644 --- a/tests/integration/test_analyzer_compatibility/test.py +++ b/tests/integration/test_analyzer_compatibility/test.py @@ -51,7 +51,7 @@ def test_two_new_versions(start_cluster): assert ( current.query( """ -SELECT hostname() AS h, getSetting('allow_experimental_analyzer') +SELECT hostname() AS h, getSetting('enable_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) ORDER BY h;""" ) @@ -62,7 +62,7 @@ ORDER BY h;""" analyzer_enabled = current.query( f""" SELECT -DISTINCT Settings['allow_experimental_analyzer'] +DISTINCT Settings['enable_analyzer'] FROM clusterAllReplicas('test_cluster_mixed', system.query_log) WHERE initial_query_id = '{query_id}';""" ) @@ -81,7 +81,7 @@ WHERE initial_query_id = '{query_id}';""" assert ( backward.query( """ -SELECT hostname() AS h, getSetting('allow_experimental_analyzer') +SELECT hostname() AS h, getSetting('enable_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) ORDER BY h;""" ) @@ -92,7 +92,7 @@ ORDER BY h;""" analyzer_enabled = backward.query( f""" SELECT -DISTINCT Settings['allow_experimental_analyzer'] +DISTINCT Settings['enable_analyzer'] FROM clusterAllReplicas('test_cluster_mixed', system.query_log) WHERE initial_query_id = '{query_id}';""" ) diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index 360087c9dda..e774876bc8b 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -89,7 +89,7 @@ def test_distributed_type_object(started_cluster): assert ( TSV( node1.query( - "SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM dist_table ORDER BY id SETTINGS allow_experimental_analyzer = 0" + "SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM dist_table ORDER BY id SETTINGS enable_analyzer = 0" ) ) == expected diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml index c654074740a..30a0b6276b7 100644 --- a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml @@ -1,7 +1,7 @@ - 1 + 1 1 default 100 diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index e5c0a072ff9..4800ab798bf 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -459,7 +459,7 @@ def test_show_profiles(): query_possible_response = [ "CREATE SETTINGS PROFILE `default`\n", - "CREATE SETTINGS PROFILE `default` SETTINGS allow_experimental_analyzer = true\n", + "CREATE SETTINGS PROFILE `default` SETTINGS enable_analyzer = true\n", ] assert ( instance.query("SHOW CREATE SETTINGS PROFILE default") @@ -470,7 +470,7 @@ def test_show_profiles(): "CREATE SETTINGS PROFILE `default`\n" "CREATE SETTINGS PROFILE `readonly` SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE `xyz`\n", - "CREATE SETTINGS PROFILE `default` SETTINGS allow_experimental_analyzer = true\n" + "CREATE SETTINGS PROFILE `default` SETTINGS enable_analyzer = true\n" "CREATE SETTINGS PROFILE `readonly` SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE `xyz`\n", ] @@ -482,7 +482,7 @@ def test_show_profiles(): "CREATE SETTINGS PROFILE `xyz`\n" ) expected_access_analyzer = ( - "CREATE SETTINGS PROFILE `default` SETTINGS allow_experimental_analyzer = true\n" + "CREATE SETTINGS PROFILE `default` SETTINGS enable_analyzer = true\n" "CREATE SETTINGS PROFILE `readonly` SETTINGS readonly = 1\n" "CREATE SETTINGS PROFILE `xyz`\n" ) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 2fc63c2c926..987500bb4f0 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -15,5 +15,5 @@ SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS - allow_experimental_analyzer=1 - \ No newline at end of file + enable_analyzer=1 + diff --git a/tests/performance/uniq_to_count.xml b/tests/performance/uniq_to_count.xml index 64e4cf1cc0d..57b0085d8fa 100644 --- a/tests/performance/uniq_to_count.xml +++ b/tests/performance/uniq_to_count.xml @@ -3,6 +3,6 @@ select uniq(number) from (select number from numbers(1000000) group by number) - select uniq(number) from (select DISTINCT number from numbers(1000000)) SETTINGS allow_experimental_analyzer=1 - select uniq(number) from (select number from numbers(1000000) group by number) SETTINGS allow_experimental_analyzer=1 + select uniq(number) from (select DISTINCT number from numbers(1000000)) SETTINGS enable_analyzer=1 + select uniq(number) from (select number from numbers(1000000) group by number) SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/00116_storage_set.sql b/tests/queries/0_stateless/00116_storage_set.sql index c156b387c8f..36ad015c69e 100644 --- a/tests/queries/0_stateless/00116_storage_set.sql +++ b/tests/queries/0_stateless/00116_storage_set.sql @@ -28,8 +28,8 @@ RENAME TABLE set2 TO set; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set; create table tab (x String) engine = MergeTree order by x as select 'Hello'; -SELECT * FROM tab PREWHERE x IN (set) WHERE x IN (set) LIMIT 1 settings allow_experimental_analyzer=0; -SELECT * FROM tab PREWHERE x IN (set) WHERE x IN (set) LIMIT 1 settings allow_experimental_analyzer=1; +SELECT * FROM tab PREWHERE x IN (set) WHERE x IN (set) LIMIT 1 settings enable_analyzer=0; +SELECT * FROM tab PREWHERE x IN (set) WHERE x IN (set) LIMIT 1 settings enable_analyzer=1; DROP TABLE tab; DROP TABLE set; diff --git a/tests/queries/0_stateless/00202_cross_join.sql b/tests/queries/0_stateless/00202_cross_join.sql index 8d62c56b3f1..ea327817a58 100644 --- a/tests/queries/0_stateless/00202_cross_join.sql +++ b/tests/queries/0_stateless/00202_cross_join.sql @@ -3,5 +3,6 @@ SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN SET join_algorithm = 'auto'; SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; -SET allow_experimental_analyzer = 1; +-- Just to test that we preserved old setting name this we use `enable_analyzer` instead of `enable_analyzer` here. +SET enable_analyzer = 1; SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql index e3634141613..b72162be49a 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql @@ -1,6 +1,6 @@ -- Tags: shard -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set enable_positional_arguments = 0; select 40 as z from (select * from system.numbers limit 3) group by z; diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.sh b/tests/queries/0_stateless/00313_const_totals_extremes.sh index 539a19817e2..6267d5066dd 100755 --- a/tests/queries/0_stateless/00313_const_totals_extremes.sh +++ b/tests/queries/0_stateless/00313_const_totals_extremes.sh @@ -4,10 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1 FORMAT JSON"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1 FORMAT JSONCompact"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03', 'UTC') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03', 'UTC') AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1 FORMAT JSON"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS SETTINGS enable_analyzer = 1 FORMAT JSONCompact"; diff --git a/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql index 78a58a979d1..a3c499f1688 100644 --- a/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql +++ b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql @@ -1,8 +1,8 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- https://github.com/ClickHouse/ClickHouse/issues/45804 -CREATE TABLE myRMT( +CREATE TABLE myRMT( key Int64, someCol String, ver DateTime diff --git a/tests/queries/0_stateless/00370_duplicate_columns_in_subqueries.sql b/tests/queries/0_stateless/00370_duplicate_columns_in_subqueries.sql index 118e50c35e0..8a93c0a9d26 100644 --- a/tests/queries/0_stateless/00370_duplicate_columns_in_subqueries.sql +++ b/tests/queries/0_stateless/00370_duplicate_columns_in_subqueries.sql @@ -1,6 +1,6 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select x, y from (select 1 as x, 2 as y, x, y); select x, y from (select 1 as x, 1 as y, x, y); diff --git a/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql b/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql index e7b59bc3f7f..941d5ecd135 100644 --- a/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql +++ b/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql @@ -2,7 +2,7 @@ SET output_format_write_statistics = 0; SET extremes = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET output_format_json_quote_64bit_integers = 1; SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSON; diff --git a/tests/queries/0_stateless/00445_join_nullable_keys.sql b/tests/queries/0_stateless/00445_join_nullable_keys.sql index 774594f90f3..bec0c76eb5f 100644 --- a/tests/queries/0_stateless/00445_join_nullable_keys.sql +++ b/tests/queries/0_stateless/00445_join_nullable_keys.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET join_use_nulls = 0; SET any_join_distinct_right_table_keys = 1; diff --git a/tests/queries/0_stateless/00490_with_select.sql b/tests/queries/0_stateless/00490_with_select.sql index c803cf1d3ad..d63f0ca4f31 100644 --- a/tests/queries/0_stateless/00490_with_select.sql +++ b/tests/queries/0_stateless/00490_with_select.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; with pow(2,2) as four select pow(four, 2), 2 as two, pow(two, 2); select `pow(four, 2)`, `pow(two, 2)` from (with pow(2,2) as four select pow(four, 2), 2 as two, pow(two, 2)); diff --git a/tests/queries/0_stateless/00597_push_down_predicate_long.reference b/tests/queries/0_stateless/00597_push_down_predicate_long.reference index 2c46edc98bf..55b7cdd3c64 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate_long.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate_long.reference @@ -390,7 +390,7 @@ ANY LEFT JOIN ) USING (id) WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -------- allow_experimental_analyzer=1 ------- +------- enable_analyzer=1 ------- 1 2000-01-01 test string 1 1 2000-01-01 test string 1 1 SELECT id, @@ -454,7 +454,7 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -------- allow_experimental_analyzer=1 ------- +------- enable_analyzer=1 ------- 1 2000-01-01 test string 1 1 2000-01-01 test string 1 1 SELECT date, @@ -484,7 +484,7 @@ ANY LEFT JOIN ) AS b USING (id) WHERE b.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -------- allow_experimental_analyzer=1 ------- +------- enable_analyzer=1 ------- 1 2000-01-01 test string 1 1 2000-01-01 test string 1 1 SELECT id, @@ -510,7 +510,7 @@ ANY LEFT JOIN ) AS b USING (date, id) WHERE b.date = toDate(\'2000-01-01\') 1 2000-01-01 test string 1 1 -------- allow_experimental_analyzer=1 ------- +------- enable_analyzer=1 ------- 2000-01-01 1 test string 1 1 SELECT date, @@ -593,7 +593,7 @@ SEMI LEFT JOIN ) AS r USING (id) WHERE r.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -------- allow_experimental_analyzer=1 ------- +------- enable_analyzer=1 ------- 1 2000-01-01 test string 1 1 2000-01-01 test string 1 1 SELECT value + t1.value AS expr FROM diff --git a/tests/queries/0_stateless/00597_push_down_predicate_long.sql b/tests/queries/0_stateless/00597_push_down_predicate_long.sql index caf6edd7372..f79b24abe56 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate_long.sql +++ b/tests/queries/0_stateless/00597_push_down_predicate_long.sql @@ -110,9 +110,9 @@ SELECT * FROM (SELECT * FROM test_00597 UNION ALL SELECT * FROM test_00597) WHER -- Optimize predicate expression with join query EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1; -SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1 SETTINGS allow_experimental_analyzer=0; -SELECT '------- allow_experimental_analyzer=1 -------'; -SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1 SETTINGS enable_analyzer=0; +SELECT '------- enable_analyzer=1 -------'; +SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 USING id WHERE value = 1; SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 USING id WHERE value = 1; @@ -123,30 +123,30 @@ SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 AS b USING -- Optimize predicate expression with join and nested subquery EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1; -SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1 SETTINGS allow_experimental_analyzer=0; -SELECT '------- allow_experimental_analyzer=1 -------'; -SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1 SETTINGS enable_analyzer=0; +SELECT '------- enable_analyzer=1 -------'; +SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1 SETTINGS enable_analyzer=1; -- Optimize predicate expression with join query and qualified EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1; -SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1 SETTINGS allow_experimental_analyzer=0; -SELECT '------- allow_experimental_analyzer=1 -------'; -SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1 SETTINGS enable_analyzer=0; +SELECT '------- enable_analyzer=1 -------'; +SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1 SETTINGS enable_analyzer=1; -- Compatibility test EXPLAIN SYNTAX SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01'); -SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01') SETTINGS allow_experimental_analyzer=0; -SELECT '------- allow_experimental_analyzer=1 -------'; -SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01') SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01') SETTINGS enable_analyzer=0; +SELECT '------- enable_analyzer=1 -------'; +SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01') SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a ANY LEFT JOIN (SELECT * FROM test_00597) AS b ON a.id = b.id) WHERE id = 1; SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a ANY LEFT JOIN (SELECT * FROM test_00597) AS b ON a.id = b.id) WHERE id = 1; -- Explain with join subquery EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1; -SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1 SETTINGS allow_experimental_analyzer=0; -SELECT '------- allow_experimental_analyzer=1 -------'; -SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1 SETTINGS enable_analyzer=0; +SELECT '------- enable_analyzer=1 -------'; +SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1 SETTINGS enable_analyzer=1; -- issue 20497 EXPLAIN SYNTAX SELECT value + t1.value AS expr FROM (SELECT t0.value, t1.value FROM test_00597 AS t0 FULL JOIN test_00597 AS t1 USING date) WHERE expr < 3; diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index b68f550a742..ab8a1499f6d 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -17,7 +17,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: g, result_type: String, source_id: 3 CONSTANT id: 7, constant_value: Tuple_(\'5\', \'6\'), constant_value_type: Tuple(String, String) - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 2 2 QUERY id: 0 @@ -42,4 +42,4 @@ QUERY id: 0 LIST id: 11, nodes: 2 COLUMN id: 8, column_name: g, result_type: String, source_id: 3 CONSTANT id: 12, constant_value: \'6\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.sql b/tests/queries/0_stateless/00621_regression_for_in_operator.sql index db1bcb4a39a..0d8c4933c65 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.sql +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.sql @@ -12,13 +12,13 @@ SELECT count() FROM regression_for_in_operator_view WHERE g IN ('5','6'); SET optimize_min_equality_disjunction_chain_length = 1; SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6'; -SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; -EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; +SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS enable_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS enable_analyzer = 1; SET optimize_min_equality_disjunction_chain_length = 3; SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6'; -SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; -EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; +SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS enable_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS enable_analyzer = 1; DROP TABLE regression_for_in_operator_view; DROP TABLE regression_for_in_operator; diff --git a/tests/queries/0_stateless/00674_join_on_syntax.sql b/tests/queries/0_stateless/00674_join_on_syntax.sql index 9ff26db1536..584e43b88bf 100644 --- a/tests/queries/0_stateless/00674_join_on_syntax.sql +++ b/tests/queries/0_stateless/00674_join_on_syntax.sql @@ -1,5 +1,5 @@ SET joined_subquery_requires_alias = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; drop table if exists tab1; drop table if exists tab2; diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index beadbdade16..1b901e04c28 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -27,7 +27,7 @@ SELECT a > 0, b > 0, g > 0 FROM decimal ORDER BY a DESC; SELECT a, g > toInt8(0), g > toInt16(0), g > toInt32(0), g > toInt64(0) FROM decimal ORDER BY a; SELECT a, g > toUInt8(0), g > toUInt16(0), g > toUInt32(0), g > toUInt64(0) FROM decimal ORDER BY a; SELECT a, b, g FROM decimal WHERE a IN(42) AND b IN(42) AND g IN(42); -SELECT a, b, g FROM decimal WHERE a IN(42) AND b IN(42) AND g IN(42) SETTINGS allow_experimental_analyzer = 1; +SELECT a, b, g FROM decimal WHERE a IN(42) AND b IN(42) AND g IN(42) SETTINGS enable_analyzer = 1; SELECT a, b, g FROM decimal WHERE a > 0 AND a <= 42 AND b <= 42 AND g <= 42; SELECT d, e, f from decimal WHERE d > 0 AND d < 1 AND e > 0 AND e < 1 AND f > 0 AND f < 1; diff --git a/tests/queries/0_stateless/00722_inner_join.sql b/tests/queries/0_stateless/00722_inner_join.sql index 0d5a543b99d..aa590f470ae 100644 --- a/tests/queries/0_stateless/00722_inner_join.sql +++ b/tests/queries/0_stateless/00722_inner_join.sql @@ -1,6 +1,6 @@ -- Tags: no-parallel -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS one; CREATE TABLE one(dummy UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index f28dcacef0e..4e7f2e09dea 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -49,7 +49,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 14, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 1 21 1 22 1 23 @@ -99,7 +99,7 @@ QUERY id: 0 LIST id: 14, nodes: 2 COLUMN id: 15, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 1 1 23 0 0 1 @@ -152,7 +152,7 @@ QUERY id: 0 CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, alias: __table1, table_name: default.bug - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 21 1 22 1 23 1 @@ -185,7 +185,7 @@ QUERY id: 0 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, alias: __table1, table_name: default.bug - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 1 21 1 22 1 23 @@ -237,7 +237,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 14, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 1 21 1 22 1 23 @@ -287,7 +287,7 @@ QUERY id: 0 LIST id: 14, nodes: 2 COLUMN id: 15, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 1 1 23 0 0 1 @@ -348,7 +348,7 @@ QUERY id: 0 CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, alias: __table1, table_name: default.bug - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 21 1 22 1 23 1 @@ -381,7 +381,7 @@ QUERY id: 0 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, alias: __table1, table_name: default.bug - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 21 1 22 1 23 1 @@ -414,4 +414,4 @@ QUERY id: 0 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, alias: __table1, table_name: default.bug - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.sql b/tests/queries/0_stateless/00736_disjunction_optimisation.sql index e5bfc81f7ae..38f77622bd4 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.sql +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.sql @@ -7,43 +7,43 @@ set optimize_min_equality_disjunction_chain_length = 2; select * from bug; select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); -select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1; -explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1; +explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); -select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; -explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; +explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug; -select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; -explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; +explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; select s, (s=21 or s=22 or s=23) from bug; -select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; -explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +select s, (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; +explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; set optimize_min_equality_disjunction_chain_length = 3; select * from bug; select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); -select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1; -explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1; +explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); -select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; -explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; +explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS enable_analyzer = 1;; select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug; -select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; -explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; +explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; select s, (s=21 or s=22 or s=23) from bug; -select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; -explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +select s, (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; +explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS enable_analyzer = 1;; select s, (s=21 or 22=s or 23=s) from bug; -select s, (s=21 or 22=s or 23=s) from bug SETTINGS allow_experimental_analyzer = 1;; -explain query tree select s, (s=21 or 22=s or 23=s) from bug SETTINGS allow_experimental_analyzer = 1;; +select s, (s=21 or 22=s or 23=s) from bug SETTINGS enable_analyzer = 1;; +explain query tree select s, (s=21 or 22=s or 23=s) from bug SETTINGS enable_analyzer = 1;; DROP TABLE bug; diff --git a/tests/queries/0_stateless/00757_enum_defaults_const.sql b/tests/queries/0_stateless/00757_enum_defaults_const.sql index 64271a37473..048c9dee88f 100644 --- a/tests/queries/0_stateless/00757_enum_defaults_const.sql +++ b/tests/queries/0_stateless/00757_enum_defaults_const.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer=0; +SET enable_analyzer=0; select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; diff --git a/tests/queries/0_stateless/00757_enum_defaults_const_analyzer.sql b/tests/queries/0_stateless/00757_enum_defaults_const_analyzer.sql index bf079539019..c202ed630db 100644 --- a/tests/queries/0_stateless/00757_enum_defaults_const_analyzer.sql +++ b/tests/queries/0_stateless/00757_enum_defaults_const_analyzer.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; diff --git a/tests/queries/0_stateless/00800_low_cardinality_join.sql b/tests/queries/0_stateless/00800_low_cardinality_join.sql index ecb5194253c..fc5f5d1860c 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set joined_subquery_requires_alias = 0; select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val; diff --git a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 index 8e2037480c7..a5f5e9b33b8 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 +++ b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; {% for join_algorithm in ['partial_merge', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/00818_alias_bug_4110.sql b/tests/queries/0_stateless/00818_alias_bug_4110.sql index d057bacc908..1242a3b605d 100644 --- a/tests/queries/0_stateless/00818_alias_bug_4110.sql +++ b/tests/queries/0_stateless/00818_alias_bug_4110.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select s.a as a, s.a + 1 as b from (select 10 as a) s; select s.a + 1 as a, s.a as b from (select 10 as a) s; diff --git a/tests/queries/0_stateless/00818_inner_join_bug_3567.sql b/tests/queries/0_stateless/00818_inner_join_bug_3567.sql index 2dec5ce3221..3b4b3cd77d3 100644 --- a/tests/queries/0_stateless/00818_inner_join_bug_3567.sql +++ b/tests/queries/0_stateless/00818_inner_join_bug_3567.sql @@ -1,5 +1,5 @@ SET output_format_pretty_color = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql index 3c0246619da..8b3d3ec1bc1 100644 --- a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql +++ b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql @@ -1,6 +1,6 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM (SELECT 1 AS a, 'x' AS b) join (SELECT 1 as a, 'y' as b) using a; SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a; diff --git a/tests/queries/0_stateless/00820_multiple_joins.sql b/tests/queries/0_stateless/00820_multiple_joins.sql index 5c7a7bebb0b..b4197570cfa 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.sql +++ b/tests/queries/0_stateless/00820_multiple_joins.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql index 3da2cad4eff..538e6967ff3 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00830_join_overwrite.sql b/tests/queries/0_stateless/00830_join_overwrite.sql index bc3662528db..f51a152ea46 100644 --- a/tests/queries/0_stateless/00830_join_overwrite.sql +++ b/tests/queries/0_stateless/00830_join_overwrite.sql @@ -13,7 +13,7 @@ SELECT joinGet('kv_overwrite', 'v', toUInt32(1)); CREATE TABLE t2 (k UInt32, v UInt32) ENGINE = Memory; INSERT INTO t2 VALUES (1, 2), (1, 3); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT v FROM (SELECT 1 as k) t1 ANY INNER JOIN t2 USING (k) SETTINGS join_any_take_last_row = 0; SELECT v FROM (SELECT 1 as k) t1 ANY INNER JOIN t2 USING (k) SETTINGS join_any_take_last_row = 1; diff --git a/tests/queries/0_stateless/00835_if_generic_case.sql b/tests/queries/0_stateless/00835_if_generic_case.sql index 051fad14603..e06e610dbb5 100644 --- a/tests/queries/0_stateless/00835_if_generic_case.sql +++ b/tests/queries/0_stateless/00835_if_generic_case.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toDateTime('2000-01-01 00:00:00', 'Asia/Istanbul') AS x, toDate('2000-01-02') AS y, x > y ? x : y AS z; SELECT materialize(toDateTime('2000-01-01 00:00:00', 'Asia/Istanbul')) AS x, toDate('2000-01-02') AS y, x > y ? x : y AS z; diff --git a/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql b/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql index 2f6cca0284c..275968236ae 100644 --- a/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql +++ b/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql @@ -1,5 +1,5 @@ SET any_join_distinct_right_table_keys = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t1_00848; DROP TABLE IF EXISTS t2_00848; diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index 6530f691087..250dd4a47ab 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -12,7 +12,7 @@ CREATE TABLE t2 (a UInt32, b Nullable(Int32)) ENGINE = Memory; CREATE TABLE t3 (a UInt32, b Nullable(Int32)) ENGINE = Memory; CREATE TABLE t4 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; --- EXPLAIN SYNTAX (old AST based optimization) SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( @@ -62,56 +62,56 @@ SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explai --- EXPLAIN QUERY TREE SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2, t3, t4) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1, t2 CROSS JOIN t3) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3) SETTINGS enable_analyzer = 1; SELECT countIf(explain like '%COMMA%' OR explain like '%CROSS%'), countIf(explain like '%INNER%') FROM ( - EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3) SETTINGS allow_experimental_analyzer = 1; + EXPLAIN QUERY TREE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3) SETTINGS enable_analyzer = 1; INSERT INTO t1 values (1,1), (2,2), (3,3), (4,4); INSERT INTO t2 values (1,1), (1, Null); INSERT INTO t3 values (1,1), (1, Null); INSERT INTO t4 values (1,1), (1, Null); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 'SELECT * FROM t1, t2'; SELECT * FROM t1, t2 diff --git a/tests/queries/0_stateless/00855_join_with_array_join.sql b/tests/queries/0_stateless/00855_join_with_array_join.sql index c278ff0738a..2d5bc6cb1f6 100644 --- a/tests/queries/0_stateless/00855_join_with_array_join.sql +++ b/tests/queries/0_stateless/00855_join_with_array_join.sql @@ -1,5 +1,5 @@ SET joined_subquery_requires_alias = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c); SELECT ax, c FROM (SELECT [3,4] ax, 0 c) JOIN (SELECT 0 c) USING (c) ARRAY JOIN ax; diff --git a/tests/queries/0_stateless/00858_issue_4756.sql b/tests/queries/0_stateless/00858_issue_4756.sql index 9eacd5ef364..3f6ab037c02 100644 --- a/tests/queries/0_stateless/00858_issue_4756.sql +++ b/tests/queries/0_stateless/00858_issue_4756.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set distributed_product_mode = 'local'; drop table if exists shard1; diff --git a/tests/queries/0_stateless/00897_flatten.sql b/tests/queries/0_stateless/00897_flatten.sql index 0d67a1708fd..45d1a225a49 100644 --- a/tests/queries/0_stateless/00897_flatten.sql +++ b/tests/queries/0_stateless/00897_flatten.sql @@ -2,7 +2,7 @@ SELECT flatten(arrayJoin([[[1, 2, 3], [4, 5]], [[6], [7, 8]]])); SELECT arrayFlatten(arrayJoin([[[[]], [[1], [], [2, 3]]], [[[4]]]])); SELECT flatten(arrayMap(x -> arrayMap(y -> arrayMap(z -> range(x), range(x)), range(x)), range(number))) FROM numbers(6); SELECT flatten(arrayMap(x -> arrayMap(y -> arrayMap(z -> range(z), range(y)), range(x)), range(number))) FROM numbers(6); -SELECT flatten(arrayMap(x -> arrayMap(x -> arrayMap(x -> range(x), range(x)), range(x)), range(number))) FROM numbers(6) SETTINGS allow_experimental_analyzer=1; +SELECT flatten(arrayMap(x -> arrayMap(x -> arrayMap(x -> range(x), range(x)), range(x)), range(number))) FROM numbers(6) SETTINGS enable_analyzer=1; SELECT arrayFlatten([[[1, 2, 3], [4, 5]], [[6], [7, 8]]]); SELECT flatten([[[]]]); SELECT arrayFlatten([]); diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql index 532539206f7..0421cadf868 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql @@ -1,4 +1,4 @@ -SET optimize_read_in_order = 1, query_plan_read_in_order = 1, allow_experimental_analyzer = 0; +SET optimize_read_in_order = 1, query_plan_read_in_order = 1, enable_analyzer = 0; drop table if exists tab; drop table if exists tab2; diff --git a/tests/queries/0_stateless/01000_subquery_requires_alias.sql b/tests/queries/0_stateless/01000_subquery_requires_alias.sql index 3cd522a8389..38ba1798dc1 100644 --- a/tests/queries/0_stateless/01000_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/01000_subquery_requires_alias.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET joined_subquery_requires_alias = 1; SELECT * FROM (SELECT 1 as A, 2 as B) X diff --git a/tests/queries/0_stateless/01013_totals_without_aggregation.sql b/tests/queries/0_stateless/01013_totals_without_aggregation.sql index ab656cd92b5..08be45754bb 100644 --- a/tests/queries/0_stateless/01013_totals_without_aggregation.sql +++ b/tests/queries/0_stateless/01013_totals_without_aggregation.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 11 AS n GROUP BY n WITH TOTALS; SELECT 12 AS n GROUP BY n WITH ROLLUP; diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index e9e754ed7a8..b2e0e8fc522 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -1,5 +1,5 @@ SET output_format_pretty_color=1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select * from system.one cross join system.one; select * from system.one cross join system.one r; diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index f17f3ac63b0..de436f1b28e 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -1,5 +1,5 @@ SET send_logs_level = 'fatal'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index adcb4a6364d..bcd80e37c00 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel SET send_logs_level = 'fatal'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01048; set allow_deprecated_database_ordinary=1; diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 index 2ebe5c373b2..872bb448027 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 @@ -1,5 +1,5 @@ -- { echoOn } -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; {% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; SET join_use_nulls = 0; @@ -19,17 +19,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -49,17 +49,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -79,17 +79,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l -- @@ -109,17 +109,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -313,7 +313,7 @@ SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; {% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; SET join_use_nulls = 0; @@ -333,17 +333,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -363,17 +363,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String String String String String str_l str_l -- @@ -393,17 +393,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String String String String String str_l str_l -- @@ -423,13 +423,13 @@ str_r Nullable(String) str_l Nullable(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 index 64ec34ef1bf..7e7b5cb1fed 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 @@ -23,9 +23,9 @@ INSERT INTO nl VALUES (0, 'str'), (2, 'str_l'); INSERT INTO l_lc VALUES (0, 'str'), (2, 'str_l'); -- { echoOn } -{% for allow_experimental_analyzer in [0, 1] -%} +{% for enable_analyzer in [0, 1] -%} -SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; +SET enable_analyzer = {{ enable_analyzer }}; {% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index c52a6fefacb..9c4d312bab6 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index b37e4ed3095..569e3b2a6bf 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index 4f8482558c8..c473bf766b0 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=0" + "--enable_analyzer=0" ) $CLICKHOUSE_CLIENT "${opts[@]}" < 3465735.3 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k SETTINGS enable_analyzer=1; 3465735.9028 3465735.9028 3465736.595947 @@ -11,7 +11,7 @@ SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY numbe 3465734.169932 3465734.863079 3465735.556226 -SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS enable_analyzer=1; 3465734.169932 3465734.863079 3465735.556226 @@ -19,7 +19,7 @@ SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 0 1 4 -SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS enable_analyzer=1; 0 1 4 diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql index 203e8322ad9..a73b122a9f9 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql @@ -1,16 +1,16 @@ set optimize_group_by_function_keys = 1; set optimize_syntax_fuse_functions = 0; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; -- { echoOn } SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k; -SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 ORDER BY k SETTINGS enable_analyzer=1; SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; -SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT round(avg(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS enable_analyzer=1; SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k; -SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS allow_experimental_analyzer=1; +SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) GROUP BY number % 5, ((number % 5) * (number % 5)) HAVING ((number % 5) * (number % 5)) < 5 ORDER BY k SETTINGS enable_analyzer=1; -- { echoOff } diff --git a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference index d47f12ff4d1..c2c37cc4de6 100644 --- a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference @@ -65,7 +65,7 @@ QUERY id: 0 SORT id: 12, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT groupArray(x) FROM ( @@ -99,7 +99,7 @@ QUERY id: 0 SORT id: 12, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT groupArray(x) FROM ( @@ -141,7 +141,7 @@ QUERY id: 0 SORT id: 15, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT key, a, @@ -203,7 +203,7 @@ QUERY id: 0 SORT id: 25, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 26, column_name: key, result_type: UInt64, source_id: 5 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT key, a @@ -229,7 +229,7 @@ QUERY id: 0 SORT id: 7, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT key, a @@ -262,7 +262,7 @@ QUERY id: 0 LIST id: 11, nodes: 2 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 QUERY id: 0 PROJECTION COLUMNS key UInt64 @@ -285,7 +285,7 @@ QUERY id: 0 SORT id: 10, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 QUERY id: 0 PROJECTION COLUMNS t1.id UInt64 @@ -314,7 +314,7 @@ QUERY id: 0 SORT id: 14, sort_direction: ASCENDING, with_fill: 0 EXPRESSION COLUMN id: 15, column_name: id, result_type: UInt64, source_id: 5 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 [0,1,2] [0,1,2] [0,1,2] diff --git a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.sql b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.sql index 738ad581e3d..fb1eed1666e 100644 --- a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.sql +++ b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.sql @@ -8,37 +8,37 @@ INSERT INTO test SELECT number, number, toString(number), number from numbers(4) set optimize_redundant_functions_in_order_by = 1; SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)); -SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)) SETTINGS allow_experimental_analyzer=1; +SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)) SETTINGS enable_analyzer=1; SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))); -SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))) SETTINGS allow_experimental_analyzer=1; +SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))) SETTINGS enable_analyzer=1; SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x); -SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x) SETTINGS allow_experimental_analyzer=1; +SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x) SETTINGS enable_analyzer=1; SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key; -SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key SETTINGS allow_experimental_analyzer=1; +SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key SETTINGS enable_analyzer=1; SELECT key, a FROM test ORDER BY key, a, exp(key + a); -SELECT key, a FROM test ORDER BY key, a, exp(key + a) SETTINGS allow_experimental_analyzer=1; +SELECT key, a FROM test ORDER BY key, a, exp(key + a) SETTINGS enable_analyzer=1; SELECT key, a FROM test ORDER BY key, exp(key + a); -SELECT key, a FROM test ORDER BY key, exp(key + a) SETTINGS allow_experimental_analyzer=1; +SELECT key, a FROM test ORDER BY key, exp(key + a) SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)); -EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)) settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(x)) settings enable_analyzer=1; EXPLAIN SYNTAX SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))); -EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))) settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY x, exp(exp(x))) settings enable_analyzer=1; EXPLAIN SYNTAX SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x); -EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x) settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT groupArray(x) from (SELECT number as x FROM numbers(3) ORDER BY exp(x), x) settings enable_analyzer=1; EXPLAIN SYNTAX SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key; -EXPLAIN QUERY TREE run_passes=1 SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s FULL JOIN test t USING(key) ORDER BY s.key, t.key settings enable_analyzer=1; EXPLAIN SYNTAX SELECT key, a FROM test ORDER BY key, a, exp(key + a); -EXPLAIN QUERY TREE run_passes=1 SELECT key, a FROM test ORDER BY key, a, exp(key + a) settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT key, a FROM test ORDER BY key, a, exp(key + a) settings enable_analyzer=1; EXPLAIN SYNTAX SELECT key, a FROM test ORDER BY key, exp(key + a); -EXPLAIN QUERY TREE run_passes=1 SELECT key, a FROM test ORDER BY key, exp(key + a) settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE run_passes=1 SELECT key FROM test GROUP BY key ORDER BY avg(a), key settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT key, a FROM test ORDER BY key, exp(key + a) settings enable_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT key FROM test GROUP BY key ORDER BY avg(a), key settings enable_analyzer=1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; CREATE TABLE t1 (id UInt64) ENGINE = MergeTree() ORDER BY id; CREATE TABLE t2 (id UInt64) ENGINE = MergeTree() ORDER BY id; -EXPLAIN QUERY TREE run_passes=1 SELECT * FROM t1 INNER JOIN t2 ON t1.id = t2.id ORDER BY t1.id, t2.id settings allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT * FROM t1 INNER JOIN t2 ON t1.id = t2.id ORDER BY t1.id, t2.id settings enable_analyzer=1; set optimize_redundant_functions_in_order_by = 0; diff --git a/tests/queries/0_stateless/01353_low_cardinality_join_types.sql b/tests/queries/0_stateless/01353_low_cardinality_join_types.sql index 93953f1d74a..2aa42f33fd3 100644 --- a/tests/queries/0_stateless/01353_low_cardinality_join_types.sql +++ b/tests/queries/0_stateless/01353_low_cardinality_join_types.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; set join_algorithm = 'hash'; select '-'; @@ -75,7 +75,7 @@ from (select toLowCardinality(number) k, toLowCardinality(toString(number)) s fr full join (select toLowCardinality(number+1) k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 using k order by js1.k, js2.k; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; set join_algorithm = 'hash'; select '-'; diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference index 6de0a5be0a5..2d16e71f099 100644 --- a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference @@ -21,4 +21,4 @@ QUERY id: 0 GROUP BY LIST id: 10, nodes: 1 COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql index a868b38b4d7..08ca9ed3c2d 100644 --- a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql @@ -38,7 +38,7 @@ EXPLAIN QUERY TREE SELECT dictGet('dictdb_01376.dict_exists', 'value', number) as val FROM numbers(2) GROUP BY val -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; DROP DICTIONARY dictdb_01376.dict_exists; DROP TABLE dictdb_01376.table_for_dict; diff --git a/tests/queries/0_stateless/01428_nullable_asof_join.sql b/tests/queries/0_stateless/01428_nullable_asof_join.sql index f07a26edd97..41f6ba8a03d 100644 --- a/tests/queries/0_stateless/01428_nullable_asof_join.sql +++ b/tests/queries/0_stateless/01428_nullable_asof_join.sql @@ -18,13 +18,13 @@ SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(ma FROM (SELECT toUInt8(number) > 0 as pk, toUInt8(number) as dt FROM numbers(3)) a ASOF LEFT JOIN (SELECT 1 as pk, toNullable(0) as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 0; +ORDER BY a.dt SETTINGS enable_analyzer = 0; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toUInt8(number) as dt FROM numbers(3)) a ASOF LEFT JOIN (SELECT 1 as pk, toNullable(0) as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 1; +ORDER BY a.dt SETTINGS enable_analyzer = 1; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toNullable(toUInt8(number)) as dt FROM numbers(3)) a @@ -70,25 +70,25 @@ SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(ma FROM (SELECT toUInt8(number) > 0 as pk, toNullable(toUInt8(number)) as dt FROM numbers(3)) a ASOF JOIN (SELECT 1 as pk, 2 as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 0; +ORDER BY a.dt SETTINGS enable_analyzer = 0; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toNullable(toUInt8(number)) as dt FROM numbers(3)) a ASOF JOIN (SELECT 1 as pk, 2 as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 1; +ORDER BY a.dt SETTINGS enable_analyzer = 1; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toUInt8(number) as dt FROM numbers(3)) a ASOF JOIN (SELECT 1 as pk, toNullable(0) as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 0; +ORDER BY a.dt SETTINGS enable_analyzer = 0; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toUInt8(number) as dt FROM numbers(3)) a ASOF JOIN (SELECT 1 as pk, toNullable(0) as dt) b USING(pk, dt) -ORDER BY a.dt SETTINGS allow_experimental_analyzer = 1; +ORDER BY a.dt SETTINGS enable_analyzer = 1; SELECT a.pk, b.pk, a.dt, b.dt, toTypeName(a.pk), toTypeName(b.pk), toTypeName(materialize(a.dt)), toTypeName(materialize(b.dt)) FROM (SELECT toUInt8(number) > 0 as pk, toNullable(toUInt8(number)) as dt FROM numbers(3)) a diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index 2920b387aa2..2b08db1f27f 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -1,10 +1,10 @@ ===http=== -{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} +{"query":"select 1 from remote('127.0.0.2', system, one) settings enable_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} -{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} +{"query":"select 1 from remote('127.0.0.2', system, one) settings enable_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} {"initial query spans with proper parent":"2"} {"unique non-empty tracestate values":"1"} diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh index e19810dd7f1..2b6da6132ed 100755 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function check_log { ${CLICKHOUSE_CLIENT} --format=JSONEachRow -nq " -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; system flush logs; -- Show queries sorted by start time. @@ -75,7 +75,7 @@ select uniqExact(value) "'"'"unique non-empty tracestate values"'"'" # Generate some random trace id so that the prevous runs of the test do not interfere. echo "===http===" -trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4())))) settings allow_experimental_analyzer = 1") +trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4())))) settings enable_analyzer = 1") # Check that the HTTP traceparent is read, and then passed through `remote` # table function. We expect 4 queries -- one initial, one SELECT and two @@ -85,7 +85,7 @@ ${CLICKHOUSE_CURL} \ --header "traceparent: 00-$trace_id-0000000000000073-01" \ --header "tracestate: some custom state" "$CLICKHOUSE_URL" \ --get \ - --data-urlencode "query=select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null" + --data-urlencode "query=select 1 from remote('127.0.0.2', system, one) settings enable_analyzer = 1 format Null" check_log diff --git a/tests/queries/0_stateless/01476_right_full_join_switch.sql b/tests/queries/0_stateless/01476_right_full_join_switch.sql index dfbdec47e1f..7c8c3157844 100644 --- a/tests/queries/0_stateless/01476_right_full_join_switch.sql +++ b/tests/queries/0_stateless/01476_right_full_join_switch.sql @@ -13,7 +13,7 @@ INSERT INTO nr VALUES (2, NULL); SET join_use_nulls = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- t.x is supertupe for `x` from left and right since `x` is inside `USING`. SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x; @@ -28,7 +28,7 @@ SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t SELECT '-'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; -- t.x is supertupe for `x` from left and right since `x` is inside `USING`. SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x; diff --git a/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql.j2 b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql.j2 index 6eafd41b411..b43db222e43 100644 --- a/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql.j2 +++ b/tests/queries/0_stateless/01477_lc_in_merge_join_left_key.sql.j2 @@ -10,11 +10,11 @@ CREATE TABLE nr (`x` Nullable(UInt32), `s` Nullable(String)) ENGINE = Memory; INSERT INTO t VALUES (1, 'l'); INSERT INTO nr VALUES (2, NULL); -{% for allow_experimental_analyzer in [0, 1] -%} +{% for enable_analyzer in [0, 1] -%} -SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; +SET enable_analyzer = {{ enable_analyzer }}; -{% if allow_experimental_analyzer -%} +{% if enable_analyzer -%} SELECT '- analyzer -'; {% endif -%} diff --git a/tests/queries/0_stateless/01479_cross_join_9855.sql b/tests/queries/0_stateless/01479_cross_join_9855.sql index 9dcf209a1cd..19cd0ab18fd 100644 --- a/tests/queries/0_stateless/01479_cross_join_9855.sql +++ b/tests/queries/0_stateless/01479_cross_join_9855.sql @@ -2,8 +2,8 @@ SET cross_to_inner_join_rewrite = 1; SELECT count() FROM numbers(4) AS n1, numbers(3) AS n2 -WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS allow_experimental_analyzer=0; +WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS enable_analyzer=0; SELECT count() FROM numbers(4) AS n1, numbers(3) AS n2, numbers(6) AS n3 -WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS allow_experimental_analyzer=0; +WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS enable_analyzer=0; diff --git a/tests/queries/0_stateless/01508_explain_header.sql b/tests/queries/0_stateless/01508_explain_header.sql index a9f876068aa..03452e4bdac 100644 --- a/tests/queries/0_stateless/01508_explain_header.sql +++ b/tests/queries/0_stateless/01508_explain_header.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; explain header = 1 select 1 as x; diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql index bbd96ef5c69..d8278e1887a 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET union_default_mode = 'DISTINCT'; set enable_global_with_statement = 1; diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 2631199cbab..b6200464293 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -1,15 +1,15 @@ -execute: --allow_experimental_analyzer=1 +execute: --enable_analyzer=1 "foo" 1 -execute: --allow_experimental_analyzer=1 --stage fetch_columns +execute: --enable_analyzer=1 --stage fetch_columns "__table1.dummy" 0 -execute: --allow_experimental_analyzer=1 --stage with_mergeable_state +execute: --enable_analyzer=1 --stage with_mergeable_state "1_UInt8" 1 -execute: --allow_experimental_analyzer=1 --stage with_mergeable_state_after_aggregation +execute: --enable_analyzer=1 --stage with_mergeable_state_after_aggregation "1_UInt8" 1 -execute: --allow_experimental_analyzer=1 --stage complete +execute: --enable_analyzer=1 --stage complete "foo" 1 diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.sh b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh index 99267458421..79c9bb6ae10 100755 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.sh +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=1" + "--enable_analyzer=1" ) function execute_query() diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index b821ba13721..db727599d2c 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -1,6 +1,6 @@ -- Tags: long -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- Too slow SET max_bytes_before_external_sort = 0; diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index 679e9a749ee..be405f8e7d9 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -111,8 +111,8 @@ clickhouse_local "INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" clickhouse_local "SELECT if(count() = 10, 'MV is working', 'MV failed') FROM db_ordinary.src_mv_with_inner" clickhouse_local "DETACH VIEW db_ordinary.src_mv_with_inner PERMANENTLY; INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" --stacktrace -clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -c "db_ordinary.src_mv_with_inner does not exist" -clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -c "Unknown table expression identifier 'db_ordinary.src_mv_with_inner'" +clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS enable_analyzer = 0" 2>&1 | grep -c "db_ordinary.src_mv_with_inner does not exist" +clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS enable_analyzer = 1" 2>&1 | grep -c "Unknown table expression identifier 'db_ordinary.src_mv_with_inner'" ## Quite silly: ATTACH MATERIALIZED VIEW don't work with short syntax (w/o select), but i can attach it using ATTACH TABLE ... clickhouse_local "ATTACH TABLE db_ordinary.src_mv_with_inner" diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 84c872856ff..64a9db37a68 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -52,7 +52,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM constraint_test_constants WHERE c > 100 @@ -70,7 +70,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: c, result_type: Int64, source_id: 3 CONSTANT id: 7, constant_value: UInt64_100, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM constraint_test_constants QUERY id: 0 @@ -81,4 +81,4 @@ QUERY id: 0 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index acde02e2c67..e549467de02 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -101,10 +101,10 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); -- EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); ---> the order of the generated checks is not consistent EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100) SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100) SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); -EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100) SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100) SETTINGS enable_analyzer = 1; DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index 3f6e8211f1a..09a6dd3d0e7 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -11,7 +11,7 @@ QUERY id: 0 TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM t_constraints_where WHERE 0 @@ -25,7 +25,7 @@ QUERY id: 0 TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM t_constraints_where WHERE 0 @@ -39,7 +39,7 @@ QUERY id: 0 TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM t_constraints_where WHERE b < 8 @@ -57,7 +57,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM t_constraints_where PREWHERE (b > 20) OR (b < 8) @@ -75,7 +75,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 7, constant_value: UInt64_8, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT count() FROM t_constraints_where QUERY id: 0 @@ -86,4 +86,4 @@ QUERY id: 0 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_constraints_where - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index d41b1988bdd..63803ec8ce6 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -9,15 +9,15 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 15 SETTINGS enable_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 20 SETTINGS enable_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2 SETTINGS allow_experimental_analyzer = 1; -- assumption -> 0 +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b < 2 SETTINGS enable_analyzer = 1; -- assumption -> 0 EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8 SETTINGS allow_experimental_analyzer = 1; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8 SETTINGS enable_analyzer = 1; -- assumption -> remove (b < 20) EXPLAIN SYNTAX SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8 SETTINGS allow_experimental_analyzer = 1; -- assumption -> remove (b < 20) +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where PREWHERE b > 20 OR b < 8 SETTINGS enable_analyzer = 1; -- assumption -> remove (b < 20) DROP TABLE t_constraints_where; @@ -26,6 +26,6 @@ CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10 INSERT INTO t_constraints_where VALUES (1, 7); EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumption -> (b < 20) -> 0; -EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5 SETTINGS allow_experimental_analyzer = 1; -- assumption -> (b < 20) -> 0; +EXPLAIN QUERY TREE SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5 SETTINGS enable_analyzer = 1; -- assumption -> (b < 20) -> 0; DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index d504a86365b..b49884b4798 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -27,7 +27,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT cityHash64(a) + 10, b + 3 @@ -57,7 +57,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10, (b AS b) + 3 @@ -87,7 +87,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10, (b AS b) + 3 @@ -117,7 +117,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10, (b AS b) + 3 @@ -147,7 +147,7 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 14, constant_value: UInt64_1, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (b AS `cityHash64(a)`) + 10 FROM column_swap_test_test WHERE b = 0 @@ -169,7 +169,7 @@ QUERY id: 0 LIST id: 8, nodes: 2 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (cityHash64(a) AS `cityHash64(a)`) + 10, a @@ -201,7 +201,7 @@ QUERY id: 0 LIST id: 13, nodes: 1 COLUMN id: 14, column_name: a, result_type: String, source_id: 7 CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT (cityHash64(a) AS b) + 10, a @@ -233,7 +233,7 @@ QUERY id: 0 LIST id: 13, nodes: 1 COLUMN id: 14, column_name: a, result_type: String, source_id: 7 CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a @@ -255,7 +255,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a @@ -277,7 +277,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a AS t1, a AS t2 @@ -299,7 +299,7 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'c\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test_test WHERE a = \'c\' @@ -317,7 +317,7 @@ QUERY id: 0 LIST id: 5, nodes: 2 COLUMN id: 6, column_name: a, result_type: String, source_id: 3 CONSTANT id: 7, constant_value: \'c\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_bad_constraint QUERY id: 0 @@ -328,4 +328,4 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_bad_constraint - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 242be87938d..ccd387c9f8d 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -14,22 +14,22 @@ INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test PREWHERE cityHash64(a) = 1 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; -EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0 SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; -EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT b + 10, a FROM column_swap_test_test WHERE b = 0 SETTINGS enable_analyzer = 1; DROP TABLE column_swap_test_test; @@ -37,13 +37,13 @@ CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 A INSERT INTO column_swap_test_test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c' SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c' SETTINGS enable_analyzer = 1; DROP TABLE column_swap_test_test; @@ -54,6 +54,6 @@ CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUIn INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000); EXPLAIN SYNTAX SELECT a FROM t_bad_constraint; -EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_bad_constraint SETTINGS enable_analyzer = 1; DROP TABLE t_bad_constraint; diff --git a/tests/queries/0_stateless/01646_rewrite_sum_if.sql b/tests/queries/0_stateless/01646_rewrite_sum_if.sql index da341a3f7db..fd98a157ac6 100644 --- a/tests/queries/0_stateless/01646_rewrite_sum_if.sql +++ b/tests/queries/0_stateless/01646_rewrite_sum_if.sql @@ -34,7 +34,7 @@ SELECT sum(if(number % 2 == 0, 0, 1)) FROM numbers(100); SELECT sum(if(number % 2 == 0 as cond_expr, 0 as zero_expr, 1 as one_expr) as if_expr), sum(cond_expr), sum(if_expr), one_expr, zero_expr FROM numbers(100); SELECT countIf(number % 2 != 0) FROM numbers(100); -set allow_experimental_analyzer = true; +set enable_analyzer = true; EXPLAIN QUERY TREE run_passes=1 SELECT sumIf(123, number % 2 == 0) FROM numbers(100); EXPLAIN QUERY TREE run_passes=1 SELECT sum(if(number % 2 == 0, 123, 0)) FROM numbers(100); diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index dd31f2941ef..b98feaa1f65 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -111,5 +111,5 @@ WITH ( ) AS t) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = -9223372036854775808, 'ok', ''); -set joined_subquery_requires_alias=0, allow_experimental_analyzer=0; -- the query is invalid with a new analyzer +set joined_subquery_requires_alias=0, enable_analyzer=0; -- the query is invalid with a new analyzer SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 1048577) USING (n) LIMIT 10 format Null; diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 4bd0eb7d908..42cdac8c01f 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -26,13 +26,13 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter should be pushed down after aggregating, column after aggregation is const" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y, 0)" echo "> (analyzer) filter should be pushed down after aggregating, column after aggregation is const" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 @@ -44,14 +44,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is aliased" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 @@ -64,14 +64,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is casted" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 @@ -84,14 +84,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 @@ -104,14 +104,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 @@ -124,13 +124,13 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter is split, one part is filtered before ARRAY JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | grep -o "Filter column: and(notEquals(y, 2), notEquals(x, 0))\|ARRAY JOIN x\|Filter column: notEquals(y, 2)" echo "> (analyzer) filter is split, one part is filtered before ARRAY JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | @@ -154,14 +154,14 @@ $CLICKHOUSE_CLIENT -q " # settings enable_optimize_predicate_expression=0" echo "> filter is pushed down before Distinct" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) ) where y != 2 settings enable_optimize_predicate_expression=0" | grep -o "Distinct\|Filter column: notEquals(y, 2)" echo "> (analyzer) filter is pushed down before Distinct" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) ) where y != 2 @@ -174,14 +174,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter is pushed down before sorting steps" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | grep -o "Sorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" echo "> (analyzer) filter is pushed down before sorting steps" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 @@ -194,14 +194,14 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression = 0" echo "> filter is pushed down before TOTALS HAVING and aggregating" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals ) where y != 2 settings enable_optimize_predicate_expression=0" | grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y, 2)" echo "> (analyzer) filter is pushed down before TOTALS HAVING and aggregating" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals ) where y != 2 @@ -224,14 +224,14 @@ $CLICKHOUSE_CLIENT -q " ) where number != 2 settings enable_optimize_predicate_expression=0" echo "> one condition of filter is pushed down before LEFT JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | grep -o "Join\|Filter column: notEquals(number, 1)" echo "> (analyzer) one condition of filter is pushed down before LEFT JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) @@ -243,14 +243,14 @@ $CLICKHOUSE_CLIENT -q " ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | sort echo "> one condition of filter is pushed down before INNER JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | grep -o "Join\|Filter column: and(notEquals(number, 1), notEquals(number, 2))\|Filter column: and(notEquals(b, 2), notEquals(b, 1))" echo "> (analyzer) one condition of filter is pushed down before INNER JOIN" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) @@ -274,12 +274,12 @@ $CLICKHOUSE_CLIENT -q " echo "> function calculation should be done after sorting and limit (if possible)" echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q " explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" | sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" echo "> (analyzer) function calculation should be done after sorting and limit (if possible)" echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting" -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " +$CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" | sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" echo "> this query should be executed without throwing an exception" diff --git a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql index 2193fc7a8f4..c6620184a43 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql +++ b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql @@ -1,7 +1,7 @@ set query_plan_merge_filters=1; -set allow_experimental_analyzer=1; +set enable_analyzer=1; select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%'; -set allow_experimental_analyzer=0; +set enable_analyzer=0; select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%'; diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index 7c2753124b3..c8b3c5ca954 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -1,23 +1,23 @@ Partial sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC - optimize_read_in_window_order=0, allow_experimental_analyzer=1 + optimize_read_in_window_order=0, enable_analyzer=1 Sort description: n ASC, x ASC optimize_read_in_window_order=1 Prefix sort description: n ASC Result sort description: n ASC, x ASC - optimize_read_in_window_order=1, allow_experimental_analyzer=1 + optimize_read_in_window_order=1, enable_analyzer=1 Prefix sort description: __table1.n ASC Result sort description: __table1.n ASC, __table1.x ASC No sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC - optimize_read_in_window_order=0, allow_experimental_analyzer=1 + optimize_read_in_window_order=0, enable_analyzer=1 Sort description: __table1.n ASC, __table1.x ASC optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC - optimize_read_in_window_order=1, allow_experimental_analyzer=1 + optimize_read_in_window_order=1, enable_analyzer=1 Prefix sort description: __table1.n ASC, __table1.x ASC Result sort description: __table1.n ASC, __table1.x ASC Complex ORDER BY diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 24c8cf5052e..d74ea328d04 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -19,25 +19,25 @@ $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" echo 'Partial sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,allow_experimental_analyzer=0" | grep -i "sort description" -echo ' optimize_read_in_window_order=0, allow_experimental_analyzer=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,allow_experimental_analyzer=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,enable_analyzer=0" | grep -i "sort description" +echo ' optimize_read_in_window_order=0, enable_analyzer=1' +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,enable_analyzer=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,allow_experimental_analyzer=0" | grep -i "sort description" -echo ' optimize_read_in_window_order=1, allow_experimental_analyzer=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,allow_experimental_analyzer=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,enable_analyzer=0" | grep -i "sort description" +echo ' optimize_read_in_window_order=1, enable_analyzer=1' +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,enable_analyzer=1" | grep -i "sort description" echo 'No sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,allow_experimental_analyzer=0" | grep -i "sort description" -echo ' optimize_read_in_window_order=0, allow_experimental_analyzer=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,allow_experimental_analyzer=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,enable_analyzer=0" | grep -i "sort description" +echo ' optimize_read_in_window_order=0, enable_analyzer=1' +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0,enable_analyzer=1" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,allow_experimental_analyzer=0" | grep -i "sort description" -echo ' optimize_read_in_window_order=1, allow_experimental_analyzer=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,allow_experimental_analyzer=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,enable_analyzer=0" | grep -i "sort description" +echo ' optimize_read_in_window_order=1, enable_analyzer=1' +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,enable_analyzer=1" | grep -i "sort description" echo 'Complex ORDER BY' $CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" diff --git a/tests/queries/0_stateless/01671_merge_join_and_constants.sql b/tests/queries/0_stateless/01671_merge_join_and_constants.sql index 7a84bd4e97a..a2153bf0093 100644 --- a/tests/queries/0_stateless/01671_merge_join_and_constants.sql +++ b/tests/queries/0_stateless/01671_merge_join_and_constants.sql @@ -1,5 +1,5 @@ SET output_format_pretty_color=1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index db004c13d96..3c2fa9877db 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -37,8 +37,8 @@ SELECT a, t1.a, t2.a FROM t1 FULL JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is SELECT '= left ='; SELECT a, t1.a, t2.a FROM t1 LEFT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); SELECT '= right ='; -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS enable_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS enable_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT '= inner ='; SELECT a, t1.a, t2.a FROM t1 INNER JOIN t2 USING (a) ORDER BY (t1.a, t2.a); @@ -119,8 +119,8 @@ SELECT a, t1.a, t2.a FROM t1 FULL JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is SELECT '= left ='; SELECT a, t1.a, t2.a FROM t1 LEFT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); SELECT '= right ='; -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS enable_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS enable_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT '= inner ='; SELECT a, t1.a, t2.a FROM t1 INNER JOIN t2 USING (a) ORDER BY (t1.a, t2.a); @@ -166,8 +166,8 @@ SELECT '= types ='; SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 FULL JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 LEFT JOIN t2 USING (a); -SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} -SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} +SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS enable_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS enable_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 INNER JOIN t2 USING (a); diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 21f4edc0049..b921dc6c1f9 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -35,9 +35,9 @@ SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_us drop table XXXX; CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=0; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, enable_analyzer=0; 0 -- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=1; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, enable_analyzer=1; 3 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index 1eca65f0892..b208063e7c4 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -38,8 +38,8 @@ CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PART INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=0; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, enable_analyzer=0; -- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=1; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, enable_analyzer=1; drop table XXXX; diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 28dbb9215a8..74a0356b11e 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -103,7 +103,7 @@ errors -- optimize_skip_unused_shards does not support non-constants select * from dist_01756 where dummy in (select * from system.one); -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- this is a constant for analyzer -select * from dist_01756 where dummy in (toUInt8(0)) settings allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01756 where dummy in (toUInt8(0)) settings enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- NOT IN does not supported select * from dist_01756 where dummy not in (0, 2); -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- @@ -140,7 +140,7 @@ select * from dist_01756_str where key in ('0', '2'); select * from dist_01756_str where key in (0, 2); 0 -- analyzer does support this -select * from dist_01756_str where key in ('0', Null) settings allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01756_str where key in ('0', Null) settings enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- select * from dist_01756_str where key in (0, 2); -- { serverError TYPE_MISMATCH } -- select * from dist_01756_str where key in (0, Null); -- { serverError TYPE_MISMATCH } diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql index 9a1a00cc0a1..bcbedeb3ada 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql @@ -113,7 +113,7 @@ select 'errors'; -- optimize_skip_unused_shards does not support non-constants select * from dist_01756 where dummy in (select * from system.one); -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- this is a constant for analyzer -select * from dist_01756 where dummy in (toUInt8(0)) settings allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01756 where dummy in (toUInt8(0)) settings enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- NOT IN does not supported select * from dist_01756 where dummy not in (0, 2); -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } @@ -146,7 +146,7 @@ create table dist_01756_str as data_01756_str engine=Distributed(test_cluster_tw select * from dist_01756_str where key in ('0', '2'); select * from dist_01756_str where key in (0, 2); -- analyzer does support this -select * from dist_01756_str where key in ('0', Null) settings allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01756_str where key in ('0', Null) settings enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- select * from dist_01756_str where key in (0, 2); -- { serverError TYPE_MISMATCH } -- select * from dist_01756_str where key in (0, Null); -- { serverError TYPE_MISMATCH } diff --git a/tests/queries/0_stateless/01757_optimize_skip_unused_shards_limit.sql b/tests/queries/0_stateless/01757_optimize_skip_unused_shards_limit.sql index 3853ccb4080..6fcf98d47cb 100644 --- a/tests/queries/0_stateless/01757_optimize_skip_unused_shards_limit.sql +++ b/tests/queries/0_stateless/01757_optimize_skip_unused_shards_limit.sql @@ -21,9 +21,9 @@ select * from dist_01757 where dummy = 0 or dummy = 1 format Null settings optim -- and negative -- disabled for analyzer cause new implementation consider `dummy = 0 and dummy = 1` as constant False. -select * from dist_01757 where dummy = 0 and dummy = 1 settings optimize_skip_unused_shards_limit=1, allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -select * from dist_01757 where dummy = 0 and dummy = 2 and dummy = 3 settings optimize_skip_unused_shards_limit=1, allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -select * from dist_01757 where dummy = 0 and dummy = 2 and dummy = 3 settings optimize_skip_unused_shards_limit=2, allow_experimental_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01757 where dummy = 0 and dummy = 1 settings optimize_skip_unused_shards_limit=1, enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01757 where dummy = 0 and dummy = 2 and dummy = 3 settings optimize_skip_unused_shards_limit=1, enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } +select * from dist_01757 where dummy = 0 and dummy = 2 and dummy = 3 settings optimize_skip_unused_shards_limit=2, enable_analyzer=0; -- { serverError UNABLE_TO_SKIP_UNUSED_SHARDS } -- and select * from dist_01757 where dummy = 0 and dummy = 1 settings optimize_skip_unused_shards_limit=2; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 367baef142b..aa610e19c12 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -57,13 +57,13 @@ EXPLAIN indexes=1 SELECT id, delete_time FROM t1 CROSS JOIN ( SELECT delete_time FROM t2 -) AS d WHERE create_time < delete_time AND id = 101 SETTINGS allow_experimental_analyzer=0; +) AS d WHERE create_time < delete_time AND id = 101 SETTINGS enable_analyzer=0; EXPLAIN indexes=1 SELECT id, delete_time FROM t1 CROSS JOIN ( SELECT delete_time FROM t2 -) AS d WHERE create_time < delete_time AND id = 101 SETTINGS allow_experimental_analyzer=1; +) AS d WHERE create_time < delete_time AND id = 101 SETTINGS enable_analyzer=1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index e3b28acdc41..828012f56bc 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for i in $(seq 0 1) do - CH_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_query_to_cnf=0 --optimize_read_in_order=1 --allow_experimental_analyzer=$i" + CH_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_query_to_cnf=0 --optimize_read_in_order=1 --enable_analyzer=$i" $CH_CLIENT -q "drop table if exists test_index" $CH_CLIENT -q "drop table if exists idx" diff --git a/tests/queries/0_stateless/01823_explain_json.sh b/tests/queries/0_stateless/01823_explain_json.sh index 39128773069..356a317ae57 100755 --- a/tests/queries/0_stateless/01823_explain_json.sh +++ b/tests/queries/0_stateless/01823_explain_json.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=1" + "--enable_analyzer=1" ) $CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw" echo "--------" diff --git a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql index 8c6937eb581..4387a697a83 100644 --- a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql +++ b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS v1; DROP TABLE IF EXISTS v2; diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns_analyzer.sql b/tests/queries/0_stateless/01872_functions_to_subcolumns_analyzer.sql index b544f6829cf..032d83890ec 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns_analyzer.sql +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns_analyzer.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS t_func_to_subcolumns; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_functions_to_subcolumns = 1; CREATE TABLE t_func_to_subcolumns (id UInt64, arr Array(UInt64), n Nullable(String), m Map(String, UInt64)) diff --git a/tests/queries/0_stateless/01890_cross_join_explain_crash.sql b/tests/queries/0_stateless/01890_cross_join_explain_crash.sql index bb2bc606870..79aea3884b7 100644 --- a/tests/queries/0_stateless/01890_cross_join_explain_crash.sql +++ b/tests/queries/0_stateless/01890_cross_join_explain_crash.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET joined_subquery_requires_alias = 0; select * FROM (SELECT 1), (SELECT 1), (SELECT 1); diff --git a/tests/queries/0_stateless/01913_names_of_tuple_literal.sql b/tests/queries/0_stateless/01913_names_of_tuple_literal.sql index 879f4c91587..d6dda4fda9b 100644 --- a/tests/queries/0_stateless/01913_names_of_tuple_literal.sql +++ b/tests/queries/0_stateless/01913_names_of_tuple_literal.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames; SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames SETTINGS legacy_column_name_of_tuple_literal = 1; diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.sql b/tests/queries/0_stateless/01925_join_materialized_columns.sql index abb6fda3bfb..ce71cfed8a3 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.sql +++ b/tests/queries/0_stateless/01925_join_materialized_columns.sql @@ -32,8 +32,8 @@ SELECT t1.dt, t2.dt FROM t1 JOIN t2 ON t1.foo = t2.bar ORDER BY t1.dt; SELECT '-'; SELECT * FROM t1 ALL JOIN t2 ON t1.dt = t2.dt ORDER BY t1.time, t2.time; SELECT '-'; -SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings allow_experimental_analyzer=0; -SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings allow_experimental_analyzer=1; +SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings enable_analyzer=0; +SELECT * FROM t1 ALL JOIN t2 USING (dt) ORDER BY t1.time, t2.time settings enable_analyzer=1; SELECT '-'; SELECT * FROM t1 JOIN t2 ON t1.dt1 = t2.dt2 ORDER BY t1.time, t1.dimension_1, t2.time, t2.dimension_2; SELECT '-'; @@ -53,5 +53,5 @@ SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time_alias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time = talias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time_alias = talias; SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias; -- { serverError AMBIGUOUS_COLUMN_NAME, INVALID_JOIN_ON_EXPRESSION } -SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings allow_experimental_analyzer=0; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings allow_experimental_analyzer=1; +SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/01925_test_storage_merge_aliases_analyzer.sql b/tests/queries/0_stateless/01925_test_storage_merge_aliases_analyzer.sql index 31035aa80cd..28b9c8650af 100644 --- a/tests/queries/0_stateless/01925_test_storage_merge_aliases_analyzer.sql +++ b/tests/queries/0_stateless/01925_test_storage_merge_aliases_analyzer.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel drop table if exists merge; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; create table merge ( dt Date, diff --git a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql index f344b7007d0..4efbb461501 100644 --- a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql +++ b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT dictGet(t.nest.a, concat(currentDatabase(), '.dict.dict'), 's', number) FROM numbers(5); -- { serverError INVALID_IDENTIFIER } diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index 945b399157f..0154265ef72 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -1,5 +1,5 @@ SET session_timezone = 'UTC'; -- disable timezone randomization -SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET enable_analyzer = 1; -- The old path formats the result with different whitespaces SELECT '-- Negative tests'; SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql index 48316691c71..41e5beb9c16 100644 --- a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -1,5 +1,5 @@ SET session_timezone = 'UTC'; -- disable timezone randomization -SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET enable_analyzer = 1; -- The old path formats the result with different whitespaces SELECT '-- Negative tests'; SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index 5acefdb365e..e786532f25a 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -71,7 +71,7 @@ Expression (Projection) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) ReadFromSystemNumbers ReadFromRemote (Read from remote replica) -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized Expression (Project names) Distinct (DISTINCT) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql index adf55a9dd7f..960fd227a39 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql @@ -4,7 +4,7 @@ set optimize_skip_unused_shards=1; set optimize_distributed_group_by_sharding_key=1; set prefer_localhost_replica=1; -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; -- { echo } explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized @@ -17,7 +17,7 @@ explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 848ecedf9e3..3bd1633ce32 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 2 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 2c1152e0ae6..da70973ed87 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -56,30 +56,30 @@ SELECT * FROM t1 RIGHT JOIN t2 ON NULL SETTINGS join_algorithm = 'auto'; -- { se SELECT * FROM t1 FULL JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } -- mixing of constant and non-constant expressions in ON is not allowed -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS allow_experimental_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS enable_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS enable_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS enable_analyzer = 1; SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND NULL; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION,ILLEGAL_TYPE_OF_ARGUMENT } SELECT * FROM t1 JOIN t2 ON 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; @@ -94,27 +94,26 @@ SELECT a + 1 FROM (SELECT 1 as x) as t1 LEFT JOIN ( SELECT 1 AS a ) AS t2 ON TRUE -SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; +SETTINGS enable_analyzer=1, join_use_nulls=1; SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) FROM (SELECT 1 as x) as t1 LEFT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 ON TRUE -SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; +SETTINGS enable_analyzer=1, join_use_nulls=1; SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) FROM (SELECT 1 as x) as t1 RIGHT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 ON TRUE -SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; +SETTINGS enable_analyzer=1, join_use_nulls=1; SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) FROM (SELECT 1 as x) as t1 FULL JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 ON TRUE -SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; +SETTINGS enable_analyzer=1, join_use_nulls=1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; - diff --git a/tests/queries/0_stateless/02030_tuple_filter.sql b/tests/queries/0_stateless/02030_tuple_filter.sql index 42853dec681..c8f344f5076 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.sql +++ b/tests/queries/0_stateless/02030_tuple_filter.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_tuple_filter; diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 2631199cbab..b6200464293 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -1,15 +1,15 @@ -execute: --allow_experimental_analyzer=1 +execute: --enable_analyzer=1 "foo" 1 -execute: --allow_experimental_analyzer=1 --stage fetch_columns +execute: --enable_analyzer=1 --stage fetch_columns "__table1.dummy" 0 -execute: --allow_experimental_analyzer=1 --stage with_mergeable_state +execute: --enable_analyzer=1 --stage with_mergeable_state "1_UInt8" 1 -execute: --allow_experimental_analyzer=1 --stage with_mergeable_state_after_aggregation +execute: --enable_analyzer=1 --stage with_mergeable_state_after_aggregation "1_UInt8" 1 -execute: --allow_experimental_analyzer=1 --stage complete +execute: --enable_analyzer=1 --stage complete "foo" 1 diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh index 182acc23a13..09a7e8efefc 100755 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=1" + "--enable_analyzer=1" ) function execute_query() diff --git a/tests/queries/0_stateless/02115_map_contains_analyzer.sql b/tests/queries/0_stateless/02115_map_contains_analyzer.sql index 46e02eca4f0..00285404987 100644 --- a/tests/queries/0_stateless/02115_map_contains_analyzer.sql +++ b/tests/queries/0_stateless/02115_map_contains_analyzer.sql @@ -5,7 +5,7 @@ CREATE TABLE t_map_contains (m Map(String, UInt32)) ENGINE = Memory; INSERT INTO t_map_contains VALUES (map('a', 1, 'b', 2)), (map('c', 3, 'd', 4)); SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE dump_tree = 0, dump_ast = 1 SELECT mapContains(m, 'a') FROM t_map_contains; SELECT mapContains(m, 'a') FROM t_map_contains; diff --git a/tests/queries/0_stateless/02116_tuple_element_analyzer.sql b/tests/queries/0_stateless/02116_tuple_element_analyzer.sql index 5aeb72c9ee4..ef3729bdc95 100644 --- a/tests/queries/0_stateless/02116_tuple_element_analyzer.sql +++ b/tests/queries/0_stateless/02116_tuple_element_analyzer.sql @@ -4,7 +4,7 @@ CREATE TABLE t_tuple_element(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, Stri INSERT INTO t_tuple_element VALUES ((1, 'a'), (2, 'b')); SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT t1.1 FROM t_tuple_element; EXPLAIN QUERY TREE dump_tree = 0, dump_ast = 1 SELECT t1.1 FROM t_tuple_element; diff --git a/tests/queries/0_stateless/02125_query_views_log_window_function.sql b/tests/queries/0_stateless/02125_query_views_log_window_function.sql index fff1e943c58..3f15cf5b998 100644 --- a/tests/queries/0_stateless/02125_query_views_log_window_function.sql +++ b/tests/queries/0_stateless/02125_query_views_log_window_function.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; set allow_experimental_window_view = 1; CREATE TABLE data ( `id` UInt64, `timestamp` DateTime) ENGINE = Memory; diff --git a/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql index 0f12bc2eb9c..20a4f2bcf5f 100644 --- a/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql +++ b/tests/queries/0_stateless/02129_window_functions_disable_optimizations.sql @@ -29,4 +29,4 @@ HAVING sum(log(2) * number) > 346.57353 ORDER BY k; SELECT round(sum(log(2) * number), 6) AS k FROM numbers(10000) GROUP BY (number % 2) * (number % 3), number % 3, number % 2 HAVING sum(log(2) * number) > 346.57353 ORDER BY k -SETTINGS allow_experimental_analyzer=1; +SETTINGS enable_analyzer=1; diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh index 1fe345d266d..00f2c92161f 100755 --- a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh @@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "#1" ${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed" echo "#2" -${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505, allow_experimental_analyzer = 1 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" +${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505, enable_analyzer = 1 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql index ae8c39b49bc..7bbdecf5501 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql @@ -22,20 +22,20 @@ SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, -i LIMIT 5; EXPLAIN PIPELINE SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, -i LIMIT 5; SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5; -EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5 settings allow_experimental_analyzer=0; -EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5 settings allow_experimental_analyzer=1; +EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5 settings enable_analyzer=0; +EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5 settings enable_analyzer=1; SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5; -EXPLAIN PIPELINE SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5 settings allow_experimental_analyzer=0; -EXPLAIN PIPELINE SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5 settings allow_experimental_analyzer=1; +EXPLAIN PIPELINE SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5 settings enable_analyzer=0; +EXPLAIN PIPELINE SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5 settings enable_analyzer=1; INSERT INTO t_read_in_order SELECT '2020-10-12', number, number FROM numbers(100000); SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i LIMIT 5; EXPLAIN SYNTAX SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5; -EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5 settings allow_experimental_analyzer=0; -EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5 settings allow_experimental_analyzer=1; +EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5 settings enable_analyzer=0; +EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5 settings enable_analyzer=1; SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5; DROP TABLE IF EXISTS t_read_in_order; diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh index 6d715604d93..007dae6e427 100755 --- a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh @@ -58,7 +58,7 @@ $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" for i in "${!queries_without_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_without_preallocation[$i]}" -q " -- the old analyzer is not supported - SELECT sum(if(getSetting('allow_experimental_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0, 1)) + SELECT sum(if(getSetting('enable_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0, 1)) FROM system.query_log WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND type = 'QueryFinish' " @@ -67,7 +67,7 @@ done for i in "${!queries_with_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_with_preallocation[$i]}" -q " -- the old analyzer is not supported - SELECT sum(if(getSetting('allow_experimental_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0, 1)) + SELECT sum(if(getSetting('enable_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0, 1)) FROM system.query_log WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND type = 'QueryFinish' " diff --git a/tests/queries/0_stateless/02154_dictionary_get_http_json.sh b/tests/queries/0_stateless/02154_dictionary_get_http_json.sh index fbaf67fff2f..bcd9f4f5c1a 100755 --- a/tests/queries/0_stateless/02154_dictionary_get_http_json.sh +++ b/tests/queries/0_stateless/02154_dictionary_get_http_json.sh @@ -32,7 +32,7 @@ $CLICKHOUSE_CLIENT -q """ echo """ SELECT dictGet(02154_test_dictionary, 'value', toUInt64(0)), dictGet(02154_test_dictionary, 'value', toUInt64(1)) - SETTINGS allow_experimental_analyzer = 1 + SETTINGS enable_analyzer = 1 FORMAT JSON """ | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&wait_end_of_query=1&output_format_write_statistics=0" -d @- diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index 4f010ebadfd..2e5066fc488 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -24,8 +24,8 @@ INSERT INTO t_02156_mt1 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_mt2 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_log SELECT number, toString(number) FROM numbers(10000); -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings allow_experimental_analyzer=1; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings allow_experimental_analyzer=0; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings enable_analyzer=1; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings enable_analyzer=0; SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v); SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge2 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache.sql b/tests/queries/0_stateless/02174_cte_scalar_cache.sql index 86cfff21446..d14475c843a 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH ( SELECT sleep(0.0001) FROM system.one ) as a1, @@ -28,7 +28,7 @@ WITH ( SELECT sleep(0.0001) FROM system.one ) as a5 SELECT '02177_CTE_NEW_ANALYZER', a1, a2, a3, a4, a5 FROM system.numbers LIMIT 100 FORMAT Null -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; SYSTEM FLUSH LOGS; SELECT diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql index ca54b9e1400..a2be0a11faf 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql @@ -14,7 +14,7 @@ CREATE MATERIALIZED VIEW mv1 TO t2 AS FROM t1 LIMIT 5; -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; -- FIRST INSERT INSERT INTO t1 @@ -61,7 +61,7 @@ WHERE AND event_date >= yesterday() AND event_time > now() - interval 10 minute; truncate table t2; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; -- FIRST INSERT ANALYZER INSERT INTO t1 @@ -100,7 +100,7 @@ WHERE DROP TABLE mv1; -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; CREATE TABLE t3 (z Int64) ENGINE = Memory; CREATE MATERIALIZED VIEW mv2 TO t3 AS @@ -134,7 +134,7 @@ WHERE AND event_date >= yesterday() AND event_time > now() - interval 10 minute; truncate table t3; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; -- SECOND INSERT ANALYZER INSERT INTO t1 @@ -162,7 +162,7 @@ WHERE DROP TABLE mv2; -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; CREATE TABLE t4 (z Int64) ENGINE = Memory; CREATE MATERIALIZED VIEW mv3 TO t4 AS @@ -197,7 +197,7 @@ WHERE AND event_date >= yesterday() AND event_time > now() - interval 10 minute; truncate table t4; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; -- THIRD INSERT ANALYZER INSERT INTO t1 diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql index c90c8b90c9e..e7d1909cae6 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql @@ -1,6 +1,6 @@ -- Tags: no-fasttest -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toIPv4('1.2.3.4') AS ipv4, diff --git a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference index 0ff24b39709..61dfd1c6779 100644 --- a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference +++ b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference @@ -40,7 +40,7 @@ QUERY id: 0 LIST id: 3, nodes: 1 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String CONSTANT id: 16, constant_value: \'world%\', constant_value_type: String - SETTINGS optimize_or_like_chain=0 allow_experimental_analyzer=1 + SETTINGS optimize_or_like_chain=0 enable_analyzer=1 SELECT materialize(\'Привет, World\') AS s WHERE multiMatchAny(s, [\'^hell\', \'(?i)привет\', \'(?i)^world\']) OR false SETTINGS optimize_or_like_chain = 1 @@ -68,7 +68,7 @@ QUERY id: 0 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String CONSTANT id: 10, constant_value: Array_[\'^hell\', \'(?i)привет\', \'(?i)^world\'], constant_value_type: Array(String) CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS optimize_or_like_chain=1 allow_experimental_analyzer=1 + SETTINGS optimize_or_like_chain=1 enable_analyzer=1 SELECT materialize(\'Привет, World\') AS s1, materialize(\'Привет, World\') AS s2 diff --git a/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql b/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql index fbebfc6d281..b23e5640b8f 100644 --- a/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql +++ b/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql @@ -1,7 +1,7 @@ EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 0; -EXPLAIN QUERY TREE run_passes=1 SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 0, allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE run_passes=1 SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 0, enable_analyzer = 1; EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1; -EXPLAIN QUERY TREE run_passes=1 SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE run_passes=1 SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, enable_analyzer = 1; EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s1, materialize('Привет, World') AS s2 WHERE (s1 LIKE 'hell%') OR (s2 ILIKE '%привет%') OR (s1 ILIKE 'world%') SETTINGS optimize_or_like_chain = 1; EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s1, materialize('Привет, World') AS s2 WHERE (s1 LIKE 'hell%') OR (s2 ILIKE '%привет%') OR (s1 ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, allow_hyperscan = 0; @@ -11,18 +11,18 @@ EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s1, materialize('П SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 1; -SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 1, allow_experimental_analyzer = 1; +SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 1, enable_analyzer = 1; SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 0; -SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 0, allow_experimental_analyzer = 1; +SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s LIKE '%привет%') OR (s ILIKE '%world') SETTINGS optimize_or_like_chain = 0, enable_analyzer = 1; SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 1; -SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 1, allow_experimental_analyzer = 1; +SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 1, enable_analyzer = 1; SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 0; -SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 0, allow_experimental_analyzer = 1; +SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 0, enable_analyzer = 1; -SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, allow_experimental_analyzer = 1; +SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, enable_analyzer = 1; -- Aliases diff --git a/tests/queries/0_stateless/02227_union_match_by_name.sql b/tests/queries/0_stateless/02227_union_match_by_name.sql index 6a19add1d37..489c3d976ea 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.sql +++ b/tests/queries/0_stateless/02227_union_match_by_name.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02233_with_total_empty_chunk.sql b/tests/queries/0_stateless/02233_with_total_empty_chunk.sql index d59319ac75e..c70b35df459 100644 --- a/tests/queries/0_stateless/02233_with_total_empty_chunk.sql +++ b/tests/queries/0_stateless/02233_with_total_empty_chunk.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT (NULL, NULL, NULL, NULL, NULL, NULL, NULL) FROM numbers(0) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]) -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER }; diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh index a6c47d80fa9..d387fda746c 100755 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh @@ -4,6 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS enable_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS enable_analyzer = 0" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql index fbcc374ba10..ecb2198237f 100644 --- a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql +++ b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET join_algorithm = 'hash'; @@ -22,7 +22,7 @@ SELECT * FROM ( SELECT count('a'), count('b'), count('c'), 2 AS x ) as t1 RIGHT SELECT 'y', * FROM (SELECT count('y'), count('y'), 2 AS x) AS t1 RIGHT JOIN (SELECT count('x'), count('y'), 3 AS x) AS t2 ON t1.x = t2.x; SELECT * FROM (SELECT arrayJoin([NULL]), 9223372036854775806, arrayJoin([NULL]), NULL AS x) AS t1 RIGHT JOIN (SELECT arrayJoin([arrayJoin([10000000000.])]), NULL AS x) AS t2 ON t1.x = t2.x; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET join_algorithm = 'hash'; SELECT * FROM ( SELECT 2 AS x ) AS t1 RIGHT JOIN ( SELECT count('x'), count('y'), 2 AS x ) AS t2 ON t1.x = t2.x; diff --git a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql index f8faa3e653b..ab89d98c608 100644 --- a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql +++ b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS github_events; diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 9f1c026f889..c26242098e3 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -1,4 +1,4 @@ -clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-client --enable_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -7,7 +7,7 @@ Header: dummy String Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-local --enable_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -16,7 +16,7 @@ Header: dummy String Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-client --enable_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -25,7 +25,7 @@ Header: dummy String Header: __table1.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-local --enable_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating diff --git a/tests/queries/0_stateless/02303_query_kind.sh b/tests/queries/0_stateless/02303_query_kind.sh index 1d883a2dcc7..7fe491b9deb 100755 --- a/tests/queries/0_stateless/02303_query_kind.sh +++ b/tests/queries/0_stateless/02303_query_kind.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=1" + "--enable_analyzer=1" ) function run_query() diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.reference b/tests/queries/0_stateless/02315_grouping_constant_folding.reference index 31816318a42..7b8c75b2304 100644 --- a/tests/queries/0_stateless/02315_grouping_constant_folding.reference +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.reference @@ -27,7 +27,7 @@ SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, 5 0 0 2 5 1 0 2 10 0 0 0 -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a, a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0, allow_experimental_analyzer=1; +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a, a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0, enable_analyzer=1; 1 0 0 3 1 0 2 3 1 0 4 3 diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.sql b/tests/queries/0_stateless/02315_grouping_constant_folding.sql index f992aa0da32..5e305d2e6c5 100644 --- a/tests/queries/0_stateless/02315_grouping_constant_folding.sql +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.sql @@ -9,7 +9,7 @@ SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0; -SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a, a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0, allow_experimental_analyzer=1; +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a, a), ()) ORDER BY (amount, a, b) SETTINGS force_grouping_standard_compatibility=0, enable_analyzer=1; -- { echoOff } DROP TABLE test02315; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 07c7bc4af56..bd7e6be3987 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -77,7 +77,7 @@ echo "-- enabled, only part of distinct columns form prefix of sorting key" $CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "=== disable new analyzer ===" -DISABLE_ANALYZER="set allow_experimental_analyzer=0" +DISABLE_ANALYZER="set enable_analyzer=0" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES @@ -99,7 +99,7 @@ echo "-- enabled, check that disabling other 'read in order' optimizations do no $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES echo "=== enable new analyzer ===" -ENABLE_ANALYZER="set allow_experimental_analyzer=1" +ENABLE_ANALYZER="set enable_analyzer=1" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" $CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql index 167eecc6fb8..c132a69ac21 100644 --- a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -1,6 +1,6 @@ -- Tags: no-parallel -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- Empty from section diff --git a/tests/queries/0_stateless/02337_multiple_joins_original_names.sql b/tests/queries/0_stateless/02337_multiple_joins_original_names.sql index 63bbfe6873d..37c7077b56e 100644 --- a/tests/queries/0_stateless/02337_multiple_joins_original_names.sql +++ b/tests/queries/0_stateless/02337_multiple_joins_original_names.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/34697 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT table1_id FROM ( SELECT first.table1_id diff --git a/tests/queries/0_stateless/02338_analyzer_constants_basic.sql b/tests/queries/0_stateless/02338_analyzer_constants_basic.sql index 6d6249538a4..536202dc2cf 100644 --- a/tests/queries/0_stateless/02338_analyzer_constants_basic.sql +++ b/tests/queries/0_stateless/02338_analyzer_constants_basic.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DESCRIBE (SELECT 1); SELECT 1; diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql index a09325fa43b..57c6a9479ec 100644 --- a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql @@ -1,6 +1,6 @@ -- Tags: no-parallel -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 'Matchers without FROM section'; diff --git a/tests/queries/0_stateless/02340_analyzer_functions.sql b/tests/queries/0_stateless/02340_analyzer_functions.sql index 101a5bfcc86..bd018302913 100644 --- a/tests/queries/0_stateless/02340_analyzer_functions.sql +++ b/tests/queries/0_stateless/02340_analyzer_functions.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DESCRIBE (SELECT 1 + 1); SELECT 1 + 1; diff --git a/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql index 9f21db8e659..8bed0c8bc1e 100644 --- a/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql +++ b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 'Aliases to constants'; diff --git a/tests/queries/0_stateless/02341_global_join_cte.reference b/tests/queries/0_stateless/02341_global_join_cte.reference index f2cfe994ffa..4f854ca3bef 100644 --- a/tests/queries/0_stateless/02341_global_join_cte.reference +++ b/tests/queries/0_stateless/02341_global_join_cte.reference @@ -1,6 +1,6 @@ -- { echo } -with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings allow_experimental_analyzer=0; -- { serverError ALIAS_REQUIRED } -with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings allow_experimental_analyzer=1; -- It works with analyzer; rhs is an alias itself. +with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings enable_analyzer=0; -- { serverError ALIAS_REQUIRED } +with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings enable_analyzer=1; -- It works with analyzer; rhs is an alias itself. 0 0 0 diff --git a/tests/queries/0_stateless/02341_global_join_cte.sql b/tests/queries/0_stateless/02341_global_join_cte.sql index b9b906afd70..f6acd822f14 100644 --- a/tests/queries/0_stateless/02341_global_join_cte.sql +++ b/tests/queries/0_stateless/02341_global_join_cte.sql @@ -1,5 +1,5 @@ -- { echo } -with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings allow_experimental_analyzer=0; -- { serverError ALIAS_REQUIRED } -with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings allow_experimental_analyzer=1; -- It works with analyzer; rhs is an alias itself. +with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings enable_analyzer=0; -- { serverError ALIAS_REQUIRED } +with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings enable_analyzer=1; -- It works with analyzer; rhs is an alias itself. with rhs as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs using (d1) order by rhs.d2 settings joined_subquery_requires_alias=0; with rhs_ as (select * from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one))) select lhs.d2 from remote('127.{1,2}', view(select dummy d1, dummy d2 from system.one)) lhs global join rhs_ rhs using (d1) order by rhs.d2 settings joined_subquery_requires_alias=0; diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.sql b/tests/queries/0_stateless/02342_analyzer_compound_types.sql index 0fd96928496..36617aab2f8 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.sql +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 'Constant tuple'; diff --git a/tests/queries/0_stateless/02342_window_view_different_struct.sql b/tests/queries/0_stateless/02342_window_view_different_struct.sql index a5b2b8daa5a..9c35459ecef 100644 --- a/tests/queries/0_stateless/02342_window_view_different_struct.sql +++ b/tests/queries/0_stateless/02342_window_view_different_struct.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS data_02342; diff --git a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql index 7e323c570b8..b55cb85c0da 100644 --- a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql +++ b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.sql b/tests/queries/0_stateless/02343_analyzer_lambdas.sql index 80fa47fc325..07f382700ae 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas.sql +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql index 3b780e1dec3..dff0d7f82ca 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql @@ -1,11 +1,11 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select so, r from - (select [('y',0),('n',1)] as cg, + (select [('y',0),('n',1)] as cg, if( arrayMap( x -> x.1, cg ) != ['y', 'n'], 'y', 'n') as so, - arrayFilter( x -> x.1 = so , cg) as r + arrayFilter( x -> x.1 = so , cg) as r ); select diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql index b07f3f33ac3..da0b4e8ef57 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT arraySum(x -> ((x.1) / ((x.2) * (x.2))), arrayZip(mag, magerr)) / arraySum(x -> (1. / (x * x)), magerr) AS weightedmeanmag, diff --git a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql index ee02b79cc32..3c7ea467734 100644 --- a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql +++ b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02345_analyzer_subqueries.sql b/tests/queries/0_stateless/02345_analyzer_subqueries.sql index c0cc242b57b..d1ec9b58e27 100644 --- a/tests/queries/0_stateless/02345_analyzer_subqueries.sql +++ b/tests/queries/0_stateless/02345_analyzer_subqueries.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql b/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql index a643df65095..e29c3c51e5e 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql @@ -13,8 +13,8 @@ ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab (k) VALUES (0); -SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS allow_experimental_analyzer=1; +SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS enable_analyzer=1; +SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS enable_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS enable_analyzer=1; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql index 7f36c423a41..a20c8dc9afd 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql @@ -31,7 +31,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -42,7 +42,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT '---'; @@ -61,7 +61,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -72,7 +72,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT '---'; @@ -91,7 +91,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -102,6 +102,6 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index db0c06c4e16..f03c36f6550 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -8,7 +8,7 @@ SET allow_experimental_annoy_index = 1; SET allow_experimental_usearch_index = 1; -SET allow_experimental_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof +SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02364_window_view_segfault.sh b/tests/queries/0_stateless/02364_window_view_segfault.sh index 4173b6c6b8d..833d8967558 100755 --- a/tests/queries/0_stateless/02364_window_view_segfault.sh +++ b/tests/queries/0_stateless/02364_window_view_segfault.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=0" + "--enable_analyzer=0" ) ${CLICKHOUSE_CLIENT} "${opts[@]}" --multiline --query """ diff --git a/tests/queries/0_stateless/02366_explain_query_tree.sql b/tests/queries/0_stateless/02366_explain_query_tree.sql index c38b2d819d1..82621ec90f7 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.sql +++ b/tests/queries/0_stateless/02366_explain_query_tree.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE run_passes = 0 SELECT 1; diff --git a/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql index f41680cd9f4..a6408abca9b 100644 --- a/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql +++ b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02368_analyzer_table_functions.sql b/tests/queries/0_stateless/02368_analyzer_table_functions.sql index 456e095c6c1..cc65848a5d8 100644 --- a/tests/queries/0_stateless/02368_analyzer_table_functions.sql +++ b/tests/queries/0_stateless/02368_analyzer_table_functions.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT c1, c2, c3, c4 FROM format('CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"'); SELECT f.c1, f.c2, f.c3, f.c4 FROM format('CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"') AS f; diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql index e60ec7e71a9..f84c96ee6a6 100644 --- a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT arrayJoin([1, 2, 3]); diff --git a/tests/queries/0_stateless/02370_analyzer_in_function.sql b/tests/queries/0_stateless/02370_analyzer_in_function.sql index a6e4400e101..9fb05ada829 100644 --- a/tests/queries/0_stateless/02370_analyzer_in_function.sql +++ b/tests/queries/0_stateless/02370_analyzer_in_function.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 1 IN 1; SELECT 1 IN (1); diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.sql b/tests/queries/0_stateless/02371_analyzer_join_cross.sql index 3624a1d2282..660dc19770a 100644 --- a/tests/queries/0_stateless/02371_analyzer_join_cross.sql +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET single_join_prefer_left_table = 0; DROP TABLE IF EXISTS test_table_join_1; diff --git a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 index 45ae63b9a49..fb75ddeac09 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 +++ b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 @@ -1,6 +1,6 @@ -- Tags: long -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET single_join_prefer_left_table = 0; DROP TABLE IF EXISTS test_table_join_1; diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql index bcec6d178a8..16a37ee7f50 100644 --- a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET join_use_nulls = 1; DROP TABLE IF EXISTS test_table_join_1; diff --git a/tests/queries/0_stateless/02374_analyzer_array_join.sql b/tests/queries/0_stateless/02374_analyzer_array_join.sql index 8c26df1806e..fe1508f5f78 100644 --- a/tests/queries/0_stateless/02374_analyzer_array_join.sql +++ b/tests/queries/0_stateless/02374_analyzer_array_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 index 26fb52716ff..2c64efc7fc8 100644 --- a/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 +++ b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02374_combine_multi_if_and_count_if_opt.sql b/tests/queries/0_stateless/02374_combine_multi_if_and_count_if_opt.sql index 4371c2e5641..05472e5e35a 100644 --- a/tests/queries/0_stateless/02374_combine_multi_if_and_count_if_opt.sql +++ b/tests/queries/0_stateless/02374_combine_multi_if_and_count_if_opt.sql @@ -4,7 +4,7 @@ create table m (a int) engine Log; insert into m values (1); -set allow_experimental_analyzer = true, optimize_rewrite_sum_if_to_count_if=1; +set enable_analyzer = true, optimize_rewrite_sum_if_to_count_if=1; EXPLAIN QUERY TREE select sum(multiIf(a = 1, 1, 0)) from m; diff --git a/tests/queries/0_stateless/02375_analyzer_union.sql b/tests/queries/0_stateless/02375_analyzer_union.sql index 5e41f07d217..cf9a99ffbba 100644 --- a/tests/queries/0_stateless/02375_analyzer_union.sql +++ b/tests/queries/0_stateless/02375_analyzer_union.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql index 72a4edb8567..295d8a8f97e 100644 --- a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql +++ b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02377_analyzer_in_function_set.sql b/tests/queries/0_stateless/02377_analyzer_in_function_set.sql index e3cbcf75a9c..00aa40ccf17 100644 --- a/tests/queries/0_stateless/02377_analyzer_in_function_set.sql +++ b/tests/queries/0_stateless/02377_analyzer_in_function_set.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index c223fcc86bc..4b9793da5bb 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -20,9 +20,9 @@ function explain_sorting { function explain_sortmode { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE + $CLICKHOUSE_CLIENT --enable_analyzer=0 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE echo "-- QUERY (analyzer): "$1 - $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE + $CLICKHOUSE_CLIENT --enable_analyzer=1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE } $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index f41afe6a950..7b3099c4f36 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET single_join_prefer_left_table = 0; DROP TABLE IF EXISTS test_table; diff --git a/tests/queries/0_stateless/02379_analyzer_subquery_depth.sql b/tests/queries/0_stateless/02379_analyzer_subquery_depth.sql index 5699a15aead..40303e0f92a 100644 --- a/tests/queries/0_stateless/02379_analyzer_subquery_depth.sql +++ b/tests/queries/0_stateless/02379_analyzer_subquery_depth.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT (SELECT a FROM (SELECT 1 AS a)) SETTINGS max_subquery_depth = 1; -- { serverError TOO_DEEP_SUBQUERIES } SELECT (SELECT a FROM (SELECT 1 AS a)) SETTINGS max_subquery_depth = 2; diff --git a/tests/queries/0_stateless/02380_analyzer_join_sample.sql b/tests/queries/0_stateless/02380_analyzer_join_sample.sql index e417f47d173..bc77f3623ae 100644 --- a/tests/queries/0_stateless/02380_analyzer_join_sample.sql +++ b/tests/queries/0_stateless/02380_analyzer_join_sample.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02381_analyzer_join_final.sql b/tests/queries/0_stateless/02381_analyzer_join_final.sql index 57fc3aedd8f..0db81ac7728 100644 --- a/tests/queries/0_stateless/02381_analyzer_join_final.sql +++ b/tests/queries/0_stateless/02381_analyzer_join_final.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql.j2 b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql.j2 index ca4af4df6b6..4e88b67e3dd 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql.j2 +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql.j2 @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql index 7983b05a69e..6a0b58e7b28 100644 --- a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql +++ b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql index c22a0f4244b..fbd6fe4db4f 100644 --- a/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql +++ b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql index f4619f20765..a8ad5c4d957 100644 --- a/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql +++ b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql index 861ada9623a..025e064d23c 100644 --- a/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql +++ b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT cast(tuple(1, 'Value'), 'Tuple(first UInt64, second String)') AS value, value.first, value.second; diff --git a/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql index c8ca3ff21d4..64d24aae1b1 100644 --- a/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql +++ b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT (NULL IN (SELECT 9223372036854775806 IN (SELECT 65536), inf, NULL IN (NULL))) IN (SELECT NULL IN (NULL)); diff --git a/tests/queries/0_stateless/02387_analyzer_cte.sql b/tests/queries/0_stateless/02387_analyzer_cte.sql index 1f10ac10438..149eab7f741 100644 --- a/tests/queries/0_stateless/02387_analyzer_cte.sql +++ b/tests/queries/0_stateless/02387_analyzer_cte.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql b/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql index 9fd2f73703d..31d6f91a39d 100644 --- a/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql +++ b/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH x -> plus(lambda(1), x) AS lambda SELECT lambda(1048576); -- { serverError UNSUPPORTED_METHOD }; diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql index 8e3777ebc15..51a10adae6a 100644 --- a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_final_setting_analyzer.reference index 780a6e5de68..763ecad84ab 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.reference +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.reference @@ -1,5 +1,5 @@ -- { echoOn } -set allow_experimental_analyzer=1; +set enable_analyzer=1; SYSTEM STOP MERGES tbl; -- simple test case create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.sql b/tests/queries/0_stateless/02420_final_setting_analyzer.sql index cbdec017602..89fff094825 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.sql +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.sql @@ -1,5 +1,5 @@ -- { echoOn } -set allow_experimental_analyzer=1; +set enable_analyzer=1; SYSTEM STOP MERGES tbl; -- simple test case diff --git a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql index fde893626c1..5a4e23c10bd 100644 --- a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql +++ b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql @@ -21,7 +21,7 @@ SELECT count() == 1 FROM dtest WHERE b IN toDecimal64('44.4000', 4); SELECT count() == 1 FROM dtest WHERE b IN toDecimal128('44.4000', 4); SELECT count() == 1 FROM dtest WHERE b IN toDecimal256('44.4000', 4); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT count() == 0 FROM (SELECT '33.3' :: Decimal(9, 1) AS a WHERE a IN ('33.33' :: Decimal(9, 2))); diff --git a/tests/queries/0_stateless/02421_explain_subquery.sql b/tests/queries/0_stateless/02421_explain_subquery.sql index 2970003cb1c..02f45e8cc3b 100644 --- a/tests/queries/0_stateless/02421_explain_subquery.sql +++ b/tests/queries/0_stateless/02421_explain_subquery.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; @@ -32,7 +32,7 @@ SELECT count() == 1 FROM (EXPLAIN ESTIMATE SELECT sum(a) FROM t1); DROP TABLE t1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: \_\_table1.number UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; diff --git a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql index a84cb5572ba..a0d92115188 100644 --- a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql +++ b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql @@ -30,7 +30,7 @@ SELECT count() == 1 FROM decimal_in_float_test WHERE a NOT IN (33.333); SELECT count() == 1 FROM decimal_in_float_test WHERE b IN (44.44); SELECT count() == 1 FROM decimal_in_float_test WHERE b NOT IN (44.4,44.444); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toDecimal32(1.555,3) IN (1.5551); diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index c6f0927db36..95f82db5454 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -72,8 +72,8 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog VALUES $CLICKHOUSE_CLIENT -q "INSERT INTO ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog VALUES ('Paper', 20, 1)" $CLICKHOUSE_CLIENT -q "CREATE VIEW ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1 AS SELECT * FROM ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog WHERE Price={price:UInt64}" $CLICKHOUSE_CLIENT -q "SELECT Price FROM ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1(price=20)" -$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -Fq "UNKNOWN_FUNCTION" && echo 'ERROR' || echo 'OK' -$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS allow_experimental_analyzer = 1" +$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS enable_analyzer = 0" 2>&1 | grep -Fq "UNKNOWN_FUNCTION" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(price=20) SETTINGS enable_analyzer = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book2', 30, 8)" diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.sh b/tests/queries/0_stateless/02451_order_by_monotonic.sh index 7d1356b4445..fa0a37678f0 100755 --- a/tests/queries/0_stateless/02451_order_by_monotonic.sh +++ b/tests/queries/0_stateless/02451_order_by_monotonic.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh opts=( - "--allow_experimental_analyzer=1" + "--enable_analyzer=1" ) function explain_sort_description() diff --git a/tests/queries/0_stateless/02459_group_by_all.sql b/tests/queries/0_stateless/02459_group_by_all.sql index 4f08ee331a4..8281c201edf 100644 --- a/tests/queries/0_stateless/02459_group_by_all.sql +++ b/tests/queries/0_stateless/02459_group_by_all.sql @@ -21,7 +21,7 @@ select substring(a, 1, 3), substring(substring(substring(a, c, count(b)), 1, cou select substring(a, 1, 3), substring(a, 1, count(b)) from group_by_all group by all; select count(b) AS len, substring(a, 1, 3), substring(a, 1, len) from group_by_all group by all; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select a, count(b) from group_by_all group by all order by a; select substring(a, 1, 3), count(b) from group_by_all group by all; diff --git a/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql b/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql index 5ac8c79d4ed..e86c867043d 100644 --- a/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql +++ b/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM (SELECT 1) FINAL; -- { serverError UNSUPPORTED_METHOD } SELECT * FROM (SELECT 1) SAMPLE 1/2; -- { serverError UNSUPPORTED_METHOD } diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql index eda90529166..c9e7ac19157 100644 --- a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT * FROM subquery; diff --git a/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql index f96c834c057..fc9e9d44b42 100644 --- a/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql +++ b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT cast(tuple(1, 2), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, value.value_1, value.value_2; diff --git a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql index ddb0f70c6de..53b97a1dd2a 100644 --- a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql +++ b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql @@ -1,2 +1,2 @@ SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1; -SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1 SETTINGS allow_experimental_analyzer=1; \ No newline at end of file +SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1 SETTINGS enable_analyzer=1; diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh index 0702c146426..4c850a6ec9e 100755 --- a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh @@ -17,61 +17,61 @@ $CLICKHOUSE_CLIENT -n -q " INSERT INTO test_table VALUES (0, 'Value'); "; -$CLICKHOUSE_CLIENT -q "SELECT value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT value_ FROM test_table SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table AS test_table_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table AS test_table_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_alias.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_alias.value_ FROM test_table AS test_table_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_alias.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_alia.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_alia.value_ FROM test_table AS test_table_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_alias.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT value_ FROM (SELECT 1 AS value) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT value_ FROM (SELECT 1 AS value) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT subquery.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT subquery.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['subquery.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT subquer.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT subquer.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['subquery.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery.value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquer.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquer.value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alias.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alias.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery_alias.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alia.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alia.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery_alias.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_valu, [1, 2, 3]) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_valu, [1, 2, 3]) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; $CLICKHOUSE_CLIENT -n -q " @@ -85,61 +85,61 @@ $CLICKHOUSE_CLIENT -n -q " INSERT INTO test_table_compound VALUES (0, tuple('Value_1')); "; -$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM test_table_compound SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alias.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alias.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound_alias.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alia.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alia.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_compound_alias.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT subquery.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT subquery.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['subquery.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT subquer.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT subquer.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['subquery.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT value.value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery.value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery.value.value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquer.value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquer.value.value_ FROM cte_subquery SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alias.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alias.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery_alias.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alia.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alia.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['cte_subquery_alias.value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_value.value_ SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_value.value_ SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_valu.value_ SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_valu.value_ SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_value.value_, [1, 2, 3]) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_value.value_, [1, 2, 3]) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; $CLICKHOUSE_CLIENT -n -q " @@ -162,25 +162,25 @@ $CLICKHOUSE_CLIENT -n -q " INSERT INTO test_table_2 VALUES (0, 'Value'); "; -$CLICKHOUSE_CLIENT -q "SELECT test_table_1.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_1.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_1.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT test_table_2.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_table_2.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_table_2.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT t1.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT t1.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['t1.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT t2.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT t2.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['t2.value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT [1] AS a, a.size1 SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT [1] AS a, a.size1 SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['a.size0'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.c SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.c SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['t.a'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['t.a.b'\]" &>/dev/null; $CLICKHOUSE_CLIENT -q "SELECT 1"; diff --git a/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql index ca937e01238..feb6786ffb6 100644 --- a/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql +++ b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT subquery_1.id, subquery_2.id FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.sql b/tests/queries/0_stateless/02476_fuse_sum_count.sql index 315bbd10a65..2319e81440b 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.sql +++ b/tests/queries/0_stateless/02476_fuse_sum_count.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_syntax_fuse_functions = 1; DROP TABLE IF EXISTS fuse_tbl; diff --git a/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql index 7e4ecb98ae8..3d2cc1b1620 100644 --- a/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql +++ b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql index 53f3a9b23ec..2fc1cc45ce0 100644 --- a/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql +++ b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.sh b/tests/queries/0_stateless/02477_analyzer_function_hints.sh index f83935e47fb..23cc5651257 100755 --- a/tests/queries/0_stateless/02477_analyzer_function_hints.sh +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.sh @@ -6,22 +6,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['plus'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['uniqExact'\]" &>/dev/null; $CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function_$CLICKHOUSE_DATABASE;" $CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE AS x -> x + 1;" -$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_function_${CLICKHOUSE_DATABASE}A(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_function_${CLICKHOUSE_DATABASE}A(1) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep -E "Maybe you meant: \[.*'test_user_defined_function_$CLICKHOUSE_DATABASE'.*\]" &>/dev/null; $CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function_$CLICKHOUSE_DATABASE"; -$CLICKHOUSE_CLIENT -q "WITH (x -> x + 1) AS lambda_function SELECT lambda_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "WITH (x -> x + 1) AS lambda_function SELECT lambda_functio(1) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['lambda_function'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -q "SELECT row_numbe() OVER (PARTITION BY 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ +$CLICKHOUSE_CLIENT -q "SELECT row_numbe() OVER (PARTITION BY 1) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['row_number'\]" &>/dev/null; $CLICKHOUSE_CLIENT -q "SELECT 1"; diff --git a/tests/queries/0_stateless/02477_exists_fuzz_43478.sql b/tests/queries/0_stateless/02477_exists_fuzz_43478.sql index 8ec876eb252..c225befed0e 100644 --- a/tests/queries/0_stateless/02477_exists_fuzz_43478.sql +++ b/tests/queries/0_stateless/02477_exists_fuzz_43478.sql @@ -1,3 +1,3 @@ create table test_rows_compact_part__fuzz_11 (x UInt32) engine = MergeTree order by x; insert into test_rows_compact_part__fuzz_11 select 1; -select 1 from test_rows_compact_part__fuzz_11 where exists(select 1) settings allow_experimental_analyzer=1; +select 1 from test_rows_compact_part__fuzz_11 where exists(select 1) settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.sql b/tests/queries/0_stateless/02477_fuse_quantiles.sql index c0719d771d7..8ddc029f75f 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.sql +++ b/tests/queries/0_stateless/02477_fuse_quantiles.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_syntax_fuse_functions = 1; DROP TABLE IF EXISTS fuse_tbl; diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 11c178ac0d0..0ce90cde108 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -19,7 +19,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (a = \'x\') OR (\'y\' = a) @@ -41,7 +41,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (a != \'x\') AND (a != \'y\') @@ -63,7 +63,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (a != \'x\') AND (\'y\' != a) @@ -85,7 +85,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (b = 0) OR (b = 1) @@ -111,7 +111,7 @@ QUERY id: 0 LIST id: 11, nodes: 2 COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 12, constant_value: UInt64_1, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (b != 0) AND (b != 1) @@ -137,4 +137,4 @@ QUERY id: 0 LIST id: 11, nodes: 2 COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 12, constant_value: UInt64_1, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql index 976b21a7e29..b328e9658d0 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql @@ -5,21 +5,21 @@ CREATE TABLE t_logical_expressions_optimizer_low_cardinality (a LowCardinality(S -- LowCardinality case, ignore optimize_min_equality_disjunction_chain_length limit, optimizer applied -- Chain of OR equals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y'; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y' SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a SETTINGS enable_analyzer = 1; -- Chain of AND notEquals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND a <> 'y'; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND a <> 'y' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND a <> 'y' SETTINGS enable_analyzer = 1; EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND 'y' <> a; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND 'y' <> a SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND 'y' <> a SETTINGS enable_analyzer = 1; -- Non-LowCardinality case, optimizer not applied for short chains -- Chain of OR equals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1 SETTINGS enable_analyzer = 1; -- Chain of AND notEquals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b <> 0 AND b <> 1; -EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b <> 0 AND b <> 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b <> 0 AND b <> 1 SETTINGS enable_analyzer = 1; DROP TABLE t_logical_expressions_optimizer_low_cardinality; diff --git a/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql index 66f50f7b26d..a1eb88c634d 100644 --- a/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql +++ b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02478_window_frame_type_groups.sql b/tests/queries/0_stateless/02478_window_frame_type_groups.sql index f762bcb61ee..a01e1813cc1 100644 --- a/tests/queries/0_stateless/02478_window_frame_type_groups.sql +++ b/tests/queries/0_stateless/02478_window_frame_type_groups.sql @@ -1,7 +1,7 @@ -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT toUInt64(dense_rank(1) OVER (ORDER BY 100 ASC GROUPS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)) FROM numbers(10); -- { serverError NOT_IMPLEMENTED } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toUInt64(dense_rank(1) OVER (ORDER BY 100 ASC GROUPS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)) FROM numbers(10); -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql index c931a3ab634..1e890740665 100644 --- a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET compile_aggregate_expressions = 1; SET min_count_to_compile_aggregate_expression = 0; diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql index 6cd3e6a9385..004e61ee1a2 100644 --- a/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT anyLast(number) FROM numbers(1) GROUP BY number WITH ROLLUP WITH TOTALS; diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql index 9f77cf39f47..503bbe63347 100644 --- a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 1; diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index cf2220073d3..b8ed7e0c03e 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS foo; CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; INSERT INTO foo VALUES (1, 'one', -1, 'een'), (2, 'two', -2, 'twee'), (3, 'three', -3, 'drie'), (4, 'four', -4, 'vier'), (5, 'five', -5, 'vijf'); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT '---'; SELECT * FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) ORDER BY key; diff --git a/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql b/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql index f6b381e5c70..07503de1b10 100644 --- a/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql +++ b/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT min(b), x AS b FROM (SELECT max(number) FROM numbers(1)); -- { serverError UNKNOWN_IDENTIFIER } diff --git a/tests/queries/0_stateless/02480_tlp_nan.reference b/tests/queries/0_stateless/02480_tlp_nan.reference index befd1f66564..29e5bffa7f0 100644 --- a/tests/queries/0_stateless/02480_tlp_nan.reference +++ b/tests/queries/0_stateless/02480_tlp_nan.reference @@ -1,21 +1,21 @@ -- {echo} -SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; nan 0 1 0 -SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; nan 0 1 0 -SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; -inf 0 1 0 -SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; -inf 0 1 0 -SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; \N \N \N 1 -SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; \N \N \N 1 -SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; inf 0 1 0 -SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; inf 0 1 0 -SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; nan 0 1 0 -SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; nan 0 1 0 diff --git a/tests/queries/0_stateless/02480_tlp_nan.sql b/tests/queries/0_stateless/02480_tlp_nan.sql index e24bc9a9830..55318e0cb30 100644 --- a/tests/queries/0_stateless/02480_tlp_nan.sql +++ b/tests/queries/0_stateless/02480_tlp_nan.sql @@ -1,15 +1,15 @@ -- {echo} -SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; -SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; -SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; -SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; -SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=1; +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS enable_analyzer=0; diff --git a/tests/queries/0_stateless/02481_aggregation_in_order_plan.sql b/tests/queries/0_stateless/02481_aggregation_in_order_plan.sql index 490060dee6f..139e0ed4b77 100644 --- a/tests/queries/0_stateless/02481_aggregation_in_order_plan.sql +++ b/tests/queries/0_stateless/02481_aggregation_in_order_plan.sql @@ -5,5 +5,5 @@ insert into tab select 0, number % 3, 2 - intDiv(number, 3), (number % 3 + 1) * insert into tab select 0, number % 3, 2 - intDiv(number, 3), (number % 3 + 1) * 100 from numbers(6); select a, any(b), c, d from tab where b = 1 group by a, c, d order by c, d settings optimize_aggregation_in_order=1, query_plan_aggregation_in_order=1; -select * from (explain actions = 1, sorting=1 select a, any(b), c, d from tab where b = 1 group by a, c, d settings optimize_aggregation_in_order=1, query_plan_aggregation_in_order=1) where explain like '%ReadFromMergeTree%' or explain like '%Aggregating%' or explain like '%Order:%' settings allow_experimental_analyzer=0; -select * from (explain actions = 1, sorting=1 select a, any(b), c, d from tab where b = 1 group by a, c, d settings optimize_aggregation_in_order=1, query_plan_aggregation_in_order=1) where explain like '%ReadFromMergeTree%' or explain like '%Aggregating%' or explain like '%Order:%' settings allow_experimental_analyzer=1; +select * from (explain actions = 1, sorting=1 select a, any(b), c, d from tab where b = 1 group by a, c, d settings optimize_aggregation_in_order=1, query_plan_aggregation_in_order=1) where explain like '%ReadFromMergeTree%' or explain like '%Aggregating%' or explain like '%Order:%' settings enable_analyzer=0; +select * from (explain actions = 1, sorting=1 select a, any(b), c, d from tab where b = 1 group by a, c, d settings optimize_aggregation_in_order=1, query_plan_aggregation_in_order=1) where explain like '%ReadFromMergeTree%' or explain like '%Aggregating%' or explain like '%Order:%' settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/02481_analyzer_join_alias_unknown_identifier_crash.sql b/tests/queries/0_stateless/02481_analyzer_join_alias_unknown_identifier_crash.sql index 0c5f0eba750..8b5b272f547 100644 --- a/tests/queries/0_stateless/02481_analyzer_join_alias_unknown_identifier_crash.sql +++ b/tests/queries/0_stateless/02481_analyzer_join_alias_unknown_identifier_crash.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; CREATE TABLE test_table_join_1 diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.sql b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.sql index ca91d137bf4..e68de0af522 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.sql +++ b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_arithmetic_operations_in_aggregate_functions = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql index fef71fdf94f..8e6b132f5b7 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql +++ b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set optimize_syntax_fuse_functions = 0; EXPLAIN QUERY TREE run_passes=1 diff --git a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql index 6d64d2685b7..0cf95d10a88 100644 --- a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql +++ b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02483_elapsed_time.sh b/tests/queries/0_stateless/02483_elapsed_time.sh index fdb23d6da01..274b30cefb9 100755 --- a/tests/queries/0_stateless/02483_elapsed_time.sh +++ b/tests/queries/0_stateless/02483_elapsed_time.sh @@ -17,7 +17,7 @@ EXCEPTION_BEFORE_START_QUERY="WITH FROM system.numbers WHERE number IN (sub) ) - SETTINGS enable_global_with_statement = 0, allow_experimental_analyzer = 1" + SETTINGS enable_global_with_statement = 0, enable_analyzer = 1" # For this query the system.query_log needs to show ExceptionBeforeStart and elapsed seconds <= 1.0 diff --git a/tests/queries/0_stateless/02489_analyzer_indexes.sql b/tests/queries/0_stateless/02489_analyzer_indexes.sql index b5438ddd31a..dcf18016da6 100644 --- a/tests/queries/0_stateless/02489_analyzer_indexes.sql +++ b/tests/queries/0_stateless/02489_analyzer_indexes.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table @@ -59,4 +59,3 @@ WHERE t1.id = 1 AND t1.value_1 = '1' AND t1.value_2 = '1' AND t1.value_3 = '1' SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx, value_3_idx'; DROP TABLE test_table; - diff --git a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.sql b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.sql index f1dbfa1f32a..171e080961a 100644 --- a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.sql +++ b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_rewrite_sum_if_to_count_if = 1; EXPLAIN QUERY TREE (SELECT sumIf(1, (number % 2) == 0) FROM numbers(10)); diff --git a/tests/queries/0_stateless/02493_analyzer_table_functions_untuple.sql b/tests/queries/0_stateless/02493_analyzer_table_functions_untuple.sql index bdbe65c643b..c9687783dc9 100644 --- a/tests/queries/0_stateless/02493_analyzer_table_functions_untuple.sql +++ b/tests/queries/0_stateless/02493_analyzer_table_functions_untuple.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT number FROM numbers(untuple(tuple(1))); diff --git a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.sql b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.sql index 5a3b2379fde..ca37c6f3833 100644 --- a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.sql +++ b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq = 1; +SET enable_analyzer = 1, optimize_injective_functions_inside_uniq = 1; -- Simple test EXPLAIN QUERY TREE SELECT uniqCombined(tuple('')) FROM numbers(1); diff --git a/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql b/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql index 3e6f9f42724..20b0bdd46b0 100644 --- a/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql +++ b/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( diff --git a/tests/queries/0_stateless/02494_query_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql index bf376b47fdb..decdd92c477 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.sql +++ b/tests/queries/0_stateless/02494_query_cache_explain.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET query_cache_system_table_handling = 'save'; SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 24803ed7420..9a07ddf0386 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -15,8 +15,8 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" -SETTINGS_NO_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" -SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_NO_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, enable_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, enable_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. diff --git a/tests/queries/0_stateless/02495_analyzer_storage_join.sql b/tests/queries/0_stateless/02495_analyzer_storage_join.sql index 7e6c03971f9..1148cd68338 100644 --- a/tests/queries/0_stateless/02495_analyzer_storage_join.sql +++ b/tests/queries/0_stateless/02495_analyzer_storage_join.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS tj; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET single_join_prefer_left_table = 0; CREATE TABLE tj (key2 UInt64, key1 Int64, a UInt64, b UInt64, x UInt64, y UInt64) ENGINE = Join(ALL, RIGHT, key1, key2); diff --git a/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql index 0791b374668..c00c0ba4e07 100644 --- a/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql +++ b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql @@ -1,4 +1,3 @@ select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=0; -select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, allow_experimental_analyzer=0; -select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, allow_experimental_analyzer=1; - +select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, enable_analyzer=0; +select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, enable_analyzer=1; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index 661b32fce72..646e2501a99 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -8,8 +8,8 @@ if [ -z ${ENABLE_ANALYZER+x} ]; then ENABLE_ANALYZER=0 fi -DISABLE_OPTIMIZATION="SET allow_experimental_analyzer=$ENABLE_ANALYZER;SET query_plan_remove_redundant_sorting=0;SET optimize_duplicate_order_by_and_distinct=0" -ENABLE_OPTIMIZATION="SET allow_experimental_analyzer=$ENABLE_ANALYZER;SET query_plan_remove_redundant_sorting=1;SET optimize_duplicate_order_by_and_distinct=0" +DISABLE_OPTIMIZATION="SET enable_analyzer=$ENABLE_ANALYZER;SET query_plan_remove_redundant_sorting=0;SET optimize_duplicate_order_by_and_distinct=0" +ENABLE_OPTIMIZATION="SET enable_analyzer=$ENABLE_ANALYZER;SET query_plan_remove_redundant_sorting=1;SET optimize_duplicate_order_by_and_distinct=0" echo "-- Disabled query_plan_remove_redundant_sorting" echo "-- ORDER BY clauses in subqueries are untouched" diff --git a/tests/queries/0_stateless/02497_analyzer_sum_if_count_if_pass_crash_fix.sql b/tests/queries/0_stateless/02497_analyzer_sum_if_count_if_pass_crash_fix.sql index 51522565014..7533a333225 100644 --- a/tests/queries/0_stateless/02497_analyzer_sum_if_count_if_pass_crash_fix.sql +++ b/tests/queries/0_stateless/02497_analyzer_sum_if_count_if_pass_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_rewrite_sum_if_to_count_if = 1; SELECT sum(if((number % 2) = 0 AS cond_expr, 1 AS one_expr, 0 AS zero_expr) AS if_expr), sum(cond_expr), sum(if_expr), one_expr, zero_expr FROM numbers(100); diff --git a/tests/queries/0_stateless/02497_having_without_actual_aggregation_bug.sql b/tests/queries/0_stateless/02497_having_without_actual_aggregation_bug.sql index b28cbd4861e..e5fd26e484e 100644 --- a/tests/queries/0_stateless/02497_having_without_actual_aggregation_bug.sql +++ b/tests/queries/0_stateless/02497_having_without_actual_aggregation_bug.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select number from numbers_mt(10) having number >= 9; diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql index 131eac390f1..cc2d839fbc1 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_if_transform_strings_to_enum = 1; SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') FROM system.numbers LIMIT 10; diff --git a/tests/queries/0_stateless/02497_storage_join_right_assert.sql b/tests/queries/0_stateless/02497_storage_join_right_assert.sql index 8f9134e9504..eabaa236379 100644 --- a/tests/queries/0_stateless/02497_storage_join_right_assert.sql +++ b/tests/queries/0_stateless/02497_storage_join_right_assert.sql @@ -7,8 +7,8 @@ CREATE TABLE t2 (key UInt64, a UInt64) ENGINE = Join(ALL, RIGHT, key); INSERT INTO t1 VALUES (1, 1), (2, 2); INSERT INTO t2 VALUES (2, 2), (3, 3); -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT * FROM t1 ALL RIGHT JOIN t2 USING (key) ORDER BY key; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM t1 ALL RIGHT JOIN t2 USING (key) ORDER BY key; diff --git a/tests/queries/0_stateless/02498_analyzer_aggregate_functions_arithmetic_operations_pass_fix.sql b/tests/queries/0_stateless/02498_analyzer_aggregate_functions_arithmetic_operations_pass_fix.sql index 8491018eb72..76c44f9e0cb 100644 --- a/tests/queries/0_stateless/02498_analyzer_aggregate_functions_arithmetic_operations_pass_fix.sql +++ b/tests/queries/0_stateless/02498_analyzer_aggregate_functions_arithmetic_operations_pass_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_arithmetic_operations_in_aggregate_functions = 1; DROP TABLE IF EXISTS test_table; diff --git a/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql b/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql index 67623869f0a..472ab358d97 100644 --- a/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql +++ b/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_functions_to_subcolumns = 0; DROP TABLE IF EXISTS test_table; diff --git a/tests/queries/0_stateless/02498_storage_join_key_positions.sql.j2 b/tests/queries/0_stateless/02498_storage_join_key_positions.sql.j2 index e2dad61a93e..e814b8eaf3f 100644 --- a/tests/queries/0_stateless/02498_storage_join_key_positions.sql.j2 +++ b/tests/queries/0_stateless/02498_storage_join_key_positions.sql.j2 @@ -13,7 +13,7 @@ INSERT INTO tjj VALUES (11, 11, 11, 1000), (21, 21, 21, 2000), (31, 31, 31, 3000 {% for use_analyzer in [0, 1] -%} -SET allow_experimental_analyzer = '{{ use_analyzer }}'; +SET enable_analyzer = '{{ use_analyzer }}'; SELECT '--- using ---'; SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, key3) ORDER BY key1; diff --git a/tests/queries/0_stateless/02499_analyzer_aggregate_function_lambda_crash_fix.sql b/tests/queries/0_stateless/02499_analyzer_aggregate_function_lambda_crash_fix.sql index f2698512112..7ac817aecde 100644 --- a/tests/queries/0_stateless/02499_analyzer_aggregate_function_lambda_crash_fix.sql +++ b/tests/queries/0_stateless/02499_analyzer_aggregate_function_lambda_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT count((t, x_0, x_1) -> ((key_2, x_0, x_1) IN (NULL, NULL, '0.3'))) FROM numbers(10); -- { serverError UNSUPPORTED_METHOD } SELECT count((t, x_0, x_1) -> ((key_2, x_0, x_1) IN (NULL, NULL, '0.3'))) OVER (PARTITION BY id) FROM numbers(10); -- { serverError UNSUPPORTED_METHOD } diff --git a/tests/queries/0_stateless/02499_analyzer_set_index.sql b/tests/queries/0_stateless/02499_analyzer_set_index.sql index f90ae61541f..52d96cfcabf 100644 --- a/tests/queries/0_stateless/02499_analyzer_set_index.sql +++ b/tests/queries/0_stateless/02499_analyzer_set_index.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02500_analyzer_storage_view_crash_fix.sql b/tests/queries/0_stateless/02500_analyzer_storage_view_crash_fix.sql index 8f4d14b95cc..f0484a68566 100644 --- a/tests/queries/0_stateless/02500_analyzer_storage_view_crash_fix.sql +++ b/tests/queries/0_stateless/02500_analyzer_storage_view_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh index c4f0994cd13..3c06119e8d2 100755 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh @@ -9,8 +9,8 @@ if [ -z ${ENABLE_ANALYZER+x} ]; then fi OPTIMIZATION_SETTING="query_plan_remove_redundant_distinct" -DISABLE_OPTIMIZATION="set allow_experimental_analyzer=$ENABLE_ANALYZER;SET $OPTIMIZATION_SETTING=0;SET optimize_duplicate_order_by_and_distinct=0" -ENABLE_OPTIMIZATION="set allow_experimental_analyzer=$ENABLE_ANALYZER;SET $OPTIMIZATION_SETTING=1;SET optimize_duplicate_order_by_and_distinct=0" +DISABLE_OPTIMIZATION="set enable_analyzer=$ENABLE_ANALYZER;SET $OPTIMIZATION_SETTING=0;SET optimize_duplicate_order_by_and_distinct=0" +ENABLE_OPTIMIZATION="set enable_analyzer=$ENABLE_ANALYZER;SET $OPTIMIZATION_SETTING=1;SET optimize_duplicate_order_by_and_distinct=0" echo "-- Disabled $OPTIMIZATION_SETTING" query="SELECT DISTINCT * diff --git a/tests/queries/0_stateless/02501_analyzer_expired_context_crash_fix.sql b/tests/queries/0_stateless/02501_analyzer_expired_context_crash_fix.sql index b9ec14501bd..e2c940c829a 100644 --- a/tests/queries/0_stateless/02501_analyzer_expired_context_crash_fix.sql +++ b/tests/queries/0_stateless/02501_analyzer_expired_context_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02502_analyzer_insert_select_crash_fix.sql b/tests/queries/0_stateless/02502_analyzer_insert_select_crash_fix.sql index 4643f65988a..a438276bda5 100644 --- a/tests/queries/0_stateless/02502_analyzer_insert_select_crash_fix.sql +++ b/tests/queries/0_stateless/02502_analyzer_insert_select_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02503_join_switch_alias_fuzz.sql b/tests/queries/0_stateless/02503_join_switch_alias_fuzz.sql index 28d64bf3881..113a8493de6 100644 --- a/tests/queries/0_stateless/02503_join_switch_alias_fuzz.sql +++ b/tests/queries/0_stateless/02503_join_switch_alias_fuzz.sql @@ -1,4 +1,4 @@ SELECT * FROM (SELECT 1 AS id, '' AS test) AS a LEFT JOIN (SELECT test, 1 AS id, NULL AS test) AS b ON b.id = a.id -SETTINGS join_algorithm = 'auto', max_rows_in_join = 1, allow_experimental_analyzer = 1 +SETTINGS join_algorithm = 'auto', max_rows_in_join = 1, enable_analyzer = 1 ; diff --git a/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql b/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql index fb50ea2c4ca..e54252b5c5f 100644 --- a/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql +++ b/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toUInt64(NULL) AS x FROM (SELECT 1) HAVING x IN (SELECT NULL FROM (SELECT x IN (SELECT x IN (SELECT 1), x IN (SELECT 1) FROM (SELECT 1 WHERE x IN (SELECT NULL FROM (SELECT NULL)))))); diff --git a/tests/queries/0_stateless/02513_analyzer_sort_msan.sql b/tests/queries/0_stateless/02513_analyzer_sort_msan.sql index e5beccaff2a..b86a15e9ef0 100644 --- a/tests/queries/0_stateless/02513_analyzer_sort_msan.sql +++ b/tests/queries/0_stateless/02513_analyzer_sort_msan.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS products; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE products (`price` UInt32) ENGINE = Memory; INSERT INTO products VALUES (1); diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index 2406be13aa8..df84e2f50b2 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -15,7 +15,7 @@ INSERT INTO c VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02515_analyzer_null_for_empty.sql b/tests/queries/0_stateless/02515_analyzer_null_for_empty.sql index de21e9b475e..e12f215743b 100644 --- a/tests/queries/0_stateless/02515_analyzer_null_for_empty.sql +++ b/tests/queries/0_stateless/02515_analyzer_null_for_empty.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET aggregate_functions_null_for_empty = 1; SELECT max(aggr) FROM (SELECT max('92233720368547758.06') AS aggr FROM system.one); diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql.j2 b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql.j2 index 09447dfce65..a199165a38f 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql.j2 +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql.j2 @@ -13,7 +13,7 @@ INNER JOIN GROUP BY 1 WITH TOTALS ) AS t2 USING (a) -SETTINGS allow_experimental_analyzer=0; +SETTINGS enable_analyzer=0; SELECT * FROM @@ -30,7 +30,7 @@ INNER JOIN GROUP BY 1 WITH TOTALS ) AS t2 USING (a) -SETTINGS allow_experimental_analyzer=1; +SETTINGS enable_analyzer=1; SELECT a FROM diff --git a/tests/queries/0_stateless/02516_projections_and_context.sql b/tests/queries/0_stateless/02516_projections_and_context.sql index 334544eb4fa..ec14fc0aa9e 100644 --- a/tests/queries/0_stateless/02516_projections_and_context.sql +++ b/tests/queries/0_stateless/02516_projections_and_context.sql @@ -1,10 +1,10 @@ DROP TABLE IF EXISTS test1__fuzz_37; CREATE TABLE test1__fuzz_37 (`i` Date) ENGINE = MergeTree ORDER BY i; insert into test1__fuzz_37 values ('2020-10-10'); -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; SELECT count() FROM test1__fuzz_37 GROUP BY dictHas(NULL, (dictHas(NULL, (('', materialize(NULL)), materialize(NULL))), 'KeyKey')), dictHas('test_dictionary', tuple(materialize('Ke\0'))), tuple(dictHas(NULL, (tuple('Ke\0Ke\0Ke\0Ke\0Ke\0Ke\0\0\0\0Ke\0'), materialize(NULL)))), 'test_dicti\0nary', (('', materialize(NULL)), dictHas(NULL, (dictHas(NULL, tuple(materialize(NULL))), 'KeyKeyKeyKeyKeyKeyKeyKey')), materialize(NULL)); -- { serverError BAD_ARGUMENTS } SELECT count() FROM test1__fuzz_37 GROUP BY dictHas('non_existing_dictionary', materialize('a')); -- { serverError BAD_ARGUMENTS } -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; SELECT count() FROM test1__fuzz_37 GROUP BY dictHas(NULL, (dictHas(NULL, (('', materialize(NULL)), materialize(NULL))), 'KeyKey')), dictHas('test_dictionary', tuple(materialize('Ke\0'))), tuple(dictHas(NULL, (tuple('Ke\0Ke\0Ke\0Ke\0Ke\0Ke\0\0\0\0Ke\0'), materialize(NULL)))), 'test_dicti\0nary', (('', materialize(NULL)), dictHas(NULL, (dictHas(NULL, tuple(materialize(NULL))), 'KeyKeyKeyKeyKeyKeyKeyKey')), materialize(NULL)); -- { serverError BAD_ARGUMENTS } SELECT count() FROM test1__fuzz_37 GROUP BY dictHas('non_existing_dictionary', materialize('a')); -- { serverError BAD_ARGUMENTS } DROP TABLE test1__fuzz_37; diff --git a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.sql b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.sql index fe882da67cb..4ed13307c29 100644 --- a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.sql +++ b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = true; +set enable_analyzer = true; -- { echoOn } set optimize_rewrite_aggregate_function_with_if = false; diff --git a/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql index 105bce6711c..50bf3cd45e4 100644 --- a/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql +++ b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql b/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql index 7842d47d757..f5d601303a2 100644 --- a/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql +++ b/tests/queries/0_stateless/02521_analyzer_array_join_crash.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02525_analyzer_function_in_crash_fix.sql b/tests/queries/0_stateless/02525_analyzer_function_in_crash_fix.sql index 95b896d38ab..dd1688ad400 100644 --- a/tests/queries/0_stateless/02525_analyzer_function_in_crash_fix.sql +++ b/tests/queries/0_stateless/02525_analyzer_function_in_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02532_analyzer_aggregation_with_rollup.sql b/tests/queries/0_stateless/02532_analyzer_aggregation_with_rollup.sql index 09097eb029f..587ef71df8a 100644 --- a/tests/queries/0_stateless/02532_analyzer_aggregation_with_rollup.sql +++ b/tests/queries/0_stateless/02532_analyzer_aggregation_with_rollup.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT sum(a.number) AS total, diff --git a/tests/queries/0_stateless/02534_analyzer_grouping_function.sql b/tests/queries/0_stateless/02534_analyzer_grouping_function.sql index 3163e03d579..ee1cc1d88d1 100644 --- a/tests/queries/0_stateless/02534_analyzer_grouping_function.sql +++ b/tests/queries/0_stateless/02534_analyzer_grouping_function.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql index 4ae5df9629a..59bbfc96289 100644 --- a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; -- { echoOn } SELECT number, number % 2, sum(number) AS val diff --git a/tests/queries/0_stateless/02535_analyzer_limit_offset.sql b/tests/queries/0_stateless/02535_analyzer_limit_offset.sql index 8f98d823e5c..96aef9557c5 100644 --- a/tests/queries/0_stateless/02535_analyzer_limit_offset.sql +++ b/tests/queries/0_stateless/02535_analyzer_limit_offset.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT number FROM numbers(100) LIMIT 10 OFFSET 10; diff --git a/tests/queries/0_stateless/02538_analyzer_create_table_as_select.sql b/tests/queries/0_stateless/02538_analyzer_create_table_as_select.sql index 168066ce2f9..16634e996fe 100644 --- a/tests/queries/0_stateless/02538_analyzer_create_table_as_select.sql +++ b/tests/queries/0_stateless/02538_analyzer_create_table_as_select.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_data; CREATE TABLE test_table_data diff --git a/tests/queries/0_stateless/02540_analyzer_matcher_alias_materialized_columns.sql b/tests/queries/0_stateless/02540_analyzer_matcher_alias_materialized_columns.sql index cc622dde8fe..58840796c7e 100644 --- a/tests/queries/0_stateless/02540_analyzer_matcher_alias_materialized_columns.sql +++ b/tests/queries/0_stateless/02540_analyzer_matcher_alias_materialized_columns.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh index eccac543215..48b3b27680c 100755 --- a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh +++ b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh @@ -115,7 +115,7 @@ EOL # NOTE: due to [1] we cannot use dot.dot notation, only tupleElement() # # [1]: https://github.com/ClickHouse/ClickHouse/issues/24607 -$CLICKHOUSE_LOCAL --allow_experimental_analyzer=1 "${gharchive_settings[@]}" --structure="${gharchive_structure[*]}" -q " +$CLICKHOUSE_LOCAL --enable_analyzer=1 "${gharchive_settings[@]}" --structure="${gharchive_structure[*]}" -q " SELECT payload.issue.labels.name AS labels, payload.pull_request.merged_by.login AS merged_by diff --git a/tests/queries/0_stateless/02541_analyzer_grouping_sets_crash_fix.sql b/tests/queries/0_stateless/02541_analyzer_grouping_sets_crash_fix.sql index d7af475bbe7..b9aa251bc96 100644 --- a/tests/queries/0_stateless/02541_analyzer_grouping_sets_crash_fix.sql +++ b/tests/queries/0_stateless/02541_analyzer_grouping_sets_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH pow(NULL, 256) AS four SELECT NULL AS two GROUP BY GROUPING SETS ((pow(two, 65536))); diff --git a/tests/queries/0_stateless/02552_analyzer_optimize_group_by_function_keys_crash.sql b/tests/queries/0_stateless/02552_analyzer_optimize_group_by_function_keys_crash.sql index ee9032472a7..85740cd85a2 100644 --- a/tests/queries/0_stateless/02552_analyzer_optimize_group_by_function_keys_crash.sql +++ b/tests/queries/0_stateless/02552_analyzer_optimize_group_by_function_keys_crash.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT NULL GROUP BY tuple('0.0000000007'), count(NULL) OVER (ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -- { serverError ILLEGAL_AGGREGATION }; diff --git a/tests/queries/0_stateless/02553_type_object_analyzer.sql b/tests/queries/0_stateless/02553_type_object_analyzer.sql index 55482a02ed1..eb4e49757cf 100644 --- a/tests/queries/0_stateless/02553_type_object_analyzer.sql +++ b/tests/queries/0_stateless/02553_type_object_analyzer.sql @@ -1,6 +1,6 @@ SET output_format_json_named_tuples_as_objects = 1; SET allow_experimental_object_type = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t_json_analyzer; CREATE TABLE t_json_analyzer (a JSON) ENGINE = Memory; diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql index f926b9037d2..0eed4a8c592 100644 --- a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql @@ -60,7 +60,7 @@ FROM (day_)) ) AS t ) -WHERE type_1 = 'all' settings allow_experimental_analyzer=0; +WHERE type_1 = 'all' settings enable_analyzer=0; -- Query plan with analyzer has less Filter steps (which is more optimal) EXPLAIN PIPELINE @@ -83,7 +83,7 @@ FROM (day_)) ) AS t ) -WHERE type_1 = 'all' settings allow_experimental_analyzer=1; +WHERE type_1 = 'all' settings enable_analyzer=1; SELECT ''; SELECT '---Result---'; @@ -129,7 +129,7 @@ FROM (day_)) ) AS t ) -WHERE day_ = '2023-01-05' settings allow_experimental_analyzer=0; +WHERE day_ = '2023-01-05' settings enable_analyzer=0; -- Query plan with analyzer has less Filter steps (which is more optimal) EXPLAIN PIPELINE @@ -151,6 +151,6 @@ FROM (day_)) ) AS t ) -WHERE day_ = '2023-01-05' settings allow_experimental_analyzer=1; +WHERE day_ = '2023-01-05' settings enable_analyzer=1; DROP TABLE test_grouping_sets_predicate; diff --git a/tests/queries/0_stateless/02560_analyzer_materialized_view.sql b/tests/queries/0_stateless/02560_analyzer_materialized_view.sql index 1f268fe1e16..3fdef366dc9 100644 --- a/tests/queries/0_stateless/02560_analyzer_materialized_view.sql +++ b/tests/queries/0_stateless/02560_analyzer_materialized_view.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02563_analyzer_merge.sql b/tests/queries/0_stateless/02563_analyzer_merge.sql index 217fb7019c4..6c252c22773 100644 --- a/tests/queries/0_stateless/02563_analyzer_merge.sql +++ b/tests/queries/0_stateless/02563_analyzer_merge.sql @@ -1,6 +1,6 @@ -- Tags: no-parallel -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP DATABASE IF EXISTS 02563_db; CREATE DATABASE 02563_db; diff --git a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql index a83cd238982..7032559e066 100644 --- a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql +++ b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02565_analyzer_limit_settings.sql b/tests/queries/0_stateless/02565_analyzer_limit_settings.sql index 7c02c2d0d20..1dd6735e64d 100644 --- a/tests/queries/0_stateless/02565_analyzer_limit_settings.sql +++ b/tests/queries/0_stateless/02565_analyzer_limit_settings.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } SET limit = 0; diff --git a/tests/queries/0_stateless/02566_analyzer_limit_settings_distributed.sql b/tests/queries/0_stateless/02566_analyzer_limit_settings_distributed.sql index 1624344b5a9..a2620f436f4 100644 --- a/tests/queries/0_stateless/02566_analyzer_limit_settings_distributed.sql +++ b/tests/queries/0_stateless/02566_analyzer_limit_settings_distributed.sql @@ -1,6 +1,6 @@ -- Tags: distributed -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 'limit', * FROM remote('127.1', view(SELECT * FROM numbers(10))) SETTINGS limit=5; SELECT 'offset', * FROM remote('127.1', view(SELECT * FROM numbers(10))) SETTINGS offset=5; diff --git a/tests/queries/0_stateless/02567_and_consistency.reference b/tests/queries/0_stateless/02567_and_consistency.reference index e0014f187a8..7e6d1f24f43 100644 --- a/tests/queries/0_stateless/02567_and_consistency.reference +++ b/tests/queries/0_stateless/02567_and_consistency.reference @@ -8,7 +8,7 @@ true ===== 1 ===== -allow_experimental_analyzer +enable_analyzer true #45440 2086579505 0 1 0 0 diff --git a/tests/queries/0_stateless/02567_and_consistency.sql b/tests/queries/0_stateless/02567_and_consistency.sql index 0eeab99e539..0442a6dad7f 100644 --- a/tests/queries/0_stateless/02567_and_consistency.sql +++ b/tests/queries/0_stateless/02567_and_consistency.sql @@ -50,9 +50,9 @@ SELECT 1 and sin(1); SELECT '====='; -SELECT 'allow_experimental_analyzer'; +SELECT 'enable_analyzer'; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT toBool(sin(SUM(number))) AS x FROM diff --git a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.sql b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.sql index 2dade7837b7..486a26613f6 100644 --- a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.sql +++ b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.sql @@ -1,3 +1,3 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN header = 1, actions = 1 SELECT number FROM (SELECT number FROM numbers(2) ORDER BY ignore(2)) WHERE ignore(2); diff --git a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.sql b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.sql index 5233f2f7e3c..b5a123e3767 100644 --- a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.sql +++ b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = true; +set enable_analyzer = true; set optimize_rewrite_array_exists_to_has = false; EXPLAIN QUERY TREE run_passes = 1 select arrayExists(x -> x = 5 , materialize(range(10))) from numbers(10); @@ -8,7 +8,7 @@ set optimize_rewrite_array_exists_to_has = true; EXPLAIN QUERY TREE run_passes = 1 select arrayExists(x -> x = 5 , materialize(range(10))) from numbers(10); EXPLAIN QUERY TREE run_passes = 1 select arrayExists(x -> 5 = x , materialize(range(10))) from numbers(10); -set allow_experimental_analyzer = false; +set enable_analyzer = false; set optimize_rewrite_array_exists_to_has = false; EXPLAIN SYNTAX select arrayExists(x -> x = 5 , materialize(range(10))) from numbers(10); diff --git a/tests/queries/0_stateless/02577_analyzer_array_join_calc_twice.sql b/tests/queries/0_stateless/02577_analyzer_array_join_calc_twice.sql index b6bb258db28..0b281dd4f81 100644 --- a/tests/queries/0_stateless/02577_analyzer_array_join_calc_twice.sql +++ b/tests/queries/0_stateless/02577_analyzer_array_join_calc_twice.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT 1 + arrayJoin(a) AS m FROM (SELECT [1, 2, 3] AS a) GROUP BY m; diff --git a/tests/queries/0_stateless/02579_fill_empty_chunk.sql b/tests/queries/0_stateless/02579_fill_empty_chunk.sql index 30942b154c9..aeae98df7a3 100644 --- a/tests/queries/0_stateless/02579_fill_empty_chunk.sql +++ b/tests/queries/0_stateless/02579_fill_empty_chunk.sql @@ -1,7 +1,7 @@ -- this SELECT produces empty chunk in FillingTransform SET enable_positional_arguments = 0; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT 2 AS x, diff --git a/tests/queries/0_stateless/02579_fill_empty_chunk_analyzer.sql b/tests/queries/0_stateless/02579_fill_empty_chunk_analyzer.sql index 8350173f443..144640149ea 100644 --- a/tests/queries/0_stateless/02579_fill_empty_chunk_analyzer.sql +++ b/tests/queries/0_stateless/02579_fill_empty_chunk_analyzer.sql @@ -1,7 +1,7 @@ -- this SELECT produces empty chunk in FillingTransform SET enable_positional_arguments = 0; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- With analyzer this special query has correct output SELECT diff --git a/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql b/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql index 10e5871cc44..33c9296a0d8 100644 --- a/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql +++ b/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } SELECT a FROM ( select 1 AS a ) AS t1, ( select 2 AS b, 3 AS c) AS t2; @@ -10,5 +10,3 @@ SELECT b FROM ( select 1 AS a UNION ALL select 1 as a ) AS t1, ( select 2 AS b, SELECT c FROM ( select 1 AS a UNION ALL select 1 as a ) AS t1, ( select 2 AS b, 3 AS c UNION ALL select 2 as b, 3 as c) AS t2; SELECT 42 FROM ( select 1 AS a UNION ALL select 1 as a ) AS t1, ( select 2 AS b, 3 AS c UNION ALL select 2 as b, 3 as c) AS t2; SELECT count() FROM ( select 1 AS a UNION ALL select 1 as a ) AS t1, ( select 2 AS b, 3 AS c UNION ALL select 2 as b, 3 as c) AS t2; - - diff --git a/tests/queries/0_stateless/02661_quantile_approx.reference b/tests/queries/0_stateless/02661_quantile_approx.reference index 0ee846a268b..239516f9ac4 100644 --- a/tests/queries/0_stateless/02661_quantile_approx.reference +++ b/tests/queries/0_stateless/02661_quantile_approx.reference @@ -33,10 +33,10 @@ FROM FROM numbers(49999) ); [24902,44518,49999] -select medianGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select medianGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -select quantileGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select quantileGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select medianGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select medianGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select quantileGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select quantileGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select medianGK(100)(number) from numbers(10); 4 select quantileGK(100)(number) from numbers(10); @@ -47,8 +47,8 @@ select quantileGK(100, 0.5, 0.75)(number) from numbers(10); -- { serverError NUM select quantileGK('abc', 0.5)(number) from numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select quantileGK(1.23, 0.5)(number) from numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select quantileGK(-100, 0.5)(number) from numbers(10); -- { serverError BAD_ARGUMENTS } -select quantilesGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select quantilesGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select quantilesGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select quantilesGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select quantilesGK(100)(number) from numbers(10); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select quantilesGK(100, 0.5)(number) from numbers(10); [4] diff --git a/tests/queries/0_stateless/02661_quantile_approx.sql b/tests/queries/0_stateless/02661_quantile_approx.sql index c0004260fa1..732ce645c98 100644 --- a/tests/queries/0_stateless/02661_quantile_approx.sql +++ b/tests/queries/0_stateless/02661_quantile_approx.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; -- { echoOn } with arrayJoin([0, 1, 2, 10]) as x select quantilesGK(100, 0.5, 0.4, 0.1)(x); @@ -29,11 +29,11 @@ FROM FROM numbers(49999) ); -select medianGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select medianGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select medianGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select medianGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -select quantileGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select quantileGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select quantileGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select quantileGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select medianGK(100)(number) from numbers(10); select quantileGK(100)(number) from numbers(10); @@ -43,8 +43,8 @@ select quantileGK('abc', 0.5)(number) from numbers(10); -- { serverError ILLEGAL select quantileGK(1.23, 0.5)(number) from numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select quantileGK(-100, 0.5)(number) from numbers(10); -- { serverError BAD_ARGUMENTS } -select quantilesGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 0; -- { serverError BAD_ARGUMENTS } -select quantilesGK()(number) from numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select quantilesGK()(number) from numbers(10) SETTINGS enable_analyzer = 0; -- { serverError BAD_ARGUMENTS } +select quantilesGK()(number) from numbers(10) SETTINGS enable_analyzer = 1; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select quantilesGK(100)(number) from numbers(10); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select quantilesGK(100, 0.5)(number) from numbers(10); diff --git a/tests/queries/0_stateless/02662_first_last_value.reference b/tests/queries/0_stateless/02662_first_last_value.reference index b0783399623..308cbf850b0 100644 --- a/tests/queries/0_stateless/02662_first_last_value.reference +++ b/tests/queries/0_stateless/02662_first_last_value.reference @@ -18,7 +18,7 @@ select last_value(b) ignore nulls from test; 5 select last_value(b) respect nulls from test; \N -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- first value select first_value(b) from test; 3 diff --git a/tests/queries/0_stateless/02662_first_last_value.sql b/tests/queries/0_stateless/02662_first_last_value.sql index 8e429e2e27d..16768bd6f1e 100644 --- a/tests/queries/0_stateless/02662_first_last_value.sql +++ b/tests/queries/0_stateless/02662_first_last_value.sql @@ -15,7 +15,7 @@ select last_value(b) from test; select last_value(b) ignore nulls from test; select last_value(b) respect nulls from test; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- first value select first_value(b) from test; diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql index 7d624195df9..dabdcfd5507 100644 --- a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS 02668_logical_optimizer; diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.reference b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference index 05feadb58a0..9d7f393e71c 100644 --- a/tests/queries/0_stateless/02674_trivial_count_analyzer.reference +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference @@ -1,5 +1,5 @@ -- { echoOn } -set allow_experimental_analyzer=1; +set enable_analyzer=1; set optimize_trivial_count_query=1; create table m3(a Int64, b UInt64) Engine=MergeTree order by tuple(); select count() from m3; diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.sql b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql index 988d1b9ba92..c13a9dc68f2 100644 --- a/tests/queries/0_stateless/02674_trivial_count_analyzer.sql +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql @@ -2,7 +2,7 @@ drop table if exists m3; drop table if exists replacing_m3; -- { echoOn } -set allow_experimental_analyzer=1; +set enable_analyzer=1; set optimize_trivial_count_query=1; create table m3(a Int64, b UInt64) Engine=MergeTree order by tuple(); diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql index 930127497ae..6c8932b5d58 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET single_join_prefer_left_table = 0; SET optimize_move_to_prewhere = 0; diff --git a/tests/queries/0_stateless/02676_analyzer_limit_offset.sql b/tests/queries/0_stateless/02676_analyzer_limit_offset.sql index 39c6b85f088..5dbe55e916c 100644 --- a/tests/queries/0_stateless/02676_analyzer_limit_offset.sql +++ b/tests/queries/0_stateless/02676_analyzer_limit_offset.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer=1; +set enable_analyzer=1; DROP TABLE IF EXISTS test; CREATE TABLE test (i UInt64) Engine = MergeTree() order by i; diff --git a/tests/queries/0_stateless/02676_distinct_reading_in_order_analyzer.sql b/tests/queries/0_stateless/02676_distinct_reading_in_order_analyzer.sql index f9ff1eed111..6a219cd3781 100644 --- a/tests/queries/0_stateless/02676_distinct_reading_in_order_analyzer.sql +++ b/tests/queries/0_stateless/02676_distinct_reading_in_order_analyzer.sql @@ -1,6 +1,6 @@ drop table if exists t; -set allow_experimental_analyzer=1; +set enable_analyzer=1; create table t (a UInt64, b UInt64) engine=MergeTree() order by (a); insert into t select number % 2, number from numbers(10); diff --git a/tests/queries/0_stateless/02677_analyzer_bitmap_has_any.sql b/tests/queries/0_stateless/02677_analyzer_bitmap_has_any.sql index c06ea009c1d..dc906a92f1e 100644 --- a/tests/queries/0_stateless/02677_analyzer_bitmap_has_any.sql +++ b/tests/queries/0_stateless/02677_analyzer_bitmap_has_any.sql @@ -18,7 +18,7 @@ FROM bitmapHasAny(bitmapBuild([toUInt64(1)]), ( SELECT groupBitmapState(toUInt64(2)) )) has2 -) SETTINGS allow_experimental_analyzer = 0; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +) SETTINGS enable_analyzer = 0; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT '--------------'; @@ -32,5 +32,4 @@ FROM bitmapHasAny(bitmapBuild([toUInt64(1)]), ( SELECT groupBitmapState(toUInt64(2)) )) has2 -) SETTINGS allow_experimental_analyzer = 1; - +) SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql index 6b7fdab8993..90781f70158 100644 --- a/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql +++ b/tests/queries/0_stateless/02677_analyzer_compound_expressions.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH ('a', 'b')::Tuple(c1 String, c2 String) AS t SELECT t.c1, t.c2; diff --git a/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql index 5779821afaa..95665979857 100644 --- a/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql +++ b/tests/queries/0_stateless/02677_get_subcolumn_array_of_tuples.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t_get_subcolumn; diff --git a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.sql b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.sql index 4bc7be13490..4911979394a 100644 --- a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.sql +++ b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.sql @@ -10,8 +10,8 @@ INSERT INTO test_table VALUES (0, 'Value'); DROP ROW POLICY IF EXISTS test_row_policy ON test_table; CREATE ROW POLICY test_row_policy ON test_table USING id >= 5 TO ALL; -EXPLAIN header = 1, actions = 1 SELECT id, value FROM test_table PREWHERE id = 5 settings allow_experimental_analyzer=0; -EXPLAIN header = 1, actions = 1 SELECT id, value FROM test_table PREWHERE id = 5 settings allow_experimental_analyzer=1; +EXPLAIN header = 1, actions = 1 SELECT id, value FROM test_table PREWHERE id = 5 settings enable_analyzer=0; +EXPLAIN header = 1, actions = 1 SELECT id, value FROM test_table PREWHERE id = 5 settings enable_analyzer=1; DROP ROW POLICY test_row_policy ON test_table; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql b/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql index 502cca20ab2..680b98fb1bd 100644 --- a/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql +++ b/tests/queries/0_stateless/02699_polygons_sym_difference_rollup.sql @@ -1,4 +1,4 @@ SELECT polygonsSymDifferenceCartesian([[[(1., 1.)]] AS x], [x]) GROUP BY x WITH ROLLUP; -SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP SETTINGS allow_experimental_analyzer=0; -SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP SETTINGS allow_experimental_analyzer=1; +SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP SETTINGS enable_analyzer=0; +SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP SETTINGS enable_analyzer=1; SELECT polygonsSymDifferenceCartesian([[[(100.0001, 1000.0001), (-20., 20.), (10., 10.), (20., 20.), (20., -20.), (1000.0001, 1.1920928955078125e-7)]],[[(0.0001, 100000000000000000000.)]] AS x],[x]) GROUP BY x WITH ROLLUP; diff --git a/tests/queries/0_stateless/02699_polygons_sym_difference_total.sql b/tests/queries/0_stateless/02699_polygons_sym_difference_total.sql index 0fac4b11320..53d0a3bb543 100644 --- a/tests/queries/0_stateless/02699_polygons_sym_difference_total.sql +++ b/tests/queries/0_stateless/02699_polygons_sym_difference_total.sql @@ -1,2 +1,2 @@ -SET allow_experimental_analyzer=0; +SET enable_analyzer=0; SELECT [(9223372036854775807, 1.1754943508222875e-38)], x, NULL, polygonsSymDifferenceCartesian([[[(1.1754943508222875e-38, 1.1920928955078125e-7), (0.5, 0.5)]], [[(1.1754943508222875e-38, 1.1920928955078125e-7), (1.1754943508222875e-38, 1.1920928955078125e-7)], [(0., 1.0001)]], [[(1., 1.0001)]] AS x], [[[(3.4028234663852886e38, 0.9999)]]]) GROUP BY GROUPING SETS ((x)) WITH TOTALS diff --git a/tests/queries/0_stateless/02699_polygons_sym_difference_total_analyzer.sql b/tests/queries/0_stateless/02699_polygons_sym_difference_total_analyzer.sql index 879e0e5297f..40f610ae5a6 100644 --- a/tests/queries/0_stateless/02699_polygons_sym_difference_total_analyzer.sql +++ b/tests/queries/0_stateless/02699_polygons_sym_difference_total_analyzer.sql @@ -1,2 +1,2 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT [(9223372036854775807, 1.1754943508222875e-38)], x, NULL, polygonsSymDifferenceCartesian([[[(1.1754943508222875e-38, 1.1920928955078125e-7), (0.5, 0.5)]], [[(1.1754943508222875e-38, 1.1920928955078125e-7), (1.1754943508222875e-38, 1.1920928955078125e-7)], [(0., 1.0001)]], [[(1., 1.0001)]] AS x], [[[(3.4028234663852886e38, 0.9999)]]]) GROUP BY GROUPING SETS ((x)) WITH TOTALS diff --git a/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql b/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql index 092bda23164..9cfc4d83058 100644 --- a/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql +++ b/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql @@ -1 +1 @@ -SELECT a, sum(b) FROM (SELECT 1 AS a, 1 AS b, 0 AS c) GROUP BY a HAVING c SETTINGS allow_experimental_analyzer=1 -- { serverError NOT_AN_AGGREGATE } +SELECT a, sum(b) FROM (SELECT 1 AS a, 1 AS b, 0 AS c) GROUP BY a HAVING c SETTINGS enable_analyzer=1 -- { serverError NOT_AN_AGGREGATE } diff --git a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql index 72ab507f541..9e2927334e1 100644 --- a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql +++ b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS 02702_logical_optimizer; diff --git a/tests/queries/0_stateless/02703_explain_query_tree_is_forbidden_with_old_analyzer.sql b/tests/queries/0_stateless/02703_explain_query_tree_is_forbidden_with_old_analyzer.sql index d351bfe402c..c028e74f1b5 100644 --- a/tests/queries/0_stateless/02703_explain_query_tree_is_forbidden_with_old_analyzer.sql +++ b/tests/queries/0_stateless/02703_explain_query_tree_is_forbidden_with_old_analyzer.sql @@ -1,2 +1,2 @@ -set allow_experimental_analyzer=0; +set enable_analyzer=0; EXPLAIN QUERY TREE run_passes = true, dump_passes = true SELECT 1; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql b/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql index 44a8fe4f049..db5eddf2a90 100644 --- a/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql +++ b/tests/queries/0_stateless/02704_storage_merge_explain_graph_crash.sql @@ -13,4 +13,4 @@ CREATE TABLE foo2_dist (`Id` UInt32, `Val` String) ENGINE = Distributed(test_sha CREATE TABLE merge1 AS foo ENGINE = Merge(currentDatabase(), '^(foo|foo2_dist)$'); EXPLAIN PIPELINE graph = 1, compact = 1 SELECT * FROM merge1 FORMAT Null; -EXPLAIN PIPELINE graph = 1, compact = 1 SELECT * FROM merge1 FORMAT Null SETTINGS allow_experimental_analyzer=1; +EXPLAIN PIPELINE graph = 1, compact = 1 SELECT * FROM merge1 FORMAT Null SETTINGS enable_analyzer=1; diff --git a/tests/queries/0_stateless/02707_analyzer_nested_lambdas_types.sql b/tests/queries/0_stateless/02707_analyzer_nested_lambdas_types.sql index f9258d61900..320e1111e65 100644 --- a/tests/queries/0_stateless/02707_analyzer_nested_lambdas_types.sql +++ b/tests/queries/0_stateless/02707_analyzer_nested_lambdas_types.sql @@ -1,24 +1,24 @@ SELECT range(1), arrayMap(x -> arrayMap(x -> x, range(x)), [1]) -SETTINGS allow_experimental_analyzer = 0; +SETTINGS enable_analyzer = 0; SELECT range(1), arrayMap(x -> arrayMap(x -> x, range(x)), [1]) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; SELECT range(1), arrayMap(x -> arrayMap(x -> 1, range(x)), [1]) -SETTINGS allow_experimental_analyzer = 0; +SETTINGS enable_analyzer = 0; SELECT range(1), arrayMap(x -> arrayMap(x -> 1, range(x)), [1]) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; SELECT range(1), arrayMap(x -> arrayMap(y -> 1, range(x)), [1]) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 index 25451a34867..6a8472fecdf 100644 --- a/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 @@ -8,7 +8,7 @@ INSERT INTO t1 VALUES (1); INSERT INTO t2 VALUES (2, 2); SET join_use_nulls = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/02725_cnf_large_check.sql b/tests/queries/0_stateless/02725_cnf_large_check.sql index 0780e6bcdd3..2567636c02c 100644 --- a/tests/queries/0_stateless/02725_cnf_large_check.sql +++ b/tests/queries/0_stateless/02725_cnf_large_check.sql @@ -7,21 +7,21 @@ INSERT INTO 02725_cnf VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0), (0, 0, 0, 0, 0, 0, SELECT count() FROM 02725_cnf WHERE (c5 AND (NOT c0)) OR ((NOT c3) AND (NOT c6) AND (NOT c1) AND (NOT c6)) OR (c7 AND (NOT c3) AND (NOT c5) AND (NOT c7)) OR ((NOT c8) AND c5) OR ((NOT c0)) OR ((NOT c8) AND (NOT c5) AND c1 AND c6 AND c3) OR (c7 AND (NOT c0) AND c6 AND c1 AND (NOT c2)) OR (c3 AND (NOT c9) AND c1) -SETTINGS convert_query_to_cnf = 1, allow_experimental_analyzer = 1; +SETTINGS convert_query_to_cnf = 1, enable_analyzer = 1; SELECT count() FROM 02725_cnf WHERE (c5 AND (NOT c0)) OR ((NOT c3) AND (NOT c6) AND (NOT c1) AND (NOT c6)) OR (c7 AND (NOT c3) AND (NOT c5) AND (NOT c7)) OR ((NOT c8) AND c5) OR ((NOT c0)) OR ((NOT c8) AND (NOT c5) AND c1 AND c6 AND c3) OR (c7 AND (NOT c0) AND c6 AND c1 AND (NOT c2)) OR (c3 AND (NOT c9) AND c1) -SETTINGS convert_query_to_cnf = 1, allow_experimental_analyzer = 0; +SETTINGS convert_query_to_cnf = 1, enable_analyzer = 0; SELECT count() FROM 02725_cnf WHERE ((NOT c2) AND c2 AND (NOT c1)) OR ((NOT c2) AND c3 AND (NOT c5)) OR ((NOT c7) AND (NOT c8)) OR (c9 AND c6 AND c8 AND (NOT c8) AND (NOT c7)) -SETTINGS convert_query_to_cnf = 1, allow_experimental_analyzer = 1; +SETTINGS convert_query_to_cnf = 1, enable_analyzer = 1; SELECT count() FROM 02725_cnf WHERE ((NOT c2) AND c2 AND (NOT c1)) OR ((NOT c2) AND c3 AND (NOT c5)) OR ((NOT c7) AND (NOT c8)) OR (c9 AND c6 AND c8 AND (NOT c8) AND (NOT c7)) -SETTINGS convert_query_to_cnf = 1, allow_experimental_analyzer = 0; +SETTINGS convert_query_to_cnf = 1, enable_analyzer = 0; DROP TABLE 02725_cnf; diff --git a/tests/queries/0_stateless/02731_analyzer_join_resolve_nested.sql.j2 b/tests/queries/0_stateless/02731_analyzer_join_resolve_nested.sql.j2 index c2f3c51b17a..4ddf41c4d6d 100644 --- a/tests/queries/0_stateless/02731_analyzer_join_resolve_nested.sql.j2 +++ b/tests/queries/0_stateless/02731_analyzer_join_resolve_nested.sql.j2 @@ -31,7 +31,7 @@ INSERT INTO nnna VALUES (1, [[([([(1,'d')],'d')], 's')]], ['s']); CREATE TABLE nnnb ( x UInt64, t Nested(t Nested(t Nested(t Nested(t UInt32, s String), s String), s String), s String) ) ENGINE = MergeTree ORDER BY x; INSERT INTO nnnb VALUES (1, [[([([(1,'d')],'d')], 's')]], ['s']); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; {% for join_use_nulls in [0, 1] -%} diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 451f0d6d485..125fa524c4a 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -33,8 +33,8 @@ =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== 0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; =============== OUTER QUERY (NO PARALLEL) =============== >T%O ,z< 10 NQTpY# W\\Xx4 10 @@ -61,5 +61,5 @@ t _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index 7693d0da295..8121d60a05b 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -38,7 +38,7 @@ FROM join_inner_table GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10; --- settings allow_experimental_analyzer=0; +-- settings enable_analyzer=0; -- SELECT -- key, @@ -49,7 +49,7 @@ LIMIT 10; -- PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) -- GROUP BY key, value1, value2 -- ORDER BY key, value1, value2 --- LIMIT 10 settings allow_experimental_analyzer=1; +-- LIMIT 10 settings enable_analyzer=1; SELECT '=============== INNER QUERY (PARALLEL) ==============='; @@ -64,7 +64,7 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; -- Parallel inner query alone SELECT @@ -77,7 +77,7 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; @@ -184,7 +184,7 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; -- Parallel full query SELECT @@ -214,7 +214,7 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02734_optimize_group_by.sql b/tests/queries/0_stateless/02734_optimize_group_by.sql index 28e86c04b0f..626805d028d 100644 --- a/tests/queries/0_stateless/02734_optimize_group_by.sql +++ b/tests/queries/0_stateless/02734_optimize_group_by.sql @@ -1,5 +1,5 @@ -SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 0; -SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 1; +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS enable_analyzer = 0; +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS enable_analyzer = 1; SELECT 'a' AS key, 'b' as value GROUP BY ignore(1) WITH CUBE; diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference index d21f914f0dc..a6c8c0bbc3e 100644 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference @@ -1,24 +1,24 @@ -view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +view enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 1 0 1 2 3 -subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +subquery enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 0 1 0 2 2 -CSE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CSE enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 0 1 0 2 2 -CSE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CSE_Multi enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 0 1 0 2 2 -CTE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CTE enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 0 1 0 2 2 -CTE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CTE_Multi enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 0 0 1 0 4 4 -view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +view enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 1 0 1 3 4 -subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +subquery enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 0 1 0 2 2 -CSE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CSE enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 0 1 0 2 2 -CSE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CSE_Multi enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 0 1 0 2 2 -CTE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CTE enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 0 1 0 2 2 -CTE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +CTE_Multi enable_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 0 1 0 4 4 diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh index 84031ad9081..b7d93b5396c 100755 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh @@ -14,14 +14,14 @@ $CLICKHOUSE_CLIENT -n -q " CREATE MATERIALIZED VIEW mv TO output SQL SECURITY NONE AS SELECT * FROM input; " -for allow_experimental_analyzer in 0 1; do +for enable_analyzer in 0 1; do query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "INSERT INTO input SELECT * FROM numbers(1)" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "INSERT INTO input SELECT * FROM numbers(1)" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 view, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -34,12 +34,12 @@ for allow_experimental_analyzer in 0 1; do " query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "SELECT * FROM system.one WHERE dummy IN (SELECT * FROM system.one) FORMAT Null" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "SELECT * FROM system.one WHERE dummy IN (SELECT * FROM system.one) FORMAT Null" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 subquery, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -51,12 +51,12 @@ for allow_experimental_analyzer in 0 1; do " query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x FORMAT Null" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x FORMAT Null" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 CSE, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -68,12 +68,12 @@ for allow_experimental_analyzer in 0 1; do " query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x, x FORMAT Null" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x, x FORMAT Null" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 CSE_Multi, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -85,12 +85,12 @@ for allow_experimental_analyzer in 0 1; do " query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x FORMAT Null" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x FORMAT Null" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 CTE, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -102,12 +102,12 @@ for allow_experimental_analyzer in 0 1; do " query_id="$(random_str 10)" - $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x UNION ALL SELECT * FROM x FORMAT Null" + $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x UNION ALL SELECT * FROM x FORMAT Null" $CLICKHOUSE_CLIENT -mn -q " SYSTEM FLUSH LOGS; SELECT 1 CTE_Multi, - $allow_experimental_analyzer allow_experimental_analyzer, + $enable_analyzer enable_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh index 0c5767314d5..2d2ee328a29 100755 --- a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh @@ -6,6 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out" # only EXPLAIN triggers the problem under MSan -$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "explain select * from numbers(1) into outfile '$out'" +$CLICKHOUSE_CLIENT --enable_analyzer=0 -q "explain select * from numbers(1) into outfile '$out'" cat "$out" rm -f "$out" diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index d86b65c3291..91ca5ef0340 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -23,12 +23,12 @@ SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError INDEX_NOT_USED } SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index 5bf3520ccdb..f60f1e0a376 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -8,5 +8,5 @@ 5935810273536892891 7885388429666205427 8124171311239967992 -1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; +1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; 0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index 88a0d2163d6..e60049f2756 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -21,7 +21,7 @@ INSERT INTO join_inner_table__fuzz_146_replicated SELECT number FROM join_inner_table__fuzz_146_replicated SETTINGS - allow_experimental_analyzer = 1, + enable_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 index 248461a98bb..74e252c785b 100644 --- a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 @@ -1,11 +1,11 @@ -{% for allow_experimental_analyzer in [0, 1] -%} +{% for enable_analyzer in [0, 1] -%} {% for join_use_nulls in [0, 1] -%} {% for kind in ['LEFT', 'RIGHT'] -%} {% for strictness in ['SEMI', 'ANTI'] -%} {% for maybe_materialize in ['', 'materialize'] -%} -SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; +SET enable_analyzer = {{ enable_analyzer }}; SET join_use_nulls = {{ join_use_nulls }}; diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql index 4da8cebff1c..b127af677ee 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -11,8 +11,8 @@ INSERT INTO source values ('2021-12-31 23:00:00', 0); SELECT * FROM source WHERE toYYYYMM(ts) = 202112; SELECT * FROM source WHERE toYear(ts) = 2021; -SELECT * FROM source WHERE toYYYYMM(ts) = 202112 SETTINGS allow_experimental_analyzer=1; -SELECT * FROM source WHERE toYear(ts) = 2021 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM source WHERE toYYYYMM(ts) = 202112 SETTINGS enable_analyzer=1; +SELECT * FROM source WHERE toYear(ts) = 2021 SETTINGS enable_analyzer=1; DROP TABLE IF EXISTS source; CREATE TABLE source @@ -46,18 +46,18 @@ SELECT count(*) FROM source WHERE toYear(dt) < 2023; SELECT count(*) FROM source WHERE toYear(dt) <= 2023; SELECT count(*) FROM source WHERE toYear(dt) > 2023; SELECT count(*) FROM source WHERE toYear(dt) >= 2023; -SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) = 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) <> 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) < 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) <= 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) > 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt) >= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) = 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) <> 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) < 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) <= 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) > 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) >= 2023 SETTINGS enable_analyzer=1; SELECT 'DateTime'; SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312; @@ -72,18 +72,18 @@ SELECT count(*) FROM source WHERE toYear(ts) < 2023; SELECT count(*) FROM source WHERE toYear(ts) <= 2023; SELECT count(*) FROM source WHERE toYear(ts) > 2023; SELECT count(*) FROM source WHERE toYear(ts) >= 2023; -SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) = 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) <> 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) < 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) <= 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) > 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts) >= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) = 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) <> 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) < 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) <= 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) > 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) >= 2023 SETTINGS enable_analyzer=1; SELECT 'Date32'; SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312; @@ -98,18 +98,18 @@ SELECT count(*) FROM source WHERE toYear(dt_32) < 2023; SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023; SELECT count(*) FROM source WHERE toYear(dt_32) > 2023; SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) = 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) < 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) > 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) = 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) < 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) > 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023 SETTINGS enable_analyzer=1; SELECT 'DateTime64'; SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312; @@ -124,16 +124,16 @@ SELECT count(*) FROM source WHERE toYear(ts_64) < 2023; SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023; SELECT count(*) FROM source WHERE toYear(ts_64) > 2023; SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) = 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) < 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) > 2023 SETTINGS allow_experimental_analyzer=1; -SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) = 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) < 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) > 2023 SETTINGS enable_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023 SETTINGS enable_analyzer=1; DROP TABLE source; diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh index 20b3efedd49..bf7170fd7fb 100755 --- a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -34,7 +34,7 @@ function run_query_with_pure_parallel_replicas () { --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ --allow_experimental_parallel_reading_from_replicas 1 \ - --allow_experimental_analyzer 0 + --enable_analyzer 0 $CLICKHOUSE_CLIENT \ --query "$2" \ @@ -43,7 +43,7 @@ function run_query_with_pure_parallel_replicas () { --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ --allow_experimental_parallel_reading_from_replicas 1 \ - --allow_experimental_analyzer 1 + --enable_analyzer 1 } function run_query_with_custom_key_parallel_replicas () { @@ -58,7 +58,7 @@ function run_query_with_custom_key_parallel_replicas () { --max_parallel_replicas 3 \ --parallel_replicas_custom_key_filter_type 'default' \ --parallel_replicas_custom_key "$2" \ - --allow_experimental_analyzer 0 + --enable_analyzer 0 $CLICKHOUSE_CLIENT \ --query "$2" \ @@ -66,7 +66,7 @@ function run_query_with_custom_key_parallel_replicas () { --max_parallel_replicas 3 \ --parallel_replicas_custom_key_filter_type 'default' \ --parallel_replicas_custom_key "$2" \ - --allow_experimental_analyzer 1 + --enable_analyzer 1 } $CLICKHOUSE_CLIENT --query " diff --git a/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql index 44b9ce4fdc1..3766e5b0c8f 100644 --- a/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql +++ b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql @@ -4,7 +4,7 @@ CREATE TABLE t_02784 (c1 UInt64, c2 UInt64) ENGINE=MergeTree() ORDER BY c1 SETTI INSERT INTO t_02784 SELECT number, number FROM numbers(1); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET move_all_conditions_to_prewhere=1; SELECT c1, c2 FROM t_02784 WHERE c1 = 0 AND c2 = 0; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference index fca48238778..c2c77a4aa31 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference @@ -39,7 +39,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 < \'1993-01-01\') OR (date1 >= \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) @@ -81,7 +81,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) @@ -115,7 +115,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) @@ -149,7 +149,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) @@ -183,7 +183,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) @@ -217,7 +217,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1998-01-01\')) AND ((id >= 1) AND (id <= 3)) @@ -259,7 +259,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) OR ((date1 >= \'1994-01-01\') AND (date1 < \'1995-01-01\'))) AND ((id >= 1) AND (id <= 3)) @@ -317,7 +317,7 @@ QUERY id: 0 LIST id: 32, nodes: 2 COLUMN id: 29, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 33, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1, toYear(date1) AS year1 @@ -366,7 +366,7 @@ QUERY id: 0 LIST id: 24, nodes: 2 COLUMN id: 21, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 25, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) @@ -400,7 +400,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t PREWHERE (date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\') @@ -441,7 +441,7 @@ QUERY id: 0 LIST id: 20, nodes: 2 COLUMN id: 17, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 21, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) @@ -481,7 +481,7 @@ QUERY id: 0 LIST id: 20, nodes: 2 COLUMN id: 17, column_name: date1, result_type: Date, source_id: 3 CONSTANT id: 21, constant_value: \'1994-01-01\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) @@ -518,7 +518,7 @@ QUERY id: 0 LIST id: 19, nodes: 2 COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 20, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) @@ -555,7 +555,7 @@ QUERY id: 0 LIST id: 19, nodes: 2 COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 20, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) @@ -597,7 +597,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1992-03-01\') AND (date1 < \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) @@ -639,7 +639,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 < \'1992-03-01\') OR (date1 >= \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) @@ -681,7 +681,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) @@ -715,7 +715,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) @@ -749,7 +749,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) @@ -783,7 +783,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) @@ -817,7 +817,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\'))) AND ((id >= 1) AND (id <= 3)) @@ -867,7 +867,7 @@ QUERY id: 0 LIST id: 27, nodes: 2 COLUMN id: 24, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 28, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM datetime_t WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) @@ -909,7 +909,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM datetime_t WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) @@ -951,7 +951,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date32_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) @@ -993,7 +993,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM date32_t WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) @@ -1035,7 +1035,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM datetime64_t WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) @@ -1077,7 +1077,7 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 SELECT value1 FROM datetime64_t WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) @@ -1119,4 +1119,4 @@ QUERY id: 0 LIST id: 22, nodes: 2 COLUMN id: 19, column_name: id, result_type: UInt32, source_id: 3 CONSTANT id: 23, constant_value: UInt64_3, constant_value_type: UInt8 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql index 9cc8dd74e5d..5ff62cb4bb3 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql @@ -2,74 +2,74 @@ DROP TABLE IF EXISTS date_t; CREATE TABLE date_t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; DROP TABLE date_t; DROP TABLE IF EXISTS datetime_t; CREATE TABLE datetime_t (id UInt32, value1 String, date1 Datetime) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; DROP TABLE datetime_t; DROP TABLE IF EXISTS date32_t; CREATE TABLE date32_t (id UInt32, value1 String, date1 Date32) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; DROP TABLE date32_t; DROP TABLE IF EXISTS datetime64_t; CREATE TABLE datetime64_t (id UInt32, value1 String, date1 Datetime64) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS enable_analyzer=1; DROP TABLE datetime64_t; diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql index 6d79aa76d18..dd72b990445 100644 --- a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql @@ -6,7 +6,7 @@ SELECT * FROM system.numbers LIMIT 10000; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT * FROM diff --git a/tests/queries/0_stateless/02812_bug_with_unused_join_columns.sql b/tests/queries/0_stateless/02812_bug_with_unused_join_columns.sql index 6c801b5b73e..d791b8f3367 100644 --- a/tests/queries/0_stateless/02812_bug_with_unused_join_columns.sql +++ b/tests/queries/0_stateless/02812_bug_with_unused_join_columns.sql @@ -1 +1 @@ -SELECT concat(func.name, comb.name) AS x FROM system.functions AS func JOIN system.aggregate_function_combinators AS comb using name WHERE is_aggregate settings allow_experimental_analyzer=1; +SELECT concat(func.name, comb.name) AS x FROM system.functions AS func JOIN system.aggregate_function_combinators AS comb using name WHERE is_aggregate settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/02815_analyzer_aggregate_functions_of_group_by_keys.sql b/tests/queries/0_stateless/02815_analyzer_aggregate_functions_of_group_by_keys.sql index ca03cbb6f9f..dfb885f5a69 100644 --- a/tests/queries/0_stateless/02815_analyzer_aggregate_functions_of_group_by_keys.sql +++ b/tests/queries/0_stateless/02815_analyzer_aggregate_functions_of_group_by_keys.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set optimize_move_functions_out_of_any = 0; SELECT 'set optimize_aggregators_of_group_by_keys = 1'; diff --git a/tests/queries/0_stateless/02815_join_algorithm_setting.sql b/tests/queries/0_stateless/02815_join_algorithm_setting.sql index a8733eebc91..4a5ae784b31 100644 --- a/tests/queries/0_stateless/02815_join_algorithm_setting.sql +++ b/tests/queries/0_stateless/02815_join_algorithm_setting.sql @@ -104,9 +104,9 @@ JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 ON t1.key = t2.key OR t1 -- But for CROSS choose `hash` algorithm even though it's not enabled SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; -- ... (not for old analyzer) SELECT * FROM ( SELECT number AS key, number * 10 AS key2 FROM numbers_mt(10) ) AS t1 CROSS JOIN ( SELECT k AS key, k + 100 AS key2 FROM t2 ) AS t2 FORMAT Null -SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } +SETTINGS enable_analyzer = 0; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/02834_analyzer_with_statement_references.sql b/tests/queries/0_stateless/02834_analyzer_with_statement_references.sql index 29ed6e3f0da..ce1eaa7ae77 100644 --- a/tests/queries/0_stateless/02834_analyzer_with_statement_references.sql +++ b/tests/queries/0_stateless/02834_analyzer_with_statement_references.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH test_aliases AS (SELECT number FROM numbers(20)), alias2 AS (SELECT number FROM test_aliases) SELECT number FROM alias2 SETTINGS enable_global_with_statement = 1; diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index d0475fa14b6..1cdd3684a0b 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table_1; CREATE TABLE test_table_1 diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.reference b/tests/queries/0_stateless/02840_merge__table_or_filter.reference index ff5e0865a22..21b54a0121e 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.reference +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.reference @@ -1,38 +1,38 @@ -- { echoOn } -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=0, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings enable_analyzer=0, convert_query_to_cnf=0; v1 1 v1 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=0, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings enable_analyzer=0, convert_query_to_cnf=0; v1 1 v2 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings enable_analyzer=0, convert_query_to_cnf=0; v1 1 -select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=0; -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=0, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings enable_analyzer=0, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings enable_analyzer=0, convert_query_to_cnf=1; v1 1 v1 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=0, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings enable_analyzer=0, convert_query_to_cnf=1; v1 1 v2 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings enable_analyzer=0, convert_query_to_cnf=1; v1 1 -select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=1; -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=1, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings enable_analyzer=0, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings enable_analyzer=1, convert_query_to_cnf=0; v1 1 v1 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=1, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings enable_analyzer=1, convert_query_to_cnf=0; v1 1 v2 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings enable_analyzer=1, convert_query_to_cnf=0; v1 1 -select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=0; -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=1, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings enable_analyzer=1, convert_query_to_cnf=0; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings enable_analyzer=1, convert_query_to_cnf=1; v1 1 v1 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=1, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings enable_analyzer=1, convert_query_to_cnf=1; v1 1 v2 2 -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings enable_analyzer=1, convert_query_to_cnf=1; v1 1 -select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=1; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings enable_analyzer=1, convert_query_to_cnf=1; diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 index 286e4545ef7..f7413dc3ee6 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 @@ -27,10 +27,10 @@ select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 an set max_threads=1; -- { echoOn } {% for settings in [ - 'allow_experimental_analyzer=0, convert_query_to_cnf=0', - 'allow_experimental_analyzer=0, convert_query_to_cnf=1', - 'allow_experimental_analyzer=1, convert_query_to_cnf=0', - 'allow_experimental_analyzer=1, convert_query_to_cnf=1' + 'enable_analyzer=0, convert_query_to_cnf=0', + 'enable_analyzer=0, convert_query_to_cnf=1', + 'enable_analyzer=1, convert_query_to_cnf=0', + 'enable_analyzer=1, convert_query_to_cnf=1' ] %} select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings {{ settings }}; select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings {{ settings }}; diff --git a/tests/queries/0_stateless/02841_valid_json_and_xml_on_http_exception.sh b/tests/queries/0_stateless/02841_valid_json_and_xml_on_http_exception.sh index c47fe5c7e94..3dda63e1e49 100755 --- a/tests/queries/0_stateless/02841_valid_json_and_xml_on_http_exception.sh +++ b/tests/queries/0_stateless/02841_valid_json_and_xml_on_http_exception.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_URL_BASE="$CLICKHOUSE_URL&http_write_exception_in_output_format=1&allow_experimental_analyzer=0" +CH_URL_BASE="$CLICKHOUSE_URL&http_write_exception_in_output_format=1&enable_analyzer=0" for wait_end_of_query in 0 1 do diff --git a/tests/queries/0_stateless/02841_with_clause_resolve.sql b/tests/queries/0_stateless/02841_with_clause_resolve.sql index b416446461b..fe94a26110d 100644 --- a/tests/queries/0_stateless/02841_with_clause_resolve.sql +++ b/tests/queries/0_stateless/02841_with_clause_resolve.sql @@ -1,16 +1,16 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; WITH -- Input 44100 AS sample_frequency , number AS tick , tick / sample_frequency AS time - + -- Delay , (time, wave, delay_, decay, count) -> arraySum(n1 -> wave(time - delay_ * n1), range(count)) AS delay , delay(time, (time -> 0.5), 0.2, 0.5, 5) AS kick - + SELECT kick @@ -23,29 +23,29 @@ WITH 44100 AS sample_frequency , number AS tick , tick / sample_frequency AS time - + -- Output control , 1 AS master_volume , level -> least(1.0, greatest(-1.0, level)) AS clamp , level -> (clamp(level) * 0x7FFF * master_volume)::Int16 AS output , x -> (x, x) AS mono - + -- Basic waves , time -> sin(time * 2 * pi()) AS sine_wave , time -> time::UInt64 % 2 * 2 - 1 AS square_wave , time -> (time - floor(time)) * 2 - 1 AS sawtooth_wave , time -> abs(sawtooth_wave(time)) * 2 - 1 AS triangle_wave - + -- Helpers , (from, to, wave, time) -> from + ((wave(time) + 1) / 2) * (to - from) AS lfo , (from, to, steps, time) -> from + floor((time - floor(time)) * steps) / steps * (to - from) AS step_lfo , (from, to, steps, time) -> exp(step_lfo(log(from), log(to), steps, time)) AS exp_step_lfo - + -- Noise , time -> cityHash64(time) / 0xFFFFFFFFFFFFFFFF AS uniform_noise , time -> erf(uniform_noise(time)) AS white_noise , time -> cityHash64(time) % 2 ? 1 : -1 AS bernoulli_noise - + -- Distortion , (x, amount) -> clamp(x * amount) AS clipping , (x, amount) -> clamp(x > 0 ? pow(x, amount) : -pow(-x, amount)) AS power_distortion @@ -53,10 +53,10 @@ WITH , (time, sample_frequency) -> round(time * sample_frequency) / sample_frequency AS desample , (time, wave, amount) -> (time - floor(time) < (1 - amount)) ? wave(time * (1 - amount)) : 0 AS thin , (time, wave, amount) -> wave(floor(time) + pow(time - floor(time), amount)) AS skew - + -- Combining , (a, b, weight) -> a * (1 - weight) + b * weight AS combine - + -- Envelopes , (time, offset, attack, hold, release) -> time < offset ? 0 @@ -64,7 +64,7 @@ WITH : (time < offset + attack + hold ? 1 : (time < offset + attack + hold + release ? (offset + attack + hold + release - time) / release : 0))) AS envelope - + , (bpm, time, offset, attack, hold, release) -> envelope( time * (bpm / 60) - floor(time * (bpm / 60)), @@ -72,20 +72,20 @@ WITH attack, hold, release) AS running_envelope - + -- Sequencers , (sequence, time) -> sequence[1 + time::UInt64 % length(sequence)] AS sequencer - + -- Delay , (time, wave, delay, decay, count) -> arraySum(n -> wave(time - delay * n) * pow(decay, n), range(count)) AS delay - - + + , delay(time, (time -> power_distortion(sine_wave(time * 80 + sine_wave(time * 2)), lfo(0.5, 1, sine_wave, time / 16)) * running_envelope(60, time, 0, 0.0, 0.01, 0.1)), 0.2, 0.5, 5) AS kick - + SELECT - + (output( kick + delay(time, (time -> @@ -95,7 +95,7 @@ SELECT lfo(1, 0.75, triangle_wave, time / 8))), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, time / 7) - + + delay(time, (time -> power_distortion( sine_wave(time * sequencer([50, 100, 200, 400], time / 2) + 1 * sine_wave(time * sequencer([50, 100, 200], time / 4) + 1/4)) @@ -103,16 +103,16 @@ SELECT lfo(1, 0.75, triangle_wave, time / 8))), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, 16 + time / 11) - + + delay(time, (time -> white_noise(time) * running_envelope(60, time, 0.75, 0.01, 0.01, 0.1)), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, 24 + time / 13) - + + sine_wave(time * 100 + 1 * sine_wave(time * 10 + 1/4)) * running_envelope(120, time, 0, 0.01, 0.01, 0.1) ), - + output( kick + delay(time + 0.01, (time -> @@ -122,7 +122,7 @@ SELECT lfo(1, 0.75, triangle_wave, time / 8))), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, time / 7) - + + delay(time - 0.01, (time -> power_distortion( sine_wave(time * sequencer([50, 100, 200, 400], time / 2) + 1 * sine_wave(time * sequencer([50, 100, 200], time / 4) + 1/4)) @@ -130,12 +130,12 @@ SELECT lfo(1, 0.75, triangle_wave, time / 8))), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, 16 + time / 11) - + + delay(time + 0.005, (time -> white_noise(time) * running_envelope(60, time, 0.75, 0.01, 0.01, 0.1)), 0.2, 0.5, 10) * lfo(0.5, 1, triangle_wave, 24 + time / 13) )) - + FROM system.numbers LIMIT 10; diff --git a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql index ea52df5d4b4..c9cdab0cea2 100644 --- a/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql +++ b/tests/queries/0_stateless/02861_filter_pushdown_const_bug.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql index 3e15ec1148e..6606cff263f 100644 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql +++ b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS tab; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; diff --git a/tests/queries/0_stateless/02866_size_of_marks_skip_idx_explain.sql b/tests/queries/0_stateless/02866_size_of_marks_skip_idx_explain.sql index b916c5ca13d..b3adf38710a 100644 --- a/tests/queries/0_stateless/02866_size_of_marks_skip_idx_explain.sql +++ b/tests/queries/0_stateless/02866_size_of_marks_skip_idx_explain.sql @@ -4,7 +4,7 @@ SET optimize_move_to_prewhere = 1; SET convert_query_to_cnf = 0; SET optimize_read_in_order = 1; -SET allow_experimental_analyzer = 1; -- slightly different operator names than w/o +SET enable_analyzer = 1; -- slightly different operator names than w/o DROP TABLE IF EXISTS test_skip_idx; diff --git a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference index c2075f72f33..3dedcedbcee 100644 --- a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference +++ b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference @@ -6,7 +6,7 @@ FROM SELECT DISTINCT a FROM test_rewrite_uniq_to_count ) -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -23,7 +23,7 @@ QUERY id: 0 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 2. test distinct with subquery alias 3 SELECT count() @@ -32,7 +32,7 @@ FROM SELECT DISTINCT a FROM test_rewrite_uniq_to_count ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -49,7 +49,7 @@ QUERY id: 0 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 3. test distinct with compound column name 3 SELECT count() @@ -58,7 +58,7 @@ FROM SELECT DISTINCT a FROM test_rewrite_uniq_to_count ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -75,7 +75,7 @@ QUERY id: 0 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 4. test distinct with select expression alias 3 SELECT count() @@ -84,7 +84,7 @@ FROM SELECT DISTINCT a AS alias_of_a FROM test_rewrite_uniq_to_count ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -101,7 +101,7 @@ QUERY id: 0 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 5. test simple group by 3 SELECT count() @@ -113,7 +113,7 @@ FROM FROM test_rewrite_uniq_to_count GROUP BY a ) -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -133,7 +133,7 @@ QUERY id: 0 GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 6. test group by with subquery alias 3 SELECT count() @@ -145,7 +145,7 @@ FROM FROM test_rewrite_uniq_to_count GROUP BY a ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -165,7 +165,7 @@ QUERY id: 0 GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 7. test group by with compound column name 3 SELECT count() @@ -177,7 +177,7 @@ FROM FROM test_rewrite_uniq_to_count GROUP BY a ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -197,7 +197,7 @@ QUERY id: 0 GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 8. test group by with select expression alias 3 SELECT count() @@ -209,7 +209,7 @@ FROM FROM test_rewrite_uniq_to_count GROUP BY alias_of_a ) AS t -SETTINGS allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 0 3 QUERY id: 0 PROJECTION COLUMNS @@ -229,4 +229,4 @@ QUERY id: 0 GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/02868_distinct_to_count_optimization.sql b/tests/queries/0_stateless/02868_distinct_to_count_optimization.sql index 66431b7c36b..d30bade4dd5 100644 --- a/tests/queries/0_stateless/02868_distinct_to_count_optimization.sql +++ b/tests/queries/0_stateless/02868_distinct_to_count_optimization.sql @@ -16,53 +16,53 @@ set optimize_uniq_to_count=true; SELECT '1. test simple distinct'; -SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings allow_experimental_analyzer=0; -SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings allow_experimental_analyzer=1; +SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings enable_analyzer=0; +SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) settings enable_analyzer=1; SELECT '2. test distinct with subquery alias'; -SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; +SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; SELECT '3. test distinct with compound column name'; -SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; +SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; SELECT '4. test distinct with select expression alias'; -SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=0; -SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings allow_experimental_analyzer=1; +SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=0; +SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t settings enable_analyzer=1; SELECT '5. test simple group by'; -SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings allow_experimental_analyzer=0; -SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings allow_experimental_analyzer=1; +SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings enable_analyzer=0; +SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) settings enable_analyzer=1; SELECT '6. test group by with subquery alias'; -SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=0; -SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=1; +SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=0; +SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=1; SELECT '7. test group by with compound column name'; -SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=0; -SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings allow_experimental_analyzer=1; +SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=0; +SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t settings enable_analyzer=1; SELECT '8. test group by with select expression alias'; -SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings allow_experimental_analyzer=0; -EXPLAIN SYNTAX SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings allow_experimental_analyzer=0; -SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings allow_experimental_analyzer=1; -EXPLAIN QUERY TREE SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings allow_experimental_analyzer=1; +SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings enable_analyzer=0; +EXPLAIN SYNTAX SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings enable_analyzer=0; +SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings enable_analyzer=1; +EXPLAIN QUERY TREE SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t settings enable_analyzer=1; drop table if exists test_rewrite_uniq_to_count; diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql index 1b1a7607344..83b38955734 100644 --- a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql @@ -7,14 +7,14 @@ CREATE TABLE test_table INSERT INTO test_table SELECT number, number FROM numbers(10); -set allow_experimental_analyzer = 0; +set enable_analyzer = 0; EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.sql b/tests/queries/0_stateless/02890_named_tuple_functions.sql index 0033102bd53..6724462562a 100644 --- a/tests/queries/0_stateless/02890_named_tuple_functions.sql +++ b/tests/queries/0_stateless/02890_named_tuple_functions.sql @@ -1,5 +1,5 @@ set enable_named_columns_in_function_tuple = 1; -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; drop table if exists x; create table x (i int, j int) engine MergeTree order by i; diff --git a/tests/queries/0_stateless/02890_untuple_column_names.reference b/tests/queries/0_stateless/02890_untuple_column_names.reference index 13a85c70138..f91a63fa15f 100644 --- a/tests/queries/0_stateless/02890_untuple_column_names.reference +++ b/tests/queries/0_stateless/02890_untuple_column_names.reference @@ -57,7 +57,7 @@ t.1: 1 Row 1: ────── t.1: 1 --- tuple() with enable_named_columns_in_function_tuple = 1 and allow_experimental_analyzer = 1 keeps the column names +-- tuple() with enable_named_columns_in_function_tuple = 1 and enable_analyzer = 1 keeps the column names Row 1: ────── t.a: 1 diff --git a/tests/queries/0_stateless/02890_untuple_column_names.sql b/tests/queries/0_stateless/02890_untuple_column_names.sql index cd490ca3522..9773e2e53a0 100644 --- a/tests/queries/0_stateless/02890_untuple_column_names.sql +++ b/tests/queries/0_stateless/02890_untuple_column_names.sql @@ -6,43 +6,43 @@ SELECT '-- tuple element alias'; -SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple('s')::Tuple(a String)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple('s')::Tuple(a String)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple('s')::Tuple(a String)) FORMAT Vertical SETTINGS enable_analyzer = 0; +SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple('s')::Tuple(a String)) FORMAT Vertical SETTINGS enable_analyzer = 1; -SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple(1)::Tuple(a Int)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -- { serverError DUPLICATE_COLUMN } -SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple(1)::Tuple(a Int)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; -- Bug: doesn't throw an exception +SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple(1)::Tuple(a Int)) FORMAT Vertical SETTINGS enable_analyzer = 0; -- { serverError DUPLICATE_COLUMN } +SELECT untuple(tuple(1)::Tuple(a Int)), untuple(tuple(1)::Tuple(a Int)) FORMAT Vertical SETTINGS enable_analyzer = 1; -- Bug: doesn't throw an exception SELECT '-- tuple element alias + untuple() alias'; -SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple('s')::Tuple(a String)) y FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple('s')::Tuple(a String)) y FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple('s')::Tuple(a String)) y FORMAT Vertical SETTINGS enable_analyzer = 0; +SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple('s')::Tuple(a String)) y FORMAT Vertical SETTINGS enable_analyzer = 1; -SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple(1)::Tuple(a Int)) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -- { serverError DUPLICATE_COLUMN } -SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple(1)::Tuple(a Int)) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; -- Bug: doesn't throw an exception +SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple(1)::Tuple(a Int)) x FORMAT Vertical SETTINGS enable_analyzer = 0; -- { serverError DUPLICATE_COLUMN } +SELECT untuple(tuple(1)::Tuple(a Int)) x, untuple(tuple(1)::Tuple(a Int)) x FORMAT Vertical SETTINGS enable_analyzer = 1; -- Bug: doesn't throw an exception SELECT '-- untuple() alias'; -SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple('s')::Tuple(String)) y FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple('s')::Tuple(String)) y FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple('s')::Tuple(String)) y FORMAT Vertical SETTINGS enable_analyzer = 0; +SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple('s')::Tuple(String)) y FORMAT Vertical SETTINGS enable_analyzer = 1; -SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple(1)::Tuple(Int)) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -- { serverError DUPLICATE_COLUMN } -SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple(1)::Tuple(Int)) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; -- Bug: doesn't throw an exception +SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple(1)::Tuple(Int)) x FORMAT Vertical SETTINGS enable_analyzer = 0; -- { serverError DUPLICATE_COLUMN } +SELECT untuple(tuple(1)::Tuple(Int)) x, untuple(tuple(1)::Tuple(Int)) x FORMAT Vertical SETTINGS enable_analyzer = 1; -- Bug: doesn't throw an exception SELECT '-- no aliases'; -SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple('s')::Tuple(String)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple('s')::Tuple(String)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple('s')::Tuple(String)) FORMAT Vertical SETTINGS enable_analyzer = 0; +SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple('s')::Tuple(String)) FORMAT Vertical SETTINGS enable_analyzer = 1; -SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -- { serverError DUPLICATE_COLUMN } -SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; -- Bug: doesn't throw an exception +SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Vertical SETTINGS enable_analyzer = 0; -- { serverError DUPLICATE_COLUMN } +SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Vertical SETTINGS enable_analyzer = 1; -- Bug: doesn't throw an exception SELECT '-- tuple() loses the column names (would be good to fix, see #36773)'; -SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 0, enable_named_columns_in_function_tuple = 0; -SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 1, enable_named_columns_in_function_tuple = 0; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS enable_analyzer = 0, enable_named_columns_in_function_tuple = 0; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS enable_analyzer = 1, enable_named_columns_in_function_tuple = 0; -SELECT '-- tuple() with enable_named_columns_in_function_tuple = 1 and allow_experimental_analyzer = 1 keeps the column names'; -SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 1, enable_named_columns_in_function_tuple = 1; +SELECT '-- tuple() with enable_named_columns_in_function_tuple = 1 and enable_analyzer = 1 keeps the column names'; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS enable_analyzer = 1, enable_named_columns_in_function_tuple = 1; SELECT '-- thankfully JSONExtract() keeps them'; -SELECT untuple(JSONExtract('{"key": "value"}', 'Tuple(key String)')) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(JSONExtract('{"key": "value"}', 'Tuple(key String)')) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(JSONExtract('{"key": "value"}', 'Tuple(key String)')) x FORMAT Vertical SETTINGS enable_analyzer = 0; +SELECT untuple(JSONExtract('{"key": "value"}', 'Tuple(key String)')) x FORMAT Vertical SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql b/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql index b082f2f33b2..77f30ba82ba 100644 --- a/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql +++ b/tests/queries/0_stateless/02911_analyzer_explain_estimate.sql @@ -1,5 +1,5 @@ -- Tags: distributed -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN ESTIMATE SELECT 0 = 1048577, NULL, groupBitmapOr(bitmapBuild([toInt32(65537)])) FROM cluster(test_cluster_two_shards) WHERE NULL = 1048575; diff --git a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.sql index 77a72c24f5a..907263168d0 100644 --- a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.sql +++ b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.sql @@ -1,4 +1,4 @@ -SET optimize_read_in_order = 1, query_plan_read_in_order = 1, allow_experimental_analyzer = 1; +SET optimize_read_in_order = 1, query_plan_read_in_order = 1, enable_analyzer = 1; drop table if exists tab; drop table if exists tab2; diff --git a/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql index 70de63c592d..d567ac6c8f3 100644 --- a/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql +++ b/tests/queries/0_stateless/02911_analyzer_remove_unused_projection_columns.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index ee5e1582015..e1e095a6eae 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -15,7 +15,7 @@ CREATE TABLE t2n (x Int64, y UInt64) ENGINE = TinyLog; INSERT INTO t1n VALUES (1,42), (2,2), (3,3); INSERT INTO t2n VALUES (2,2), (3,33), (4,42); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) ORDER BY t1.x NULLS LAST; diff --git a/tests/queries/0_stateless/02911_support_alias_column_in_indices.sql b/tests/queries/0_stateless/02911_support_alias_column_in_indices.sql index 4d68786d7db..5ab50044e64 100644 --- a/tests/queries/0_stateless/02911_support_alias_column_in_indices.sql +++ b/tests/queries/0_stateless/02911_support_alias_column_in_indices.sql @@ -17,8 +17,8 @@ settings index_granularity = 8192, min_index_granularity_bytes = 1024, index_gra insert into test1 select * from numbers(10); insert into test1 select * from numbers(11, 20); -explain indexes = 1 select * from test1 where a > 10 settings allow_experimental_analyzer = 0; -explain indexes = 1 select * from test1 where a > 10 settings allow_experimental_analyzer = 1; +explain indexes = 1 select * from test1 where a > 10 settings enable_analyzer = 0; +explain indexes = 1 select * from test1 where a > 10 settings enable_analyzer = 1; create table test2 ( @@ -34,7 +34,7 @@ settings index_granularity = 8192, min_index_granularity_bytes = 1024, index_gra insert into test2 select * from numbers(10); insert into test2 select * from numbers(11, 20); -explain indexes = 1 select * from test2 where a2 > 15 settings allow_experimental_analyzer = 0; -explain indexes = 1 select * from test2 where a2 > 15 settings allow_experimental_analyzer = 1; +explain indexes = 1 select * from test2 where a2 > 15 settings enable_analyzer = 0; +explain indexes = 1 select * from test2 where a2 > 15 settings enable_analyzer = 1; drop database 02911_support_alias_column_in_indices; diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql index 94849453063..f1d606ab1cd 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql @@ -1,2 +1,2 @@ -set allow_experimental_analyzer=1; +set enable_analyzer=1; SELECT concat('With ', materialize(_CAST('ba\0', 'LowCardinality(FixedString(3))'))) AS `concat('With ', materialize(CAST('ba\\0', 'LowCardinality(FixedString(3))')))` FROM system.one GROUP BY 'With '; diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_2.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_2.sql index ca9fff68446..8921d36c546 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_2.sql +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_2.sql @@ -1,5 +1,4 @@ SET aggregate_functions_null_for_empty = 1; ---set allow_experimental_analyzer=1; +--set enable_analyzer=1; create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into t_delete_projection select number, toString(number) from numbers(8192 * 10); - diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_5.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_5.sql index 29d06d2c315..d75d4f4eb72 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_5.sql +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_5.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer=1; +set enable_analyzer=1; SET max_block_size = 1000; SET max_threads = 4; SET max_rows_to_group_by = 3000, group_by_overflow_mode = 'any'; diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql index b4eb1b4aff4..cc276ec4074 100644 --- a/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_6.sql @@ -1,5 +1,5 @@ set allow_suspicious_low_cardinality_types=1; -set allow_experimental_analyzer=1; +set enable_analyzer=1; create table tab (x LowCardinality(Nullable(Float64))) engine = MergeTree order by x settings allow_nullable_key=1; insert into tab select number from numbers(2); diff --git a/tests/queries/0_stateless/02918_join_pm_lc_crash.sql b/tests/queries/0_stateless/02918_join_pm_lc_crash.sql index 123208ee981..0326acff4a6 100644 --- a/tests/queries/0_stateless/02918_join_pm_lc_crash.sql +++ b/tests/queries/0_stateless/02918_join_pm_lc_crash.sql @@ -1,31 +1,30 @@ SET joined_subquery_requires_alias = 0, join_algorithm = 'partial_merge'; -SET allow_experimental_analyzer = 0, join_use_nulls = 0; +SET enable_analyzer = 0, join_use_nulls = 0; SELECT * FROM (SELECT dummy AS val FROM system.one) JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one GROUP BY val WITH TOTALS) USING (val); -SET allow_experimental_analyzer = 0, join_use_nulls = 1; +SET enable_analyzer = 0, join_use_nulls = 1; SELECT * FROM (SELECT dummy AS val FROM system.one) JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one GROUP BY val WITH TOTALS) USING (val); -SET allow_experimental_analyzer = 1, join_use_nulls = 0; +SET enable_analyzer = 1, join_use_nulls = 0; SELECT * FROM (SELECT dummy AS val FROM system.one) JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one GROUP BY val WITH TOTALS) USING (val); -SET allow_experimental_analyzer = 1, join_use_nulls = 1; +SET enable_analyzer = 1, join_use_nulls = 1; SELECT * FROM (SELECT dummy AS val FROM system.one) JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one GROUP BY val WITH TOTALS) USING (val); - diff --git a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.sql b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.sql index 9feb2aa2ad6..f5647f01ab9 100644 --- a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.sql +++ b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.sql @@ -15,7 +15,7 @@ INSERT INTO mt2 VALUES (1); SELECT count() FROM merge; -- can use the trivial count optimization -EXPLAIN SELECT count() FROM merge settings allow_experimental_analyzer=0; +EXPLAIN SELECT count() FROM merge settings enable_analyzer=0; CREATE TABLE mt3 (id UInt64) ENGINE = TinyLog; @@ -24,7 +24,7 @@ INSERT INTO mt2 VALUES (2); SELECT count() FROM merge; -- can't use the trivial count optimization as TinyLog doesn't support it -EXPLAIN SELECT count() FROM merge settings allow_experimental_analyzer=0; +EXPLAIN SELECT count() FROM merge settings enable_analyzer=0; DROP TABLE IF EXISTS mt1; DROP TABLE IF EXISTS mt2; diff --git a/tests/queries/0_stateless/02922_respect_nulls_parser.sql b/tests/queries/0_stateless/02922_respect_nulls_parser.sql index c9a17fdfbfe..ccd67dbe676 100644 --- a/tests/queries/0_stateless/02922_respect_nulls_parser.sql +++ b/tests/queries/0_stateless/02922_respect_nulls_parser.sql @@ -13,7 +13,7 @@ SELECT toDateTimeNonExistingFunction(now()) RESPECT NULLS b; -- { serverError UN SELECT toDateTime(now()) RESPECT NULLS b; -- { serverError SYNTAX_ERROR } SELECT count() from numbers(10) where in(number, (0)) RESPECT NULLS; -- { serverError SYNTAX_ERROR } SELECT if(number > 0, number, 0) respect nulls from numbers(0); -- { serverError SYNTAX_ERROR } -WITH (x -> x + 1) AS lambda SELECT lambda(number) RESPECT NULLS FROM numbers(10) SETTINGS allow_experimental_analyzer = 1; -- { serverError SYNTAX_ERROR } +WITH (x -> x + 1) AS lambda SELECT lambda(number) RESPECT NULLS FROM numbers(10) SETTINGS enable_analyzer = 1; -- { serverError SYNTAX_ERROR } SELECT * from system.one WHERE indexHint(dummy = 1) RESPECT NULLS; -- { serverError SYNTAX_ERROR } SELECT arrayJoin([[3,4,5], [6,7], [2], [1,1]]) IGNORE NULLS; -- { serverError SYNTAX_ERROR } SELECT number, grouping(number % 2, number) RESPECT NULLS AS gr FROM numbers(10) GROUP BY GROUPING SETS ((number), (number % 2)) SETTINGS force_grouping_standard_compatibility = 0; -- { serverError SYNTAX_ERROR } diff --git a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql index 43dad8eb8e0..5492d061c12 100644 --- a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql +++ b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; -- { echoOn } Select sum(number + 1) from numbers(10); diff --git a/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql b/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql index 3daaf36188a..e5cbe100a58 100644 --- a/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql +++ b/tests/queries/0_stateless/02932_parallel_replicas_fuzzer.sql @@ -35,4 +35,4 @@ ORDER BY nan DESC, _CAST([0, NULL, NULL, NULL, NULL], 'Array(Nullable(UInt8))') DESC FORMAT Null -SETTINGS receive_timeout = 10., receive_data_timeout_ms = 10000, use_hedged_requests = 0, allow_suspicious_low_cardinality_types = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, log_queries = 1, table_function_remote_max_addresses = 200, allow_experimental_analyzer = 1; +SETTINGS receive_timeout = 10., receive_data_timeout_ms = 10000, use_hedged_requests = 0, allow_suspicious_low_cardinality_types = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, log_queries = 1, table_function_remote_max_addresses = 200, enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02933_paste_join.sql b/tests/queries/0_stateless/02933_paste_join.sql index 604078d1c3a..6c5a923d028 100644 --- a/tests/queries/0_stateless/02933_paste_join.sql +++ b/tests/queries/0_stateless/02933_paste_join.sql @@ -41,10 +41,10 @@ INSERT INTO test SELECT number from numbers(6); SELECT * FROM (SELECT number FROM test) PASTE JOIN (SELECT number FROM numbers(6) ORDER BY number) SETTINGS joined_subquery_requires_alias = 0; SELECT * FROM (SELECT number FROM test PASTE JOIN (Select number FROM numbers(7))) PASTE JOIN (SELECT number FROM numbers(6) PASTE JOIN (SELECT number FROM test)) SETTINGS joined_subquery_requires_alias = 0; SELECT * FROM (SELECT number FROM test PASTE JOIN (SELECT number FROM test PASTE JOIN (Select number FROM numbers(7)))) PASTE JOIN (SELECT number FROM numbers(6) PASTE JOIN (SELECT number FROM test)) SETTINGS joined_subquery_requires_alias = 0; -SELECT * FROM (SELECT 1 AS a) PASTE JOIN (SELECT 2 AS b) PASTE JOIN (SELECT 3 AS c) SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM (SELECT 1 AS a) PASTE JOIN (SELECT 2 AS b) PASTE JOIN (SELECT 3 AS a) SETTINGS allow_experimental_analyzer = 1; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM (SELECT 1 AS a) PASTE JOIN (SELECT 2 AS b) PASTE JOIN (SELECT 3 AS c) SETTINGS enable_analyzer = 1; +SELECT * FROM (SELECT 1 AS a) PASTE JOIN (SELECT 2 AS b) PASTE JOIN (SELECT 3 AS a) SETTINGS enable_analyzer = 1; -- { serverError AMBIGUOUS_COLUMN_NAME } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE test1 (a Int32) engine=MergeTree order by a; INSERT INTO test1 SELECT * FROM numbers(2); CREATE TABLE test2 (a Int32) engine=MergeTree order by a; diff --git a/tests/queries/0_stateless/02943_order_by_all.sql b/tests/queries/0_stateless/02943_order_by_all.sql index 0d5e0ea52e4..4ce59e84e6c 100644 --- a/tests/queries/0_stateless/02943_order_by_all.sql +++ b/tests/queries/0_stateless/02943_order_by_all.sql @@ -13,40 +13,40 @@ INSERT INTO order_by_all VALUES ('B', 3), ('C', NULL), ('D', 1), ('A', 2); SELECT '-- no modifiers'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT a, b FROM order_by_all ORDER BY ALL; SELECT b, a FROM order_by_all ORDER BY ALL; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT a, b FROM order_by_all ORDER BY ALL; SELECT b, a FROM order_by_all ORDER BY ALL; SELECT '-- with ASC/DESC modifiers'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT a, b FROM order_by_all ORDER BY ALL ASC; SELECT a, b FROM order_by_all ORDER BY ALL DESC; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT a, b FROM order_by_all ORDER BY ALL ASC; SELECT a, b FROM order_by_all ORDER BY ALL DESC; SELECT '-- with NULLS FIRST/LAST modifiers'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT b, a FROM order_by_all ORDER BY ALL NULLS FIRST; SELECT b, a FROM order_by_all ORDER BY ALL NULLS LAST; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT b, a FROM order_by_all ORDER BY ALL NULLS FIRST; SELECT b, a FROM order_by_all ORDER BY ALL NULLS LAST; SELECT '-- SELECT *'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT * FROM order_by_all ORDER BY all; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM order_by_all ORDER BY all; DROP TABLE order_by_all; @@ -65,7 +65,7 @@ INSERT INTO order_by_all VALUES ('B', 3, 10), ('C', NULL, 40), ('D', 1, 20), ('A SELECT ' -- columns'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT a, b, all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT a FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } @@ -73,7 +73,7 @@ SELECT a FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT * FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT * FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT a, b, all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT a FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; @@ -86,30 +86,30 @@ SELECT * FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT ' -- column aliases'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT a, b AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT a, b AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT a, b AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT a, b AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT ' -- expressions'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION } SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false; SELECT ' -- ORDER BY ALL loses its special meaning when used in conjunction with other columns'; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT a, b, all FROM order_by_all ORDER BY all, a; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT a, b, all FROM order_by_all ORDER BY all, a; DROP TABLE order_by_all; diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql index 42175cbb2c6..5ad54c872fa 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql @@ -41,7 +41,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -52,7 +52,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT * FROM @@ -63,7 +63,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -74,7 +74,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT '---'; @@ -96,7 +96,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -107,7 +107,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT * FROM @@ -118,7 +118,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -129,7 +129,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT '---'; @@ -150,7 +150,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM ( @@ -160,7 +160,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; SELECT * FROM @@ -171,7 +171,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 0; + enable_analyzer = 0; SELECT * FROM @@ -182,7 +182,7 @@ FROM WHERE explain LIKE '%Granules: %' SETTINGS - allow_experimental_analyzer = 1; + enable_analyzer = 1; DROP TABLE tokenbf_tab; DROP TABLE ngrambf_tab; diff --git a/tests/queries/0_stateless/02944_variant_as_common_type_analyzer.sql b/tests/queries/0_stateless/02944_variant_as_common_type_analyzer.sql index 77bed6cf796..7947c8a0ce1 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type_analyzer.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type_analyzer.sql @@ -1,7 +1,7 @@ -- this test is just like 02944_variant_as_common_type, but with different expected output, because -- analyzer changes some return types. Specifically, if(c, x, y) always casts to the common type of -- x and y, even if c is constant. -set allow_experimental_analyzer=1; +set enable_analyzer=1; set allow_experimental_variant_type=1; set use_variant_as_common_type=1; @@ -76,4 +76,3 @@ select toTypeName(res), array([1, 2, 3], [[1, 2, 3]]) as res; select toTypeName(res), map('a', 1, 'b', 'str_1') as res; select toTypeName(res), map('a', 1, 'b', map('c', 2, 'd', 'str_1')) as res; select toTypeName(res), map('a', 1, 'b', [1, 2, 3], 'c', [[4, 5, 6]]) as res; - diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.sql b/tests/queries/0_stateless/02952_conjunction_optimization.sql index 94bc352e4c5..fb6c26a2e1f 100644 --- a/tests/queries/0_stateless/02952_conjunction_optimization.sql +++ b/tests/queries/0_stateless/02952_conjunction_optimization.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS 02952_disjunction_optimization; diff --git a/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql index a8029fdd3d6..886944e30c0 100644 --- a/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql +++ b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql @@ -12,4 +12,4 @@ WITH ROLLUP ORDER BY count() ASC, number DESC NULLS LAST - SETTINGS limit = 2, allow_experimental_analyzer = 1; + SETTINGS limit = 2, enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 index e0c6a439112..95a79c56194 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 @@ -10,7 +10,7 @@ a -- { echoOn } -- USING alias column contains default in old analyzer (but both queries below should have the same result) -SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; 738 ba 7386 aa 13332 a @@ -18,11 +18,11 @@ SELECT y * 2, s || 'a' FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2 738 ba 7386 aa 13332 a -SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; (1,369,123,'b',124) (1,3693,1231,'a',0) (1,6666,0,'',48) -SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; (1,369,'b') (1,3693,'a') (1,6666,'') @@ -42,27 +42,27 @@ SELECT (1, t1.*, t2.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2 (1,'',0,6666) (1,'a',3693,0) (1,'b',369,369) -SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; 0 0 43 0 48 0 124 124 0 1232 0 1232 -SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; 126 0 0 42 369 123 b 124 0 3693 1231 a 0 1231 6666 0 48 0 -SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; 0 126 0 42 0 6666 48 0 123 b 369 124 0 1231 a 3693 0 1231 -SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS allow_experimental_analyzer = 1; +SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS enable_analyzer = 1; (1,0,'',126,0,42) (1,0,'',6666,48,0) (1,123,'b',369,124,0) (1,1231,'a',3693,0,1231) -SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; 369 3693 6666 @@ -78,11 +78,11 @@ SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) a b -SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 0; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS enable_analyzer = 1, join_use_nulls = 0; 369 3693 6666 -SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 1; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS enable_analyzer = 1, join_use_nulls = 1; 369 3693 6666 diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 index f5b81231afe..1dde83149b6 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 @@ -24,21 +24,21 @@ SELECT count() FROM t1 INNER JOIN t2 USING (y); SELECT count() FROM t2 INNER JOIN t1 USING (y); -- `SELECT *` works differently for ALIAS columns with analyzer -SELECT * FROM t1 INNER JOIN t2 USING (y, z) SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t2 INNER JOIN t1 USING (y, z) SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 INNER JOIN t2 USING (y, z) SETTINGS enable_analyzer = 1; +SELECT * FROM t2 INNER JOIN t1 USING (y, z) SETTINGS enable_analyzer = 1; SELECT t2.z FROM t1 INNER JOIN t2 USING (y); -SELECT * FROM t1 INNER JOIN t3 USING (y) SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t3 INNER JOIN t1 USING (y, z) SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 INNER JOIN t3 USING (y) SETTINGS enable_analyzer = 1; +SELECT * FROM t3 INNER JOIN t1 USING (y, z) SETTINGS enable_analyzer = 1; SELECT s FROM t1 INNER JOIN t3 USING (y); -- { echoOn } -- USING alias column contains default in old analyzer (but both queries below should have the same result) -SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT y * 2, s || 'a' FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; -SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT (1, t1.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; SELECT (1, t1.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; @@ -46,19 +46,19 @@ SELECT (1, t1.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 U SELECT (1, t1.*, t2.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; SELECT (1, t1.*, t2.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; -SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; -SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; -SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS allow_experimental_analyzer = 1; +SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; +SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS enable_analyzer = 1; +SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT y FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; SELECT s FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; -SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 0; -SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 1; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS enable_analyzer = 1, join_use_nulls = 0; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS enable_analyzer = 1, join_use_nulls = 1; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql index 98259fc8029..0f658379ff9 100644 --- a/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql +++ b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS spark_bar_test; CREATE TABLE spark_bar_test (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date; @@ -9,4 +9,3 @@ SELECT sparkbar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FR SELECT sparkBar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_test GROUP BY event_date); DROP TABLE IF EXISTS spark_bar_test; - diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index 87aef8b1a71..96190241da5 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -9,7 +9,7 @@ CREATE TABLE r (`x` LowCardinality(Nullable(UInt32)), `s` Nullable(String)) ENGI INSERT INTO r SELECT number, toString(number) FROM numbers(2, 8); INSERT INTO r VALUES (NULL, NULL); -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL ; @@ -21,7 +21,7 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL ; diff --git a/tests/queries/0_stateless/02967_analyzer_fuzz.sql b/tests/queries/0_stateless/02967_analyzer_fuzz.sql index 7f2d9afcc71..dab6ec2af22 100644 --- a/tests/queries/0_stateless/02967_analyzer_fuzz.sql +++ b/tests/queries/0_stateless/02967_analyzer_fuzz.sql @@ -7,7 +7,7 @@ GROUP BY makeDateTime64(NULL, NULL, pow(NULL, '257') - '-1', '0.2147483647', 257), makeDateTime64(pow(pow(NULL, '21474836.46') - '0.0000065535', 1048577), '922337203685477580.6', NULL, NULL, pow(NULL, 1.0001) - 65536, NULL) WITH CUBE - SETTINGS allow_experimental_analyzer = 1; + SETTINGS enable_analyzer = 1; CREATE TABLE data_01223 (`key` Int) ENGINE = Memory; @@ -17,4 +17,4 @@ SELECT count(round('92233720368547758.07', '-0.01', NULL, nan, '25.7', '-9223372 FROM dist_01223 WHERE round(NULL, 1025, 1.1754943508222875e-38, NULL) WITH TOTALS - SETTINGS allow_experimental_analyzer = 1; + SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh index e49a340ab67..e954cb0e78e 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh @@ -26,7 +26,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, +SETTINGS enable_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" @@ -35,7 +35,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS enable_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -51,7 +51,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, +SETTINGS enable_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" @@ -60,7 +60,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS enable_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -77,7 +77,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" @@ -86,7 +86,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -102,7 +102,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" @@ -111,7 +111,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -127,7 +127,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" @@ -136,7 +136,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 100e4e500cd..36f02b2f764 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -2,7 +2,7 @@ set parallel_replicas_prefer_local_join = 0; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -18,7 +18,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression ReadFromRemoteParallelReplicas -- @@ -29,7 +29,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -52,7 +52,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -65,7 +65,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 54 54 50 50 12 12 0 64 64 0 0 0 0 1 explain description=0 @@ -75,7 +75,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -90,7 +90,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -113,7 +113,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -130,7 +130,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -153,7 +153,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -174,7 +174,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 6 6 6 6 0 0 8 8 8 8 0 0 @@ -197,7 +197,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Join Expression @@ -215,7 +215,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 @@ -237,7 +237,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -258,7 +258,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -281,7 +281,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -294,7 +294,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -317,7 +317,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } Expression Sorting Expression @@ -339,7 +339,7 @@ Expression ReadFromRemoteParallelReplicas set parallel_replicas_prefer_local_join = 1; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -355,7 +355,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression ReadFromRemoteParallelReplicas -- @@ -366,7 +366,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -389,7 +389,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -402,7 +402,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 54 54 50 50 12 12 0 64 64 0 0 0 0 1 explain description=0 @@ -412,7 +412,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -427,7 +427,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -450,7 +450,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -467,7 +467,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -490,7 +490,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -511,7 +511,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 6 6 6 6 0 0 8 8 8 8 0 0 @@ -534,7 +534,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Join Expression @@ -552,7 +552,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 @@ -574,7 +574,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -595,7 +595,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -618,7 +618,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting Expression @@ -631,7 +631,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; 0 0 0 0 0 0 1 1 0 0 0 0 3 3 0 0 0 0 @@ -654,7 +654,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; Expression Sorting Expression diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 54505b147a3..775663768fe 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -17,8 +17,8 @@ insert into tab3 select number * 4, number * 4 from numbers(4); set parallel_replicas_prefer_local_join = {{use_global_in}}; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -27,7 +27,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -36,7 +36,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -45,7 +45,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -54,7 +54,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) select * from sub5 order by key -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -63,7 +63,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -72,7 +72,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -81,7 +81,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -90,7 +90,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -99,7 +99,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -108,7 +108,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -116,7 +116,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x != 2), @@ -124,7 +124,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) -select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- Subqueries for IN allowed @@ -134,7 +134,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -143,7 +143,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -- -- Subqueries for IN are not allowed @@ -153,7 +153,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; explain description=0 with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -162,6 +162,6 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) select * from sub5 order by x -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1, parallel_replicas_allow_in_with_subquery=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; {%- endfor %} diff --git a/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql index 15f2550a63e..a7d0c7793db 100644 --- a/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql +++ b/tests/queries/0_stateless/02969_analyzer_eliminate_injective_functions.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; EXPLAIN QUERY TREE SELECT toString(toString(number + 1)) as val, count() diff --git a/tests/queries/0_stateless/02969_functions_to_subcolumns_if_null.sql b/tests/queries/0_stateless/02969_functions_to_subcolumns_if_null.sql index 361fd7c7a4e..859d26af1a1 100644 --- a/tests/queries/0_stateless/02969_functions_to_subcolumns_if_null.sql +++ b/tests/queries/0_stateless/02969_functions_to_subcolumns_if_null.sql @@ -7,21 +7,21 @@ INSERT INTO t_subcolumns_if SELECT number::Nullable(Int64) as number FROM number SELECT sum(multiIf(id IS NOT NULL, 1, 0)) FROM t_subcolumns_if -SETTINGS allow_experimental_analyzer = 1, optimize_functions_to_subcolumns = 1; +SETTINGS enable_analyzer = 1, optimize_functions_to_subcolumns = 1; SELECT sum(multiIf(id IS NULL, 1, 0)) FROM t_subcolumns_if -SETTINGS allow_experimental_analyzer = 0, optimize_functions_to_subcolumns = 1; +SETTINGS enable_analyzer = 0, optimize_functions_to_subcolumns = 1; SELECT sum(multiIf(id IS NULL, 1, 0)) FROM t_subcolumns_if -SETTINGS allow_experimental_analyzer = 1, optimize_functions_to_subcolumns = 0; +SETTINGS enable_analyzer = 1, optimize_functions_to_subcolumns = 0; SELECT sum(multiIf(id IS NULL, 1, 0)) FROM t_subcolumns_if -SETTINGS allow_experimental_analyzer = 1, optimize_functions_to_subcolumns = 1; +SETTINGS enable_analyzer = 1, optimize_functions_to_subcolumns = 1; DROP TABLE IF EXISTS t_subcolumns_if; diff --git a/tests/queries/0_stateless/02971_analyzer_remote_id.sh b/tests/queries/0_stateless/02971_analyzer_remote_id.sh index ab3c5292529..6d504ce3568 100755 --- a/tests/queries/0_stateless/02971_analyzer_remote_id.sh +++ b/tests/queries/0_stateless/02971_analyzer_remote_id.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="CREATE TABLE ${CLICKHOUSE_DATABASE}.x ENGINE = MergeTree() ORDER BY number AS SELECT * FROM numbers(2)" -${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}', '${CLICKHOUSE_DATABASE}.x') SETTINGS allow_experimental_analyzer = 1" 2>&1 \ +${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}', '${CLICKHOUSE_DATABASE}.x') SETTINGS enable_analyzer = 1" 2>&1 \ | grep -av "ASan doesn't fully support makecontext/swapcontext functions" diff --git a/tests/queries/0_stateless/02971_functions_to_subcolumns_column_names.sql b/tests/queries/0_stateless/02971_functions_to_subcolumns_column_names.sql index 48e5232d18b..6df2f27dbbf 100644 --- a/tests/queries/0_stateless/02971_functions_to_subcolumns_column_names.sql +++ b/tests/queries/0_stateless/02971_functions_to_subcolumns_column_names.sql @@ -5,7 +5,7 @@ CREATE TABLE t_column_names (arr Array(UInt64), n Nullable(String)) ENGINE = Mem INSERT INTO t_column_names VALUES ([1, 2, 3], 'foo'); SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE dump_tree = 0, dump_ast = 1 SELECT length(arr), isNull(n) FROM t_column_names; SELECT length(arr), isNull(n) FROM t_column_names FORMAT JSONEachRow; diff --git a/tests/queries/0_stateless/02971_functions_to_subcolumns_map.sql b/tests/queries/0_stateless/02971_functions_to_subcolumns_map.sql index e8a752a82d5..c53a03b8ccd 100644 --- a/tests/queries/0_stateless/02971_functions_to_subcolumns_map.sql +++ b/tests/queries/0_stateless/02971_functions_to_subcolumns_map.sql @@ -5,7 +5,7 @@ CREATE TABLE t_func_to_subcolumns_map (id UInt64, m Map(String, UInt64)) ENGINE INSERT INTO t_func_to_subcolumns_map VALUES (1, map('aaa', 1, 'bbb', 2)) (2, map('ccc', 3)); SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE dump_tree = 0, dump_ast = 1 SELECT length(m) FROM t_func_to_subcolumns_map; SELECT length(m) FROM t_func_to_subcolumns_map; diff --git a/tests/queries/0_stateless/02971_functions_to_subcolumns_variant.sql b/tests/queries/0_stateless/02971_functions_to_subcolumns_variant.sql index 511bcc44514..2612664a7b2 100644 --- a/tests/queries/0_stateless/02971_functions_to_subcolumns_variant.sql +++ b/tests/queries/0_stateless/02971_functions_to_subcolumns_variant.sql @@ -7,7 +7,7 @@ CREATE TABLE t_func_to_subcolumns_variant (id UInt64, v Variant(String, UInt64)) INSERT INTO t_func_to_subcolumns_variant VALUES (1, 'foo') (2, 111); SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE dump_tree = 0, dump_ast = 1 SELECT variantElement(v, 'String') FROM t_func_to_subcolumns_variant; SELECT variantElement(v, 'String') FROM t_func_to_subcolumns_variant; diff --git a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql index 083b0ecc5c9..767cd61216b 100644 --- a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql +++ b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql @@ -15,15 +15,15 @@ WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; --- Testing that it is disabled for allow_experimental_analyzer=0. With analyzer it will be supported (with correct result) +-- Testing that it is disabled for enable_analyzer=0. With analyzer it will be supported (with correct result) WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS enable_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } -- Disabled for any value of allow_experimental_parallel_reading_from_replicas != 1, not just 2 WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 512, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS enable_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 512, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } -- Sanitizer SELECT count() FROM pr_2 JOIN numbers(10) as pr_1 ON pr_2.a = pr_1.number diff --git a/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql b/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql index 30fb3c76c1f..1fd103d0bb0 100644 --- a/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql +++ b/tests/queries/0_stateless/02974_analyzer_array_join_subcolumn.sql @@ -7,16 +7,16 @@ INSERT INTO t2 VALUES (1, {'a': (1, 2), 'b': (2, 3)}),; CREATE TABLE t3 (id Int32, c Tuple(v String, pe Map(String, Tuple(a UInt64, b UInt64)))) ENGINE = MergeTree ORDER BY id; INSERT INTO t3 VALUES (1, ('A', {'a':(1, 2),'b':(2, 3)})); -SELECT pe, pe.values.a FROM (SELECT * FROM t2) ARRAY JOIN pe SETTINGS allow_experimental_analyzer = 1; -SELECT p, p.values.a FROM (SELECT * FROM t2) ARRAY JOIN pe AS p SETTINGS allow_experimental_analyzer = 1; +SELECT pe, pe.values.a FROM (SELECT * FROM t2) ARRAY JOIN pe SETTINGS enable_analyzer = 1; +SELECT p, p.values.a FROM (SELECT * FROM t2) ARRAY JOIN pe AS p SETTINGS enable_analyzer = 1; SELECT pe, pe.values.a FROM t2 ARRAY JOIN pe; SELECT p, p.values.a FROM t2 ARRAY JOIN pe AS p; -SELECT c.pe, c.pe.values.a FROM (SELECT * FROM t3) ARRAY JOIN c.pe SETTINGS allow_experimental_analyzer = 1; -SELECT p, p.values.a FROM (SELECT * FROM t3) ARRAY JOIN c.pe as p SETTINGS allow_experimental_analyzer = 1; +SELECT c.pe, c.pe.values.a FROM (SELECT * FROM t3) ARRAY JOIN c.pe SETTINGS enable_analyzer = 1; +SELECT p, p.values.a FROM (SELECT * FROM t3) ARRAY JOIN c.pe as p SETTINGS enable_analyzer = 1; -SELECT c.pe, c.pe.values.a FROM t3 ARRAY JOIN c.pe SETTINGS allow_experimental_analyzer = 1; +SELECT c.pe, c.pe.values.a FROM t3 ARRAY JOIN c.pe SETTINGS enable_analyzer = 1; SELECT p, p.values.a FROM t3 ARRAY JOIN c.pe as p; diff --git a/tests/queries/0_stateless/02987_logical_optimizer_pass_lowcardinality.sql b/tests/queries/0_stateless/02987_logical_optimizer_pass_lowcardinality.sql index be7689025b2..266270562cc 100644 --- a/tests/queries/0_stateless/02987_logical_optimizer_pass_lowcardinality.sql +++ b/tests/queries/0_stateless/02987_logical_optimizer_pass_lowcardinality.sql @@ -2,4 +2,4 @@ CREATE TABLE 02987_logical_optimizer_table (key Int, value Int) ENGINE=Memory(); CREATE VIEW v1 AS SELECT * FROM 02987_logical_optimizer_table; CREATE TABLE 02987_logical_optimizer_merge AS v1 ENGINE=Merge(currentDatabase(), 'v1'); -SELECT _table, key FROM 02987_logical_optimizer_merge WHERE (_table = toFixedString(toFixedString(toFixedString('v1', toNullable(2)), 2), 2)) OR ((value = toLowCardinality(toNullable(10))) AND (_table = toFixedString(toNullable('v3'), 2))) OR ((value = 20) AND (_table = toFixedString(toFixedString(toFixedString('v1', 2), 2), 2)) AND (_table = toFixedString(toLowCardinality(toFixedString('v3', 2)), 2))) SETTINGS allow_experimental_analyzer = true, join_use_nulls = true, convert_query_to_cnf = true; +SELECT _table, key FROM 02987_logical_optimizer_merge WHERE (_table = toFixedString(toFixedString(toFixedString('v1', toNullable(2)), 2), 2)) OR ((value = toLowCardinality(toNullable(10))) AND (_table = toFixedString(toNullable('v3'), 2))) OR ((value = 20) AND (_table = toFixedString(toFixedString(toFixedString('v1', 2), 2), 2)) AND (_table = toFixedString(toLowCardinality(toFixedString('v3', 2)), 2))) SETTINGS enable_analyzer = true, join_use_nulls = true, convert_query_to_cnf = true; diff --git a/tests/queries/0_stateless/02989_join_using_parent_scope.reference b/tests/queries/0_stateless/02989_join_using_parent_scope.reference index 965bfaf7c43..72678a1db4d 100644 --- a/tests/queries/0_stateless/02989_join_using_parent_scope.reference +++ b/tests/queries/0_stateless/02989_join_using_parent_scope.reference @@ -23,7 +23,7 @@ SELECT b AS a, a FROM tb JOIN tabc USING (a) ORDER BY ALL; 3 3 SELECT 1 AS b FROM tb JOIN ta USING (b); -- { serverError UNKNOWN_IDENTIFIER } -- SELECT * returns all columns from both tables in new analyzer -SELECT 3 AS a, a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT 3 AS a, a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 3 3 0 3 abc3 3 3 1 3 abc3 3 3 2 3 abc3 @@ -31,57 +31,57 @@ SELECT 3 AS a, a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS allow 3 3 \N 0 abc0 3 3 \N 1 abc1 3 3 \N 2 abc2 -SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 abc1 2 1 2 abc2 3 2 3 abc3 -SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 abc1 2 1 2 abc2 3 2 3 abc3 -SELECT b + 1 AS a, * FROM tb LEFT JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb LEFT JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 abc1 2 1 2 abc2 3 2 3 abc3 4 3 \N \N -SELECT b + 1 AS a, * FROM tb RIGHT JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb RIGHT JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 abc1 2 1 2 abc2 3 2 3 abc3 \N \N 0 abc0 -SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 abc1 2 1 2 abc2 3 2 3 abc3 4 3 \N \N \N \N 0 abc0 -SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS asterisk_include_alias_columns = 1, allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS asterisk_include_alias_columns = 1, enable_analyzer = 1; 1 0 1 abc1 2 3 2 1 2 abc2 3 4 3 2 3 abc3 4 5 4 3 \N \N \N \N \N \N 0 abc0 1 2 -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 2 2 1 2 3 3 2 3 4 -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 LEFT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 LEFT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 2 2 1 2 3 3 2 3 4 4 3 \N \N -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 RIGHT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 RIGHT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 2 2 1 2 3 3 2 3 4 \N \N 0 1 -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 FULL JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 FULL JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; 1 0 1 2 2 1 2 3 3 2 3 4 4 3 \N \N \N \N 0 1 -SELECT b + 1 AS a, s FROM tb FULL OUTER JOIN tabc USING (a) PREWHERE a > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, s FROM tb FULL OUTER JOIN tabc USING (a) PREWHERE a > 2 ORDER BY ALL SETTINGS enable_analyzer = 1; 3 abc3 4 \N \N abc0 @@ -94,7 +94,7 @@ SELECT a + 2 AS b FROM tb JOIN tabc USING (b) ORDER BY ALL SETTINGS analyzer_compatibility_join_using_top_level_identifier = 1; -- { serverError UNKNOWN_IDENTIFIER } -- In new analyzer with `analyzer_compatibility_join_using_top_level_identifier = 0` we get `b` from left table SELECT a + 2 AS b FROM tb JOIN tabc USING (b) ORDER BY ALL -SETTINGS analyzer_compatibility_join_using_top_level_identifier = 0, allow_experimental_analyzer = 1; +SETTINGS analyzer_compatibility_join_using_top_level_identifier = 0, enable_analyzer = 1; 2 3 4 @@ -108,7 +108,7 @@ SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 1; +SETTINGS enable_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 1; uid name u2.uid u2.name Int16 String Int16 String 1231 Ksenia 6666 Ksenia @@ -116,7 +116,7 @@ SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 0; +SETTINGS enable_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 0; uid name u2.uid u2.name Int16 String Int16 String 1231 Ksenia 1231 John @@ -125,7 +125,7 @@ SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 0; +SETTINGS enable_analyzer = 0; uid name u2.uid u2.name Int16 String Int16 String 1231 Ksenia 6666 Ksenia diff --git a/tests/queries/0_stateless/02989_join_using_parent_scope.sql b/tests/queries/0_stateless/02989_join_using_parent_scope.sql index 2e4d9f097f7..4283d8b6eca 100644 --- a/tests/queries/0_stateless/02989_join_using_parent_scope.sql +++ b/tests/queries/0_stateless/02989_join_using_parent_scope.sql @@ -24,21 +24,21 @@ SELECT b AS a, a FROM tb JOIN tabc USING (a) ORDER BY ALL; SELECT 1 AS b FROM tb JOIN ta USING (b); -- { serverError UNKNOWN_IDENTIFIER } -- SELECT * returns all columns from both tables in new analyzer -SELECT 3 AS a, a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT 3 AS a, a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; -SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM tb LEFT JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM tb RIGHT JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS asterisk_include_alias_columns = 1, allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM tb JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM tb LEFT JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM tb RIGHT JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM tb FULL JOIN tabc USING (a) ORDER BY ALL SETTINGS asterisk_include_alias_columns = 1, enable_analyzer = 1; -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 LEFT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 RIGHT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; -SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 FULL JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 LEFT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 RIGHT JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT b + 1 AS a, * FROM (SELECT b FROM tb) t1 FULL JOIN (SELECT a, b FROM tabc) t2 USING (a) ORDER BY ALL SETTINGS enable_analyzer = 1; -SELECT b + 1 AS a, s FROM tb FULL OUTER JOIN tabc USING (a) PREWHERE a > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT b + 1 AS a, s FROM tb FULL OUTER JOIN tabc USING (a) PREWHERE a > 2 ORDER BY ALL SETTINGS enable_analyzer = 1; -- It's a default behavior for old analyzer and new with analyzer_compatibility_join_using_top_level_identifier @@ -49,7 +49,7 @@ SETTINGS analyzer_compatibility_join_using_top_level_identifier = 1; -- { server -- In new analyzer with `analyzer_compatibility_join_using_top_level_identifier = 0` we get `b` from left table SELECT a + 2 AS b FROM tb JOIN tabc USING (b) ORDER BY ALL -SETTINGS analyzer_compatibility_join_using_top_level_identifier = 0, allow_experimental_analyzer = 1; +SETTINGS analyzer_compatibility_join_using_top_level_identifier = 0, enable_analyzer = 1; -- This is example where query may return different results with different `analyzer_compatibility_join_using_top_level_identifier` @@ -63,19 +63,19 @@ SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 1; +SETTINGS enable_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 1; SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 0; +SETTINGS enable_analyzer = 1, analyzer_compatibility_join_using_top_level_identifier = 0; SELECT u1.uid, u1.spouse_name as name, u2.uid, u2.name FROM users u1 JOIN users u2 USING (name) ORDER BY u1.uid FORMAT TSVWithNamesAndTypes -SETTINGS allow_experimental_analyzer = 0; +SETTINGS enable_analyzer = 0; DROP TABLE IF EXISTS users; diff --git a/tests/queries/0_stateless/02991_count_rewrite_analyzer.sql b/tests/queries/0_stateless/02991_count_rewrite_analyzer.sql index b11aeedd225..bb0d3a1a9d1 100644 --- a/tests/queries/0_stateless/02991_count_rewrite_analyzer.sql +++ b/tests/queries/0_stateless/02991_count_rewrite_analyzer.sql @@ -1,5 +1,5 @@ -- Regression test for https://github.com/ClickHouse/ClickHouse/issues/59919 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT toTypeName(sum(toNullable('a') IN toNullable('a'))) AS x; SELECT toTypeName(count(toNullable('a') IN toNullable('a'))) AS x; diff --git a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql index 2a9e673d7bc..efe18918c93 100644 --- a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql +++ b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; -- Illegal column String of first argument of function concatWithSeparator. Must be a constant String. SELECT concatWithSeparator('a', 'b') GROUP BY 'a'; diff --git a/tests/queries/0_stateless/02996_analyzer_prewhere_projection.sql b/tests/queries/0_stateless/02996_analyzer_prewhere_projection.sql index 9d676001010..66f7c37a893 100644 --- a/tests/queries/0_stateless/02996_analyzer_prewhere_projection.sql +++ b/tests/queries/0_stateless/02996_analyzer_prewhere_projection.sql @@ -4,4 +4,4 @@ CREATE TABLE t__fuzz_0 (`i` LowCardinality(Int32), `j` Int32, `k` Int32, PROJECT INSERT INTO t__fuzz_0 Select number, number, number FROM numbers(100); SELECT * FROM t__fuzz_0 PREWHERE 7 AND (i < 2147483647) AND (j IN (2147483646, -2, 1)) -SETTINGS allow_experimental_analyzer = true; +SETTINGS enable_analyzer = true; diff --git a/tests/queries/0_stateless/02998_analyzer_prewhere_report.sql b/tests/queries/0_stateless/02998_analyzer_prewhere_report.sql index b3027181901..5a7cab854e9 100644 --- a/tests/queries/0_stateless/02998_analyzer_prewhere_report.sql +++ b/tests/queries/0_stateless/02998_analyzer_prewhere_report.sql @@ -15,4 +15,4 @@ SELECT arrayFilter(x -> (x IN (2, 3)), data) AS filtered FROM hits WHERE arrayExists(x -> (x IN (2, 3)), data) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference index ffd2f68990b..ccd9540cb49 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference @@ -1,5 +1,5 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); QUERY id: 0 PROJECTION COLUMNS @@ -7,7 +7,7 @@ QUERY id: 0 encrypt(\'aes-256-ofb\', [HIDDEN id: 3], [HIDDEN id: 2]) Nullable(String) PROJECTION LIST id: 1, nodes: 2 - CONSTANT id: 2, constant_value: \'\\n&\', constant_value_type: Nullable(String) + CONSTANT id: 2, constant_value: \'\\n��&\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -15,7 +15,7 @@ QUERY id: 0 CONSTANT id: 5, constant_value: \'aes-256-ofb\', constant_value_type: String CONSTANT id: 6, constant_value: [HIDDEN id: 1], constant_value_type: Nullable(String) CONSTANT id: 7, constant_value: [HIDDEN id: 2], constant_value_type: String - CONSTANT id: 8, constant_value: \'\', constant_value_type: Nullable(String) + CONSTANT id: 8, constant_value: \'��\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 9, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -33,7 +33,7 @@ QUERY id: 0 encrypt(\'aes-256-ofb\', _subquery_2, \'12345678901234567890123456789012\') Nullable(String) PROJECTION LIST id: 1, nodes: 2 - CONSTANT id: 2, constant_value: \'\\n&\', constant_value_type: Nullable(String) + CONSTANT id: 2, constant_value: \'\\n��&\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 JOIN TREE TABLE id: 10, table_name: system.one CONSTANT id: 11, constant_value: \'12345678901234567890123456789012\', constant_value_type: String - CONSTANT id: 12, constant_value: \'\', constant_value_type: Nullable(String) + CONSTANT id: 12, constant_value: \'��\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 13, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql index f40b40b6c8c..a216f886f8a 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.sql @@ -2,7 +2,7 @@ -- encrypt function doesn't exist in the fastest build -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE SELECT encrypt('aes-256-ofb', (SELECT 'qwerty'), '12345678901234567890123456789012'), encrypt('aes-256-ofb', (SELECT 'asdf'), '12345678901234567890123456789012'); diff --git a/tests/queries/0_stateless/02998_projection_after_attach_partition.reference b/tests/queries/0_stateless/02998_projection_after_attach_partition.reference index 1cb984f0f34..93cae129842 100644 --- a/tests/queries/0_stateless/02998_projection_after_attach_partition.reference +++ b/tests/queries/0_stateless/02998_projection_after_attach_partition.reference @@ -19,12 +19,12 @@ INSERT INTO visits_order SELECT 2, 'user2', number from numbers(1, 10); INSERT INTO visits_order SELECT 2, 'another_user2', number*2 from numbers(1, 10); INSERT INTO visits_order SELECT 2, 'yet_another_user2', number*3 from numbers(1, 10); ALTER TABLE visits_order_dst ATTACH PARTITION ID '2' FROM visits_order; -SET allow_experimental_analyzer=0; +SET enable_analyzer=0; EXPLAIN SELECT * FROM visits_order_dst WHERE user_name='another_user2'; Expression ((Projection + Before ORDER BY)) Filter ReadFromMergeTree (user_name_projection) -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; EXPLAIN SELECT * FROM visits_order_dst WHERE user_name='another_user2'; Expression ((Project names + Projection)) Filter diff --git a/tests/queries/0_stateless/02998_projection_after_attach_partition.sql b/tests/queries/0_stateless/02998_projection_after_attach_partition.sql index 4e0121dafe9..72ee4ad81e8 100644 --- a/tests/queries/0_stateless/02998_projection_after_attach_partition.sql +++ b/tests/queries/0_stateless/02998_projection_after_attach_partition.sql @@ -25,10 +25,10 @@ INSERT INTO visits_order SELECT 2, 'yet_another_user2', number*3 from numbers(1, ALTER TABLE visits_order_dst ATTACH PARTITION ID '2' FROM visits_order; -SET allow_experimental_analyzer=0; +SET enable_analyzer=0; EXPLAIN SELECT * FROM visits_order_dst WHERE user_name='another_user2'; -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; EXPLAIN SELECT * FROM visits_order_dst WHERE user_name='another_user2'; diff --git a/tests/queries/0_stateless/02999_analyzer_preimage_null.sql b/tests/queries/0_stateless/02999_analyzer_preimage_null.sql index 07d3a0f69c1..0fc61cf0836 100644 --- a/tests/queries/0_stateless/02999_analyzer_preimage_null.sql +++ b/tests/queries/0_stateless/02999_analyzer_preimage_null.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET optimize_time_filter_with_preimage=1; CREATE TABLE date_t__fuzz_0 (`id` UInt32, `value1` String, `date1` Date) ENGINE = ReplacingMergeTree ORDER BY id SETTINGS allow_nullable_key=1; diff --git a/tests/queries/0_stateless/03001_analyzer_nullable_nothing.sql b/tests/queries/0_stateless/03001_analyzer_nullable_nothing.sql index 32c378ebf0a..c1c7ca87b5f 100644 --- a/tests/queries/0_stateless/03001_analyzer_nullable_nothing.sql +++ b/tests/queries/0_stateless/03001_analyzer_nullable_nothing.sql @@ -3,4 +3,4 @@ SELECT count(_CAST(NULL, 'Nullable(Nothing)')), round(avg(_CAST(NULL, 'Nullable(Nothing)'))) AS k FROM numbers(256) - SETTINGS allow_experimental_analyzer = 1; + SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03002_analyzer_prewhere.sql b/tests/queries/0_stateless/03002_analyzer_prewhere.sql index 0edf16f1cbe..976e7cab73d 100644 --- a/tests/queries/0_stateless/03002_analyzer_prewhere.sql +++ b/tests/queries/0_stateless/03002_analyzer_prewhere.sql @@ -1,4 +1,4 @@ -SET max_threads = 16, receive_timeout = 10., receive_data_timeout_ms = 10000, allow_suspicious_low_cardinality_types = true, enable_positional_arguments = false, log_queries = true, table_function_remote_max_addresses = 200, any_join_distinct_right_table_keys = true, joined_subquery_requires_alias = false, allow_experimental_analyzer = true, max_execution_time = 10., max_memory_usage = 10000000000, log_comment = '/workspace/ch/tests/queries/0_stateless/01710_projection_in_index.sql', send_logs_level = 'fatal', enable_optimize_predicate_expression = false, prefer_localhost_replica = true, allow_introspection_functions = true, optimize_functions_to_subcolumns = false, transform_null_in = true, optimize_use_projections = true, allow_deprecated_syntax_for_merge_tree = true, parallelize_output_from_storages = false; +SET max_threads = 16, receive_timeout = 10., receive_data_timeout_ms = 10000, allow_suspicious_low_cardinality_types = true, enable_positional_arguments = false, log_queries = true, table_function_remote_max_addresses = 200, any_join_distinct_right_table_keys = true, joined_subquery_requires_alias = false, enable_analyzer = true, max_execution_time = 10., max_memory_usage = 10000000000, log_comment = '/workspace/ch/tests/queries/0_stateless/01710_projection_in_index.sql', send_logs_level = 'fatal', enable_optimize_predicate_expression = false, prefer_localhost_replica = true, allow_introspection_functions = true, optimize_functions_to_subcolumns = false, transform_null_in = true, optimize_use_projections = true, allow_deprecated_syntax_for_merge_tree = true, parallelize_output_from_storages = false; CREATE TABLE t__fuzz_0 (`i` Int32, `j` Nullable(Int32), `k` Int32, PROJECTION p (SELECT * ORDER BY j)) ENGINE = MergeTree ORDER BY i SETTINGS index_granularity = 1, allow_nullable_key=1; diff --git a/tests/queries/0_stateless/03003_analyzer_setting.sql b/tests/queries/0_stateless/03003_analyzer_setting.sql index 2e5cab71277..3dbdaed4ad0 100644 --- a/tests/queries/0_stateless/03003_analyzer_setting.sql +++ b/tests/queries/0_stateless/03003_analyzer_setting.sql @@ -1,9 +1,9 @@ CREATE TABLE test (dummy Int8) ENGINE = Distributed(test_cluster_two_shards, 'system', 'one'); -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; -SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 1); -- { serverError INCORRECT_QUERY } +SELECT * FROM (SELECT * FROM test SETTINGS enable_analyzer = 1); -- { serverError INCORRECT_QUERY } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -SELECT * FROM (SELECT * FROM test SETTINGS allow_experimental_analyzer = 0); -- { serverError INCORRECT_QUERY } +SELECT * FROM (SELECT * FROM test SETTINGS enable_analyzer = 0); -- { serverError INCORRECT_QUERY } diff --git a/tests/queries/0_stateless/03003_functions_to_subcolumns_final.sql b/tests/queries/0_stateless/03003_functions_to_subcolumns_final.sql index 3fe29139c5f..b2ca478daa4 100644 --- a/tests/queries/0_stateless/03003_functions_to_subcolumns_final.sql +++ b/tests/queries/0_stateless/03003_functions_to_subcolumns_final.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_length_1; DROP TABLE IF EXISTS t_length_2; SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_on_insert = 0; CREATE TABLE t_length_1 (id UInt64, arr Array(UInt64)) ENGINE = ReplacingMergeTree ORDER BY id; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 index f15fced161c..a09dc18739f 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 @@ -18,7 +18,7 @@ INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET allow_experimental_join_condition=1; SET join_use_nulls=0; -- { echoOn } diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 index a97153ce3aa..009ae10e4ff 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 @@ -18,7 +18,7 @@ INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET allow_experimental_join_condition=1; SET join_use_nulls=0; -- { echoOn } diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 index 3235019821b..37eaaa8ab5c 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -15,7 +15,7 @@ CREATE TABLE t2 ( INSERT INTO t1 (key, a) VALUES (1, 10), (2, 15), (3, 20); INSERT INTO t2 (key, a) VALUES (1, 5), (2, 10), (4, 25); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET allow_experimental_join_condition=1; SET join_algorithm='hash'; -- { echoOn } 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 a363101ca69..61ad5ec0bf1 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 @@ -6,7 +6,7 @@ INSERT INTO t1 VALUES ('key1', 'a', 1, 1, 2), ('key1', 'b', 2, 3, 2), ('key1', ' CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key; INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET allow_experimental_join_condition=1; SET join_use_nulls=0; -- { echoOn } diff --git a/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql b/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql index df4ec9d26a3..7c8d6dd9aff 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_cte_explain_syntax_crash.sql @@ -20,7 +20,7 @@ ORDER BY n AS SELECT * FROM numbers(10); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=0; EXPLAIN SYNTAX diff --git a/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql index 9479044e0e0..44f6642d2a5 100644 --- a/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql +++ b/tests/queries/0_stateless/03007_column_nullable_uninitialzed_value.sql @@ -1 +1 @@ -SELECT count(NULL) IGNORE NULLS > avg(toDecimal32(NULL)) IGNORE NULLS, count() FROM numbers(1000) WITH TOTALS SETTINGS allow_experimental_analyzer = 1; +SELECT count(NULL) IGNORE NULLS > avg(toDecimal32(NULL)) IGNORE NULLS, count() FROM numbers(1000) WITH TOTALS SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql index 71a175faac8..5ec6ee5a996 100644 --- a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql +++ b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.sql @@ -1,11 +1,11 @@ SET optimize_rewrite_sum_if_to_count_if = 1; -SET allow_experimental_analyzer = 0; +SET enable_analyzer = 0; SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); SELECT (tuple(sum(if((number % 2) = 0, toNullable(0), 123)) IGNORE NULLS), toUInt8(3)) FROM numbers(100); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); EXPLAIN QUERY TREE SELECT (sumIf(toNullable(1), (number % 2) = 0), NULL) FROM numbers(10); SELECT (sum(if((number % 2) = 0, toNullable(1), 0)), NULL) FROM numbers(10); diff --git a/tests/queries/0_stateless/03013_group_by_use_nulls_with_materialize_and_analyzer.sql b/tests/queries/0_stateless/03013_group_by_use_nulls_with_materialize_and_analyzer.sql index b15593b2abd..7b57dbd807d 100644 --- a/tests/queries/0_stateless/03013_group_by_use_nulls_with_materialize_and_analyzer.sql +++ b/tests/queries/0_stateless/03013_group_by_use_nulls_with_materialize_and_analyzer.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; set group_by_use_nulls = 1; set optimize_group_by_function_keys = 1; set optimize_injective_functions_in_group_by = 1; @@ -8,4 +8,3 @@ SELECT materialize(3) from numbers(10) GROUP BY GROUPING SETS (('str'), (materia SELECT ignore(3) from numbers(10) GROUP BY GROUPING SETS (('str'), (ignore(3))) order by all; SELECT materialize(ignore(3)) from numbers(10) GROUP BY GROUPING SETS (('str'), (materialize(ignore(3)))) order by all; SELECT ignore(materialize(3)) from numbers(10) GROUP BY GROUPING SETS (('str'), (ignore(materialize(3)))) order by all; - diff --git a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql index 094614cb78d..295f89c5a0a 100644 --- a/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql +++ b/tests/queries/0_stateless/03014_analyzer_groupby_fuzz_60317.sql @@ -6,7 +6,7 @@ SELECT FROM system.one GROUP BY _CAST(30, 'Nullable(UInt8)') -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; -- WITH CUBE (note that result is different with the analyzer (analyzer is correct including all combinations) SELECT @@ -24,4 +24,4 @@ GROUP BY _CAST(30, 'Nullable(UInt8)') WITH CUBE WITH TOTALS -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03014_group_by_use_nulls_injective_functions_and_analyzer.sql b/tests/queries/0_stateless/03014_group_by_use_nulls_injective_functions_and_analyzer.sql index e7ea964b876..d700f9ba3b1 100644 --- a/tests/queries/0_stateless/03014_group_by_use_nulls_injective_functions_and_analyzer.sql +++ b/tests/queries/0_stateless/03014_group_by_use_nulls_injective_functions_and_analyzer.sql @@ -1,5 +1,4 @@ -set allow_experimental_analyzer=1, group_by_use_nulls=1, optimize_injective_functions_in_group_by=1; +set enable_analyzer=1, group_by_use_nulls=1, optimize_injective_functions_in_group_by=1; SELECT bitNot(bitNot(number)) + 3 FROM numbers(10) GROUP BY GROUPING SETS (('str', bitNot(bitNot(number))), ('str')) order by all; SELECT tuple(tuple(tuple(number))) FROM numbers(10) GROUP BY GROUPING SETS (('str', tuple(tuple(number))), ('str')) order by all; SELECT materialize(3) + 3 FROM numbers(10) GROUP BY GROUPING SETS (('str', materialize(materialize(3))), ('str')) order by all; - diff --git a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql index d3bd9ef0ce3..5190b8635d6 100644 --- a/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql +++ b/tests/queries/0_stateless/03015_analyzer_groupby_fuzz_60772.sql @@ -7,7 +7,7 @@ GROUP BY toFixedString(toFixedString('2018-01-02 22:33:44', 19), 19), 'gr', '2018-01-02 22:33:44' -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; -- WITH CUBE (note that result is different with the analyzer (analyzer is correct including all combinations) SELECT @@ -20,4 +20,4 @@ GROUP BY 'gr', '2018-01-02 22:33:44' WITH CUBE -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql index c00a75a631c..6c926c32887 100644 --- a/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql +++ b/tests/queries/0_stateless/03016_analyzer_groupby_fuzz_59796.sql @@ -3,4 +3,4 @@ SELECT GROUP BY concat(unhex('00'), toFixedString(materialize(toFixedString(' key="v" ', 9)), 9), toFixedString(toFixedString('00', 2), toNullable(2)), toFixedString(toFixedString(toFixedString(' key="v" ', 9), 9), 9)), concat(' key="v" ') -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql index 53a5cfe9b1a..b22ea42b686 100644 --- a/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql +++ b/tests/queries/0_stateless/03017_analyzer_groupby_fuzz_61600.sql @@ -11,7 +11,7 @@ FROM set_index_not__fuzz_0 GROUP BY toNullable(3), concat(concat(NULLIF(1, 1), toNullable(toNullable(3)))) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; -- WITH ROLLUP (note that result is different with the analyzer (analyzer is correct including all combinations) SELECT @@ -22,4 +22,4 @@ GROUP BY toNullable(3), concat(concat(NULLIF(1, 1), toNullable(toNullable(3)))) WITH ROLLUP -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03023_analyzer_optimize_group_by_function_keys_with_nulls.sql b/tests/queries/0_stateless/03023_analyzer_optimize_group_by_function_keys_with_nulls.sql index f0b60caca36..0d0a81c9105 100644 --- a/tests/queries/0_stateless/03023_analyzer_optimize_group_by_function_keys_with_nulls.sql +++ b/tests/queries/0_stateless/03023_analyzer_optimize_group_by_function_keys_with_nulls.sql @@ -1,5 +1,4 @@ -set allow_experimental_analyzer=1; +set enable_analyzer=1; set group_by_use_nulls=1; set optimize_group_by_function_keys=1; SELECT ignore(toLowCardinality(number)) FROM numbers(10) GROUP BY GROUPING SETS ((ignore(toLowCardinality(number)), toLowCardinality(number))); - diff --git a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql index b8c173520a9..d3d6ecaadaf 100644 --- a/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql +++ b/tests/queries/0_stateless/03023_group_by_use_nulls_analyzer_crashes.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = 1, group_by_use_nulls = 1; +set enable_analyzer = 1, group_by_use_nulls = 1; SELECT tuple(tuple(number)) as x FROM numbers(10) GROUP BY (number, tuple(number)) with cube order by x; @@ -56,4 +56,4 @@ SELECT arraySplit(number -> toUInt8(number), []) from numbers(1) GROUP BY toUInt SELECT count(arraySplit(number -> toUInt8(number), [arraySplit(x -> toUInt8(number), [])])) FROM numbers(10) GROUP BY number, [number] WITH ROLLUP settings group_by_use_nulls=1; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -SELECT count(arraySplit(x -> toUInt8(number), [])) FROM numbers(10) GROUP BY number, [number] WITH ROLLUP settings group_by_use_nulls=1; \ No newline at end of file +SELECT count(arraySplit(x -> toUInt8(number), [])) FROM numbers(10) GROUP BY number, [number] WITH ROLLUP settings group_by_use_nulls=1; diff --git a/tests/queries/0_stateless/03023_remove_unused_column_distinct.sql b/tests/queries/0_stateless/03023_remove_unused_column_distinct.sql index c2f32bfe3c1..af8756f1fbc 100644 --- a/tests/queries/0_stateless/03023_remove_unused_column_distinct.sql +++ b/tests/queries/0_stateless/03023_remove_unused_column_distinct.sql @@ -12,4 +12,4 @@ FROM FROM numbers(10) ) ) -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03031_filter_float64_logical_error.sql b/tests/queries/0_stateless/03031_filter_float64_logical_error.sql index 59d4a06c8f7..df35cc7b14a 100644 --- a/tests/queries/0_stateless/03031_filter_float64_logical_error.sql +++ b/tests/queries/0_stateless/03031_filter_float64_logical_error.sql @@ -26,7 +26,7 @@ PREWHERE (id = NULL) AND 1024 WHERE 0.0001 GROUP BY '0.03' WITH ROLLUP -SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx', allow_experimental_analyzer=0; +SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx', enable_analyzer=0; SELECT @@ -37,4 +37,4 @@ PREWHERE (id = NULL) AND 1024 WHERE 0.0001 GROUP BY '0.03' WITH ROLLUP -SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx', allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SETTINGS force_primary_key = 1, force_data_skipping_indices = 'value_1_idx, value_2_idx', enable_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } diff --git a/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql index 97a19cda7d3..42bd5004933 100644 --- a/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql +++ b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test; -SET allow_suspicious_low_cardinality_types = true, allow_experimental_analyzer = true; +SET allow_suspicious_low_cardinality_types = true, enable_analyzer = true; CREATE TABLE test (`id` LowCardinality(UInt32)) ENGINE = MergeTree ORDER BY id AS SELECT 0; diff --git a/tests/queries/0_stateless/03032_redundant_equals.sql b/tests/queries/0_stateless/03032_redundant_equals.sql index de85ec5cf00..63073dbcefd 100644 --- a/tests/queries/0_stateless/03032_redundant_equals.sql +++ b/tests/queries/0_stateless/03032_redundant_equals.sql @@ -9,7 +9,7 @@ ORDER BY k; INSERT INTO test_table SELECT number FROM numbers(10000000); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM test_table WHERE k in (100) = 1; SELECT * FROM test_table WHERE k = (100) = 1; @@ -25,7 +25,7 @@ SELECT * FROM test_table WHERE (NOT ((k not in (100) = 0) OR (k in (100) = 1))) SELECT * FROM test_table WHERE (NOT ((k in (101) = 0) OR (k in (100) = 1))) = 1; SELECT * FROM test_table WHERE ((k not in (101) = 0) OR (k in (100) = 1)) = 1; SELECT * FROM test_table WHERE ((k not in (99) = 1) AND (k in (100) = 1)) = 1; --- we skip optimizing queries with toNullable(0 or 1) but lets make sure they still work +-- we skip optimizing queries with toNullable(0 or 1) but lets make sure they still work SELECT * FROM test_table WHERE (k = 101) = toLowCardinality(toNullable(1)); SELECT * FROM test_table WHERE (k = 101) = toNullable(1); SELECT * FROM test_table WHERE (k = 101) = toLowCardinality(1); diff --git a/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql index 9be1152bbbf..d01e458a544 100644 --- a/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql +++ b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql @@ -3,6 +3,5 @@ drop table if exists test; create table test (`x` LowCardinality(Nullable(UInt32)), `y` String) engine = MergeTree order by tuple(); insert into test values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); create table m_table (x UInt32, y String) engine = Merge(currentDatabase(), 'test*'); -select toTypeName(x), x FROM m_table SETTINGS additional_table_filters = {'m_table':'x != 4'}, optimize_move_to_prewhere=1, allow_experimental_analyzer=1; +select toTypeName(x), x FROM m_table SETTINGS additional_table_filters = {'m_table':'x != 4'}, optimize_move_to_prewhere=1, enable_analyzer=1; drop table test; - diff --git a/tests/queries/0_stateless/03033_analyzer_query_parameters.sh b/tests/queries/0_stateless/03033_analyzer_query_parameters.sh index cf46067df99..67ddf2449eb 100755 --- a/tests/queries/0_stateless/03033_analyzer_query_parameters.sh +++ b/tests/queries/0_stateless/03033_analyzer_query_parameters.sh @@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 1" -${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 0" +${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS enable_analyzer = 1" +${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS enable_analyzer = 0" diff --git a/tests/queries/0_stateless/03033_cte_numbers_memory.sql b/tests/queries/0_stateless/03033_cte_numbers_memory.sql index 66b11cbfaa5..b362f42f89a 100644 --- a/tests/queries/0_stateless/03033_cte_numbers_memory.sql +++ b/tests/queries/0_stateless/03033_cte_numbers_memory.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61238 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH (SELECT number FROM system.numbers LIMIT 1) as w1, diff --git a/tests/queries/0_stateless/03033_recursive_cte_basic.reference b/tests/queries/0_stateless/03033_recursive_cte_basic.reference index a890fccef2b..dc6a9cd8f89 100644 --- a/tests/queries/0_stateless/03033_recursive_cte_basic.reference +++ b/tests/queries/0_stateless/03033_recursive_cte_basic.reference @@ -1,6 +1,6 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte WHERE n < 10) SELECT n FROM recursive_cte; 1 diff --git a/tests/queries/0_stateless/03033_recursive_cte_basic.sql b/tests/queries/0_stateless/03033_recursive_cte_basic.sql index f85e1ffe0bd..63014e9ccd4 100644 --- a/tests/queries/0_stateless/03033_recursive_cte_basic.sql +++ b/tests/queries/0_stateless/03033_recursive_cte_basic.sql @@ -1,6 +1,6 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH RECURSIVE recursive_cte AS (SELECT 1 AS n UNION ALL SELECT n + 1 FROM recursive_cte WHERE n < 10) SELECT n FROM recursive_cte; diff --git a/tests/queries/0_stateless/03033_with_fill_interpolate.sql b/tests/queries/0_stateless/03033_with_fill_interpolate.sql index 0ec0050a922..48457341e0d 100644 --- a/tests/queries/0_stateless/03033_with_fill_interpolate.sql +++ b/tests/queries/0_stateless/03033_with_fill_interpolate.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55794 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS 03033_example_table; CREATE TABLE 03033_example_table diff --git a/tests/queries/0_stateless/03034_normalized_ast.sql b/tests/queries/0_stateless/03034_normalized_ast.sql index 385af4e2c34..8b518d6d11b 100644 --- a/tests/queries/0_stateless/03034_normalized_ast.sql +++ b/tests/queries/0_stateless/03034_normalized_ast.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/49472 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT concat(database, table) AS name, count() diff --git a/tests/queries/0_stateless/03034_recursive_cte_tree.sql b/tests/queries/0_stateless/03034_recursive_cte_tree.sql index a3ff43e435a..fa62298939a 100644 --- a/tests/queries/0_stateless/03034_recursive_cte_tree.sql +++ b/tests/queries/0_stateless/03034_recursive_cte_tree.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tree; CREATE TABLE tree diff --git a/tests/queries/0_stateless/03034_recursive_cte_tree_fuzz_crash_fix.sql b/tests/queries/0_stateless/03034_recursive_cte_tree_fuzz_crash_fix.sql index ae04fa8f377..1e26c53769b 100644 --- a/tests/queries/0_stateless/03034_recursive_cte_tree_fuzz_crash_fix.sql +++ b/tests/queries/0_stateless/03034_recursive_cte_tree_fuzz_crash_fix.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET enable_global_with_statement=1; SET session_timezone = 'Etc/UTC'; diff --git a/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.reference b/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.reference index 85f8c3267c6..f5b950db867 100644 --- a/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.reference +++ b/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.reference @@ -1,6 +1,6 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tree; CREATE TABLE tree ( diff --git a/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.sql b/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.sql index 15acbaf6da9..231aae296e6 100644 --- a/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.sql +++ b/tests/queries/0_stateless/03034_recursive_cte_tree_merge_tree.sql @@ -1,6 +1,6 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tree; CREATE TABLE tree diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql index 74463743b01..8f60808d700 100644 --- a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql +++ b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44414 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS alias_bug; DROP TABLE IF EXISTS alias_bug_dist; CREATE TABLE alias_bug diff --git a/tests/queries/0_stateless/03035_internal_functions_direct_call.sql b/tests/queries/0_stateless/03035_internal_functions_direct_call.sql index 951e0733dbb..e358e498343 100644 --- a/tests/queries/0_stateless/03035_internal_functions_direct_call.sql +++ b/tests/queries/0_stateless/03035_internal_functions_direct_call.sql @@ -4,7 +4,7 @@ SELECT __actionName(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT __actionName('aaa', 'aaa', 'aaa'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT __actionName('aaa', '') SETTINGS allow_experimental_analyzer = 1; -- { serverError BAD_ARGUMENTS } +SELECT __actionName('aaa', '') SETTINGS enable_analyzer = 1; -- { serverError BAD_ARGUMENTS } SELECT __actionName('aaa', materialize('aaa')); -- { serverError BAD_ARGUMENTS,ILLEGAL_COLUMN } SELECT __actionName(materialize('aaa'), 'aaa'); -- { serverError ILLEGAL_COLUMN } SELECT __actionName('aaa', 'aaa'); diff --git a/tests/queries/0_stateless/03035_recursive_cte_postgres_1.reference b/tests/queries/0_stateless/03035_recursive_cte_postgres_1.reference index e6cbdc971eb..568421f3fba 100644 --- a/tests/queries/0_stateless/03035_recursive_cte_postgres_1.reference +++ b/tests/queries/0_stateless/03035_recursive_cte_postgres_1.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- WITH RECURSIVE -- sum of 1..100 diff --git a/tests/queries/0_stateless/03035_recursive_cte_postgres_1.sql b/tests/queries/0_stateless/03035_recursive_cte_postgres_1.sql index 8026dadc331..9a4e313ce90 100644 --- a/tests/queries/0_stateless/03035_recursive_cte_postgres_1.sql +++ b/tests/queries/0_stateless/03035_recursive_cte_postgres_1.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- WITH RECURSIVE diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index d94a68aa4d8..e1a13d1ce71 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET optimize_move_to_prewhere = 0; SET query_plan_convert_outer_join_to_inner_join = 0; diff --git a/tests/queries/0_stateless/03036_recursive_cte_postgres_2.reference b/tests/queries/0_stateless/03036_recursive_cte_postgres_2.reference index adc06d2edd2..a10e1da6907 100644 --- a/tests/queries/0_stateless/03036_recursive_cte_postgres_2.reference +++ b/tests/queries/0_stateless/03036_recursive_cte_postgres_2.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- Some examples with a tree -- diff --git a/tests/queries/0_stateless/03036_recursive_cte_postgres_2.sql b/tests/queries/0_stateless/03036_recursive_cte_postgres_2.sql index f06b1c62426..b8e850b4721 100644 --- a/tests/queries/0_stateless/03036_recursive_cte_postgres_2.sql +++ b/tests/queries/0_stateless/03036_recursive_cte_postgres_2.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- Some examples with a tree diff --git a/tests/queries/0_stateless/03036_with_numbers.sql b/tests/queries/0_stateless/03036_with_numbers.sql index 3463ce826e2..bd0f6b6179c 100644 --- a/tests/queries/0_stateless/03036_with_numbers.sql +++ b/tests/queries/0_stateless/03036_with_numbers.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/13843 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH 10 AS n SELECT * FROM numbers(n); diff --git a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference index 3f8f47ac23e..f0ff70a7c60 100644 --- a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference +++ b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- different tree example -- diff --git a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql index 3c74ee889c4..213e8bc995d 100644 --- a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql +++ b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- different tree example diff --git a/tests/queries/0_stateless/03037_union_view.sql b/tests/queries/0_stateless/03037_union_view.sql index 3ea81b829ba..d963444fd91 100644 --- a/tests/queries/0_stateless/03037_union_view.sql +++ b/tests/queries/0_stateless/03037_union_view.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55803 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS broken_table; DROP TABLE IF EXISTS broken_view; @@ -28,4 +28,4 @@ FROM broken_view v FINAL WHERE v.start IS NOT NULL; DROP TABLE IF EXISTS broken_table; -DROP TABLE IF EXISTS broken_view; \ No newline at end of file +DROP TABLE IF EXISTS broken_view; diff --git a/tests/queries/0_stateless/03038_ambiguous_column.sql b/tests/queries/0_stateless/03038_ambiguous_column.sql index 9df3cd9bc9b..131bc552f56 100644 --- a/tests/queries/0_stateless/03038_ambiguous_column.sql +++ b/tests/queries/0_stateless/03038_ambiguous_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/48308 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS 03038_table; CREATE TABLE 03038_table diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index b920fc298b3..cf070eebc38 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- test cycle detection -- diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index a6fe3b1e55c..7dad74893b9 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- test cycle detection diff --git a/tests/queries/0_stateless/03039_recursive_cte_postgres_5.reference b/tests/queries/0_stateless/03039_recursive_cte_postgres_5.reference index 2910ac174e7..43399a0ab95 100644 --- a/tests/queries/0_stateless/03039_recursive_cte_postgres_5.reference +++ b/tests/queries/0_stateless/03039_recursive_cte_postgres_5.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- test multiple WITH queries -- diff --git a/tests/queries/0_stateless/03039_recursive_cte_postgres_5.sql b/tests/queries/0_stateless/03039_recursive_cte_postgres_5.sql index e188a455434..eb4043cca42 100644 --- a/tests/queries/0_stateless/03039_recursive_cte_postgres_5.sql +++ b/tests/queries/0_stateless/03039_recursive_cte_postgres_5.sql @@ -33,7 +33,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- test multiple WITH queries diff --git a/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql index 640d217d2f9..652085d9f5a 100644 --- a/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql +++ b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/45535 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT *, diff --git a/tests/queries/0_stateless/03040_alias_column_join.sql b/tests/queries/0_stateless/03040_alias_column_join.sql index 54f579c0feb..6ffd749a6c4 100644 --- a/tests/queries/0_stateless/03040_alias_column_join.sql +++ b/tests/queries/0_stateless/03040_alias_column_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44365 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS 03040_test; CREATE TABLE 03040_test diff --git a/tests/queries/0_stateless/03040_array_sum_and_join.sql b/tests/queries/0_stateless/03040_array_sum_and_join.sql index 9aeddc9f765..90d3d83c9a2 100644 --- a/tests/queries/0_stateless/03040_array_sum_and_join.sql +++ b/tests/queries/0_stateless/03040_array_sum_and_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select t.1 as cnt, t.2 as name, diff --git a/tests/queries/0_stateless/03040_recursive_cte_postgres_6.reference b/tests/queries/0_stateless/03040_recursive_cte_postgres_6.reference index 2d9d7bb9f6c..7ccc3df9f0d 100644 --- a/tests/queries/0_stateless/03040_recursive_cte_postgres_6.reference +++ b/tests/queries/0_stateless/03040_recursive_cte_postgres_6.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- error cases -- diff --git a/tests/queries/0_stateless/03040_recursive_cte_postgres_6.sql b/tests/queries/0_stateless/03040_recursive_cte_postgres_6.sql index bff2ece6ece..6a6044198c8 100644 --- a/tests/queries/0_stateless/03040_recursive_cte_postgres_6.sql +++ b/tests/queries/0_stateless/03040_recursive_cte_postgres_6.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- -- error cases diff --git a/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql index 7906e65f8b8..88f7fc562b1 100644 --- a/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql +++ b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE IF NOT EXISTS first engine = MergeTree PARTITION BY (inn, toYYYYMM(received)) ORDER BY (inn, sessionId) AS SELECT now() AS received, '123456789' AS inn, '42' AS sessionId; diff --git a/tests/queries/0_stateless/03041_recursive_cte_postgres_7.reference b/tests/queries/0_stateless/03041_recursive_cte_postgres_7.reference index 6bcc3f89dcb..b98b2ec7e89 100644 --- a/tests/queries/0_stateless/03041_recursive_cte_postgres_7.reference +++ b/tests/queries/0_stateless/03041_recursive_cte_postgres_7.reference @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH RECURSIVE foo AS (SELECT 1 AS i UNION ALL diff --git a/tests/queries/0_stateless/03041_recursive_cte_postgres_7.sql b/tests/queries/0_stateless/03041_recursive_cte_postgres_7.sql index 71c34b7361f..5f4455efcc6 100644 --- a/tests/queries/0_stateless/03041_recursive_cte_postgres_7.sql +++ b/tests/queries/0_stateless/03041_recursive_cte_postgres_7.sql @@ -32,7 +32,7 @@ -- { echoOn } -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; WITH RECURSIVE foo AS (SELECT 1 AS i diff --git a/tests/queries/0_stateless/03041_select_with_query_result.sql b/tests/queries/0_stateless/03041_select_with_query_result.sql index 061223b43e1..e5897ea12cf 100644 --- a/tests/queries/0_stateless/03041_select_with_query_result.sql +++ b/tests/queries/0_stateless/03041_select_with_query_result.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44153 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS parent; DROP TABLE IF EXISTS join_table_1; DROP TABLE IF EXISTS join_table_2; diff --git a/tests/queries/0_stateless/03042_analyzer_alias_join.sql b/tests/queries/0_stateless/03042_analyzer_alias_join.sql index dac3b6a4983..d9a8d8b4c7b 100644 --- a/tests/queries/0_stateless/03042_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03042_analyzer_alias_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/14978 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test1(id UInt64, t1value UInt64) ENGINE=MergeTree ORDER BY tuple(); CREATE TABLE test2(id UInt64, t2value String) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/03042_not_found_column_c1.sql b/tests/queries/0_stateless/03042_not_found_column_c1.sql index b4dce2af489..08202dc0dca 100644 --- a/tests/queries/0_stateless/03042_not_found_column_c1.sql +++ b/tests/queries/0_stateless/03042_not_found_column_c1.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/42399 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE IF NOT EXISTS t0 (c0 Int32) ENGINE = Memory() ; CREATE TABLE t1 (c0 Int32, c1 Int32, c2 Int32) ENGINE = Memory() ; diff --git a/tests/queries/0_stateless/03043_group_array_result_is_expected.sql b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql index 5311927ae3c..e2c79e5c41e 100644 --- a/tests/queries/0_stateless/03043_group_array_result_is_expected.sql +++ b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/27115 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; drop table if exists fill_ex; create table fill_ex ( diff --git a/tests/queries/0_stateless/03044_analyzer_alias_join.sql b/tests/queries/0_stateless/03044_analyzer_alias_join.sql index 3ab8edb005f..7636edbb411 100644 --- a/tests/queries/0_stateless/03044_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03044_analyzer_alias_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/17319 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TEMPORARY TABLE hits (date Date, data Float64) engine=Memory(); SELECT diff --git a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql index 0cf05763202..4885b7e3f81 100644 --- a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql +++ b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql @@ -1,3 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11813 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select 1 from (select 1 x) l join (select 1 y, [1] a) r on l.x = r.y array join r.a; diff --git a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql index ee8756b9460..cbc46726467 100644 --- a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql +++ b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/13210 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test_a_table ( name String, a_col String diff --git a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql index d97dfc880b3..967b7b24787 100644 --- a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql +++ b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23053 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS repl_tbl; CREATE TEMPORARY TABLE repl_tbl diff --git a/tests/queries/0_stateless/03046_column_in_block_array_join.sql b/tests/queries/0_stateless/03046_column_in_block_array_join.sql index c6b4613af3f..f91a18da831 100644 --- a/tests/queries/0_stateless/03046_column_in_block_array_join.sql +++ b/tests/queries/0_stateless/03046_column_in_block_array_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/37729 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS nested_test; DROP TABLE IF EXISTS join_test; diff --git a/tests/queries/0_stateless/03047_analyzer_alias_join.sql b/tests/queries/0_stateless/03047_analyzer_alias_join.sql index 7d44c92b6f1..29fc711aaf4 100644 --- a/tests/queries/0_stateless/03047_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03047_analyzer_alias_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT 1 AS value, * diff --git a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql index cfaf1df44bd..d12e85ca03d 100644 --- a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql +++ b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql @@ -1,4 +1,4 @@ -- https://github.com/ClickHouse/ClickHouse/issues/32639 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT 0 AND id ? 1 : 2 AS a, sum(id) FROM (SELECT 1 AS id) GROUP BY a; diff --git a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql index 42fd581e142..f511ea81e26 100644 --- a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql +++ b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/41964 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS ab_12_aaa; DROP TABLE IF EXISTS ab_12_bbb; diff --git a/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql index d25babe6788..712d1c27e6e 100644 --- a/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql +++ b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/7520 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test (`a` UInt32, `b` UInt32) ENGINE = Memory; INSERT INTO test VALUES (1,2), (1,3), (2,4); diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql index 938f270b9e4..0efe59a1f1c 100644 --- a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql +++ b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54317 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03050_select_one_one_one.sql b/tests/queries/0_stateless/03050_select_one_one_one.sql index 28a55e0c471..09f3f20c35d 100644 --- a/tests/queries/0_stateless/03050_select_one_one_one.sql +++ b/tests/queries/0_stateless/03050_select_one_one_one.sql @@ -1,4 +1,4 @@ -- https://github.com/ClickHouse/ClickHouse/issues/36973 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT 1, 1, 1; SELECT * FROM (SELECT 1, 1, 1); diff --git a/tests/queries/0_stateless/03051_many_ctes.sql b/tests/queries/0_stateless/03051_many_ctes.sql index d4e613bd279..e442813b6a2 100644 --- a/tests/queries/0_stateless/03051_many_ctes.sql +++ b/tests/queries/0_stateless/03051_many_ctes.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/40955 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH toInt64(2) AS new_x SELECT new_x AS x FROM (SELECT 1 AS x) t; WITH toInt64(2) AS new_x SELECT * replace(new_x as x) FROM (SELECT 1 AS x) t; SELECT 2 AS x FROM (SELECT 1 AS x) t; diff --git a/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql index 24e9ab0f36e..55993175bf3 100644 --- a/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql +++ b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/pull/40065 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT ( diff --git a/tests/queries/0_stateless/03053_analyzer_join_alias.sql b/tests/queries/0_stateless/03053_analyzer_join_alias.sql index 894b8af7c6f..677cf9d4d5e 100644 --- a/tests/queries/0_stateless/03053_analyzer_join_alias.sql +++ b/tests/queries/0_stateless/03053_analyzer_join_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23104 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03054_analyzer_join_alias.sql b/tests/queries/0_stateless/03054_analyzer_join_alias.sql index e124aa33a9b..f018f57cc6f 100644 --- a/tests/queries/0_stateless/03054_analyzer_join_alias.sql +++ b/tests/queries/0_stateless/03054_analyzer_join_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/21584 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT count() FROM ( diff --git a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql index 25b6dcb3564..29ba1dd7c5b 100644 --- a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql +++ b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23344 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT logTrace(repeat('Hello', 100)), ignore(*) FROM ( SELECT ignore((SELECT groupArrayState(([number], [number])) FROM numbers(19000))) diff --git a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql index 2217af327fa..92f603ed595 100644 --- a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql +++ b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/10276 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT sum(x.n) as n, sum(z.n) as n2 diff --git a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql index 3cce77f0240..ef3c0e5f63d 100644 --- a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql +++ b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/4567 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS fact; DROP TABLE IF EXISTS animals; DROP TABLE IF EXISTS colors; @@ -23,4 +23,3 @@ select id, animal_name, animal_key, color_name, color_key from fact a left join (select toInt64(animal_key) animal_key, animal_name from animals) b on (a.animal_key = b.animal_key) left join (select toInt64(color_key) color_key, color_name from colors) c on (a.color_key = c.color_key); -- { serverError AMBIGUOUS_IDENTIFIER } - diff --git a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql index 27782462075..164a42e5ba3 100644 --- a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql +++ b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/17710 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE id_val(id UInt32, val UInt32) ENGINE = Memory; CREATE TABLE id_val_join0(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 0; CREATE TABLE id_val_join1(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1; diff --git a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql index f8cd8690ee5..0556683b97a 100644 --- a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql +++ b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11068 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table vt(datetime_value DateTime, value Float64) Engine=Memory; create view computed_datum_hours as diff --git a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql index 6fee6d1f73d..a1b50967a46 100644 --- a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql +++ b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/24395 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE xxxx_yyy (key UInt32, key_b ALIAS key) ENGINE=MergeTree() ORDER BY key; INSERT INTO xxxx_yyy SELECT number FROM numbers(10); diff --git a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql index 9748175e4d4..487d74b3317 100644 --- a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql +++ b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23416 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table test (TOPIC String, PARTITION UInt64, OFFSET UInt64, ID UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03062', 'r2') ORDER BY (TOPIC, PARTITION, OFFSET); create table test_join (TOPIC String, PARTITION UInt64, OFFSET UInt64) ENGINE = Join(ANY, LEFT, `TOPIC`, `PARTITION`) SETTINGS join_any_take_last_row = 1; diff --git a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql index 7eab1fa846a..5655d4a0110 100644 --- a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql +++ b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23162 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE t1 ( k Int64, x Int64) ENGINE = Memory; CREATE TABLE t2( x Int64 ) ENGINE = Memory; @@ -14,4 +14,3 @@ WHERE (t1.d >= now()); -- { serverError UNKNOWN_IDENTIFIER } SELECT * FROM t1 INNER JOIN s ON t1.k = s.k WHERE (t1.d >= now()); -- { serverError UNKNOWN_IDENTIFIER } - diff --git a/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql index 59ebb9d9af3..d5696411091 100644 --- a/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql +++ b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/25655 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT sum(t.b) / 1 a, sum(t.a) diff --git a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql index 7e6befe181e..5034e2eed8f 100644 --- a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql +++ b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql @@ -1,3 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11757 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select * from (select [1, 2] a) aa cross join (select [3, 4] b) bb array join aa.a, bb.b; diff --git a/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql index 8983be242c3..2b879ed73da 100644 --- a/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql +++ b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH 0 AS test SELECT * FROM diff --git a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql index 052a9eaf734..58845b93771 100644 --- a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql +++ b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; with d as (select 'key'::Varchar(255) c, 'x'::Varchar(255) s) SELECT r1, c as r2 FROM ( diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql index 542380feb7c..459c8f5e8ac 100644 --- a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -1,7 +1,7 @@ -- Tags: no-replicated-database -- Closes: https://github.com/ClickHouse/ClickHouse/issues/6571 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE LINEITEM_shard ON CLUSTER test_shard_localhost ( L_ORDERKEY UInt64, diff --git a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql index 09d2985fe60..5ec04cbc0f5 100644 --- a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql +++ b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/4432 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH [1, 2] AS zz SELECT x FROM system.one diff --git a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql index 7aadab2ca73..a94ae811476 100644 --- a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql +++ b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/8259 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; with (select 25) as something select *, something diff --git a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql index e2eb758d649..211fa2a3119 100644 --- a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql +++ b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/9233 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT * FROM ( diff --git a/tests/queries/0_stateless/03071_fix_short_circuit_logic.sql b/tests/queries/0_stateless/03071_fix_short_circuit_logic.sql index 7745bceca0b..dc5fb5085fb 100644 --- a/tests/queries/0_stateless/03071_fix_short_circuit_logic.sql +++ b/tests/queries/0_stateless/03071_fix_short_circuit_logic.sql @@ -59,4 +59,4 @@ FROM ) WHERE (address = hex2bytes('0xd387a6e4e84a6c86bd90c158c6028a58cc8ac459')) AND (transfer_id NOT LIKE 'gas%') AND (value > 0) AND (dictGetOrDefault(token_data_map, 'is_blacklisted', (token_address_hex, 'zksync'), true)) ) -SETTINGS max_threads = 1, short_circuit_function_evaluation = 'enable', allow_experimental_analyzer = 0; \ No newline at end of file +SETTINGS max_threads = 1, short_circuit_function_evaluation = 'enable', enable_analyzer = 0; diff --git a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql index e2846033913..ec3b067cbdf 100644 --- a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql +++ b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql @@ -1,3 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/14699 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select * from (select number from numbers(1)) where not ignore(*); diff --git a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql index 5599324c62b..bba51e28ba3 100644 --- a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql +++ b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/27068 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test ( id String, create_time DateTime ) ENGINE = MergeTree ORDER BY id; insert into test values(1,'1970-02-01 00:00:00'); diff --git a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql index 4df5f6f48e6..314b6c0e8d6 100644 --- a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql +++ b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/28687 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create view alias (dummy int, n alias dummy) as select * from system.one; select n from alias; diff --git a/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql index 416815e761b..4f097350da6 100644 --- a/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql +++ b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/28777 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT sum(q0.a2) AS a1, sum(q0.a1) AS a9 diff --git a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql index 7ac9fe6b446..894e3bc56ee 100644 --- a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql +++ b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29734 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT * FROM ( @@ -49,4 +49,3 @@ INNER JOIN SELECT number AS x FROM numbers(10) ) AS c ON a.x = c.x; -- { serverError UNKNOWN_IDENTIFIER } - diff --git a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql index 5a181023c57..d4335d35e51 100644 --- a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql +++ b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/33825 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE t1 (i Int64, j Int64) ENGINE = Memory; INSERT INTO t1 SELECT number, number FROM system.numbers LIMIT 10; SELECT diff --git a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql index d91a9ed106d..b9b850619ea 100644 --- a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql +++ b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/33825 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE t2 (first_column Int64, second_column Int64) ENGINE = Memory; INSERT INTO t2 SELECT number, number FROM system.numbers LIMIT 10; diff --git a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql index 955d3b49a00..80e681c0776 100644 --- a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql +++ b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE testdata (`1` String) ENGINE=MergeTree ORDER BY tuple(); INSERT INTO testdata VALUES ('testdata'); diff --git a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql index 01ab868f9ea..2138828cd27 100644 --- a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql +++ b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/35652 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test ( id UInt64 ) diff --git a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql index 7682e6ce866..a34c71a44e2 100644 --- a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql +++ b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29838 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET distributed_foreground_insert=1; CREATE TABLE first_table_lr diff --git a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql index e6a540dc5df..3cb02512a7f 100644 --- a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql +++ b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/36189 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test ( `dt` Date, diff --git a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql index 8f17248ed0d..3b83f978326 100644 --- a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql +++ b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/39634 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE test1 ( `pk` String, diff --git a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql index 930726898b5..8a7258f5838 100644 --- a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql +++ b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/47432 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table t1 engine = MergeTree() order by tuple() diff --git a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql index fd67194b08b..c360e86197f 100644 --- a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql +++ b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54910 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT toTypeName(stat_standard_id) AS stat_standard_id_1, count(1) AS value FROM ( SELECT 'string value' AS stat_standard_id ) GROUP BY stat_standard_id_1 LIMIT 1 diff --git a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql index 31747328d1f..7e44b37f865 100644 --- a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql +++ b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/57321 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT ver, max(ver) OVER () AS ver_max diff --git a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql index 6546e50c99e..a00ca4960d7 100644 --- a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql +++ b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/59154 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT * FROM ( diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql index e6f1ed81f91..3670404d124 100644 --- a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql +++ b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61014 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; create database {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql index 069da5fdd65..5526e1aaf7d 100644 --- a/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql +++ b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61950 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; with dummy + 1 as dummy select dummy from system.one; diff --git a/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql index c35f33782ff..08ea103d3c9 100644 --- a/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql +++ b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55647 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT * diff --git a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql index 599275c66e8..11984aec496 100644 --- a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql +++ b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61947 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql index 10d18324c3c..83b1a902721 100644 --- a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql +++ b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61947 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/03093_analyzer_column_alias.sql b/tests/queries/0_stateless/03093_analyzer_column_alias.sql index 9ff0f78ba24..edf89108b56 100644 --- a/tests/queries/0_stateless/03093_analyzer_column_alias.sql +++ b/tests/queries/0_stateless/03093_analyzer_column_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/26674 -SET allow_experimental_analyzer = true; +SET enable_analyzer = true; SELECT Carrier, diff --git a/tests/queries/0_stateless/03093_analyzer_miel_test.sql b/tests/queries/0_stateless/03093_analyzer_miel_test.sql index f408882dcd5..4915864bb51 100644 --- a/tests/queries/0_stateless/03093_analyzer_miel_test.sql +++ b/tests/queries/0_stateless/03093_analyzer_miel_test.sql @@ -8,7 +8,7 @@ INSERT INTO test_03093 VALUES ('x1', 123, {'k1': ''}); INSERT INTO test_03093 VALUES ('x1', 123, {'k1': '', 'k11': ''}); INSERT INTO test_03093 VALUES ('x1', 12, {'k1': ''}); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select app, arrayZip(untuple(sumMap(k.keys, replicate(1, k.keys)))) from test_03093 PREWHERE c > 1 group by app; select app, arrayZip(untuple(sumMap(k.keys, replicate(1, k.keys)))) from test_03093 WHERE c > 1 group by app; diff --git a/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql b/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql index 62fa3f437af..90f0a8a3b8e 100644 --- a/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql +++ b/tests/queries/0_stateless/03093_bug37909_query_does_not_finish.sql @@ -75,4 +75,4 @@ FROM /* WHERE (v_date >= '2022-05-08') AND (v_date <= '2022-06-07') placing condition has same effect */ GROUP BY vDate ORDER BY vDate ASC -SETTINGS allow_experimental_analyzer = 1; -- the query times out if allow_experimental_analyzer = 0 +SETTINGS enable_analyzer = 1; -- the query times out if enable_analyzer = 0 diff --git a/tests/queries/0_stateless/03094_analyzer_fiddle_multiif.sql b/tests/queries/0_stateless/03094_analyzer_fiddle_multiif.sql index 1b1603be18e..8426749917a 100644 --- a/tests/queries/0_stateless/03094_analyzer_fiddle_multiif.sql +++ b/tests/queries/0_stateless/03094_analyzer_fiddle_multiif.sql @@ -5,7 +5,7 @@ INSERT INTO users_03094 VALUES ('John', 33); INSERT INTO users_03094 VALUES ('Ksenia', 48); INSERT INTO users_03094 VALUES ('Alice', 50); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT multiIf((age > 30) or (true), '1', '2') AS a, diff --git a/tests/queries/0_stateless/03094_named_tuple_bug24607.sql b/tests/queries/0_stateless/03094_named_tuple_bug24607.sql index e3c97f3fe41..698c339e53e 100644 --- a/tests/queries/0_stateless/03094_named_tuple_bug24607.sql +++ b/tests/queries/0_stateless/03094_named_tuple_bug24607.sql @@ -1,4 +1,4 @@ SELECT JSONExtract('{"a":1, "b":"test"}', 'Tuple(a UInt8, b String)') AS x, x.a -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 1f6bd99df7f..6ae4e4d4d3c 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -2,7 +2,7 @@ -- (no-tsan because it has a small maximum stack size and the test would fail with TOO_DEEP_RECURSION) SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower -SET allow_experimental_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES +SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.sql b/tests/queries/0_stateless/03095_window_functions_qualify.sql index adedff2e2cf..72903992745 100644 --- a/tests/queries/0_stateless/03095_window_functions_qualify.sql +++ b/tests/queries/0_stateless/03095_window_functions_qualify.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT number, COUNT() OVER (PARTITION BY number % 3) AS partition_count FROM numbers(10) QUALIFY partition_count = 4 ORDER BY number; diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index b1ddd141e04..0e60ee77fe5 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -1,4 +1,4 @@ -set allow_experimental_analyzer = true; +set enable_analyzer = true; select count; -- { serverError UNKNOWN_IDENTIFIER } diff --git a/tests/queries/0_stateless/03097_query_log_join_processes.sql b/tests/queries/0_stateless/03097_query_log_join_processes.sql index daf3136e3fe..135160f4709 100644 --- a/tests/queries/0_stateless/03097_query_log_join_processes.sql +++ b/tests/queries/0_stateless/03097_query_log_join_processes.sql @@ -2,6 +2,6 @@ SYSTEM FLUSH LOGS; -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT count(1) as num, hostName() as hostName FROM system.query_log as a INNER JOIN system.processes as b on a.query_id = b.query_id and type = 'QueryStart' and dateDiff('second', event_time, now()) > 5 and current_database = currentDatabase() FORMAT Null; diff --git a/tests/queries/0_stateless/03098_prefer_column_to_alias_subquery.sql b/tests/queries/0_stateless/03098_prefer_column_to_alias_subquery.sql index cf3768e2f29..cb41151b9c9 100644 --- a/tests/queries/0_stateless/03098_prefer_column_to_alias_subquery.sql +++ b/tests/queries/0_stateless/03098_prefer_column_to_alias_subquery.sql @@ -13,7 +13,7 @@ CREATE TABLE clickhouse_alias_issue_2 ( column_2 Nullable(Float32) ) Engine=Memory; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; INSERT INTO `clickhouse_alias_issue_1` VALUES (1, 100), (2, 200), (3, 300); @@ -28,7 +28,7 @@ VALUES (1, 10), (2, 20), (3, 30); -- \N 30 3 -- \N 20 2 -- \N 10 1 -SELECT * +SELECT * FROM ( SELECT diff --git a/tests/queries/0_stateless/03099_analyzer_multi_join.sql b/tests/queries/0_stateless/03099_analyzer_multi_join.sql index 67985962ba8..dfdaeeea770 100644 --- a/tests/queries/0_stateless/03099_analyzer_multi_join.sql +++ b/tests/queries/0_stateless/03099_analyzer_multi_join.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/56503 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT tb1.owner_id AS owner_id, diff --git a/tests/queries/0_stateless/03100_analyzer_constants_in_multiif.sql b/tests/queries/0_stateless/03100_analyzer_constants_in_multiif.sql index 04666411760..c9a4000d25e 100644 --- a/tests/queries/0_stateless/03100_analyzer_constants_in_multiif.sql +++ b/tests/queries/0_stateless/03100_analyzer_constants_in_multiif.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/59101 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE users (name String, age Int16) ENGINE=Memory; INSERT INTO users VALUES ('John', 33); diff --git a/tests/queries/0_stateless/03101_analyzer_identifiers_1.sql b/tests/queries/0_stateless/03101_analyzer_identifiers_1.sql index 2e0ad7d8a5b..499f712e57a 100644 --- a/tests/queries/0_stateless/03101_analyzer_identifiers_1.sql +++ b/tests/queries/0_stateless/03101_analyzer_identifiers_1.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23194 -- This test add query-templates for fuzzer -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03101_analyzer_identifiers_2.sql b/tests/queries/0_stateless/03101_analyzer_identifiers_2.sql index 35f34e33689..92c3e98265f 100644 --- a/tests/queries/0_stateless/03101_analyzer_identifiers_2.sql +++ b/tests/queries/0_stateless/03101_analyzer_identifiers_2.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23194 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TEMPORARY TABLE test1 (a String, nest Nested(x String, y String)); diff --git a/tests/queries/0_stateless/03101_analyzer_identifiers_3.sql b/tests/queries/0_stateless/03101_analyzer_identifiers_3.sql index 77a0f040e88..997fee91930 100644 --- a/tests/queries/0_stateless/03101_analyzer_identifiers_3.sql +++ b/tests/queries/0_stateless/03101_analyzer_identifiers_3.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel -- Looks like you cannot use the query parameter as a column name. -- https://github.com/ClickHouse/ClickHouse/issues/23194 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP DATABASE IF EXISTS db1_03101; DROP DATABASE IF EXISTS db2_03101; diff --git a/tests/queries/0_stateless/03101_analyzer_identifiers_4.sql b/tests/queries/0_stateless/03101_analyzer_identifiers_4.sql index eba6ad09a3f..869310aa181 100644 --- a/tests/queries/0_stateless/03101_analyzer_identifiers_4.sql +++ b/tests/queries/0_stateless/03101_analyzer_identifiers_4.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23194 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03101_analyzer_invalid_join_on.sql b/tests/queries/0_stateless/03101_analyzer_invalid_join_on.sql index 8539376e136..d7e26862d36 100644 --- a/tests/queries/0_stateless/03101_analyzer_invalid_join_on.sql +++ b/tests/queries/0_stateless/03101_analyzer_invalid_join_on.sql @@ -3,23 +3,23 @@ drop table if exists t1; drop table if exists t2; -set allow_experimental_analyzer=1; +set enable_analyzer=1; create table t1 (c3 String, primary key(c3)) engine = MergeTree; create table t2 (c11 String, primary key(c11)) engine = MergeTree; insert into t1 values ('succeed'); insert into t2 values ('succeed'); -select +select ref_0.c11 as c_2_c30_0 - from + from t2 as ref_0 - cross join (select - ref_1.c3 as c_6_c28_15 - from + cross join (select + ref_1.c3 as c_6_c28_15 + from t1 as ref_1 ) as subq_0 where subq_0.c_6_c28_15 = (select c11 from t2 order by c11 limit 1); drop table if exists t1; -drop table if exists t2; \ No newline at end of file +drop table if exists t2; diff --git a/tests/queries/0_stateless/03102_prefer_column_name_to_alias.sql b/tests/queries/0_stateless/03102_prefer_column_name_to_alias.sql index 48e97fd0841..fafdb660e44 100644 --- a/tests/queries/0_stateless/03102_prefer_column_name_to_alias.sql +++ b/tests/queries/0_stateless/03102_prefer_column_name_to_alias.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS loans; CREATE TABLE loans (loan_number int, security_id text) ENGINE=Memory; -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; INSERT INTO loans VALUES (1, 'AAA'); INSERT INTO loans VALUES (1, 'AAA'); diff --git a/tests/queries/0_stateless/03103_positional_arguments.sql b/tests/queries/0_stateless/03103_positional_arguments.sql index ad30719dc3e..eecaa3f4ea6 100644 --- a/tests/queries/0_stateless/03103_positional_arguments.sql +++ b/tests/queries/0_stateless/03103_positional_arguments.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/56466 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TABLE IF EXISTS users; diff --git a/tests/queries/0_stateless/03104_create_view_join.sql b/tests/queries/0_stateless/03104_create_view_join.sql index bed3d81c9a8..a39be92b1f1 100644 --- a/tests/queries/0_stateless/03104_create_view_join.sql +++ b/tests/queries/0_stateless/03104_create_view_join.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_table_01; DROP TABLE IF EXISTS test_table_02; DROP TABLE IF EXISTS test_view_01; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE test_table_01 ( column Int32 @@ -15,12 +15,12 @@ CREATE TABLE test_table_02 ( ) ENGINE = Memory(); CREATE VIEW test_view_01 AS -SELECT +SELECT t1.column, t2.column -FROM test_table_01 AS t1 +FROM test_table_01 AS t1 INNER JOIN test_table_02 AS t2 ON t1.column = t2.column; DROP TABLE IF EXISTS test_table_01; DROP TABLE IF EXISTS test_table_02; -DROP TABLE IF EXISTS test_view_01; \ No newline at end of file +DROP TABLE IF EXISTS test_view_01; diff --git a/tests/queries/0_stateless/03107_ill_formed_select_in_materialized_view.sql b/tests/queries/0_stateless/03107_ill_formed_select_in_materialized_view.sql index d142bf1662e..5f6ec74bdeb 100644 --- a/tests/queries/0_stateless/03107_ill_formed_select_in_materialized_view.sql +++ b/tests/queries/0_stateless/03107_ill_formed_select_in_materialized_view.sql @@ -6,10 +6,10 @@ DROP TABLE iF EXISTS b; CREATE TABLE a ( a UInt64, b UInt64) ENGINE = Memory; CREATE TABLE b ( b UInt64) ENGINE = Memory; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET joined_subquery_requires_alias = 0; CREATE MATERIALIZED VIEW view_4 ( bb UInt64, cnt UInt64) Engine=MergeTree ORDER BY bb POPULATE AS SELECT bb, count() AS cnt FROM (SELECT a, b AS j, b AS bb FROM a INNER JOIN (SELECT b AS j, b AS bb FROM b ) USING (j)) GROUP BY bb; -- { serverError UNKNOWN_IDENTIFIER } DROP TABLE IF EXISTS a; -DROP TABLE iF EXISTS b; \ No newline at end of file +DROP TABLE iF EXISTS b; diff --git a/tests/queries/0_stateless/03108_describe_union_all.sql b/tests/queries/0_stateless/03108_describe_union_all.sql index c5172902a2b..7e207ae2854 100644 --- a/tests/queries/0_stateless/03108_describe_union_all.sql +++ b/tests/queries/0_stateless/03108_describe_union_all.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/8030 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; -DESCRIBE (SELECT 1, 1 UNION ALL SELECT 1, 2); \ No newline at end of file +DESCRIBE (SELECT 1, 1 UNION ALL SELECT 1, 2); diff --git a/tests/queries/0_stateless/03109_ast_too_big.sql b/tests/queries/0_stateless/03109_ast_too_big.sql index 3eedf305f44..1464f90fe83 100644 --- a/tests/queries/0_stateless/03109_ast_too_big.sql +++ b/tests/queries/0_stateless/03109_ast_too_big.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/32139 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; WITH data AS ( @@ -50,4 +50,4 @@ SELECT value12 AS v12, value13 AS v13, value14 AS v14 -FORMAT Null; \ No newline at end of file +FORMAT Null; diff --git a/tests/queries/0_stateless/03110_unicode_alias.sql b/tests/queries/0_stateless/03110_unicode_alias.sql index b8cbe7390fa..aa33195ea51 100644 --- a/tests/queries/0_stateless/03110_unicode_alias.sql +++ b/tests/queries/0_stateless/03110_unicode_alias.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/47288 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; select 1 as `c0` from ( diff --git a/tests/queries/0_stateless/03111_inner_join_group_by.sql b/tests/queries/0_stateless/03111_inner_join_group_by.sql index fcc8c55a566..6ebaacfc3fe 100644 --- a/tests/queries/0_stateless/03111_inner_join_group_by.sql +++ b/tests/queries/0_stateless/03111_inner_join_group_by.sql @@ -1,6 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/50705 -set allow_experimental_analyzer=1; +set enable_analyzer=1; SELECT count(s0.number), @@ -15,4 +15,4 @@ INNER JOIN LIMIT 10 ) AS s1 ON s0.number = s1.number GROUP BY s0.number > 5 -LIMIT 10 -- {serverError NOT_AN_AGGREGATE} \ No newline at end of file +LIMIT 10 -- {serverError NOT_AN_AGGREGATE} diff --git a/tests/queries/0_stateless/03112_analyzer_not_found_column_in_block.sql b/tests/queries/0_stateless/03112_analyzer_not_found_column_in_block.sql index eb07ff1d837..cc734b2c49a 100644 --- a/tests/queries/0_stateless/03112_analyzer_not_found_column_in_block.sql +++ b/tests/queries/0_stateless/03112_analyzer_not_found_column_in_block.sql @@ -15,7 +15,7 @@ PRIMARY KEY (user_id, timestamp); INSERT INTO my_first_table (user_id, message, timestamp, metric) VALUES (101, 'Hello, ClickHouse!', now(), -1.0 ), (102, 'Insert a lot of rows per batch', yesterday(), 1.41421 ), (102, 'Sort your data based on your commonly-used queries', today(), 2.718 ), (101, 'Granules are the smallest chunks of data read', now() + 5, 3.14159 ); -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT user_id @@ -24,4 +24,4 @@ FROM my_first_table WHERE timestamp > 0 and user_id IN (101) LIMIT 2 BY user_id; -DROP TABLE IF EXISTS my_first_table; \ No newline at end of file +DROP TABLE IF EXISTS my_first_table; diff --git a/tests/queries/0_stateless/03113_analyzer_not_found_column_in_block_2.sql b/tests/queries/0_stateless/03113_analyzer_not_found_column_in_block_2.sql index 7bcc6565ea0..4389bdf83fa 100644 --- a/tests/queries/0_stateless/03113_analyzer_not_found_column_in_block_2.sql +++ b/tests/queries/0_stateless/03113_analyzer_not_found_column_in_block_2.sql @@ -4,11 +4,11 @@ drop table if exists t; create table t (ID String) Engine= Memory() ; insert into t values('a'),('b'),('c'); - + -- This optimization is disabled by default and even its description says that it could lead to -- inconsistencies for distributed queries. set optimize_if_transform_strings_to_enum=0; -set allow_experimental_analyzer=1; +set enable_analyzer=1; SELECT multiIf( ((multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') as y) = 'c') OR (multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') = 'b') OR @@ -18,4 +18,4 @@ SELECT multiIf( ((multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') FROM remote('127.0.0.{1,2}', currentDatabase(), t) GROUP BY alias; -drop table if exists t; \ No newline at end of file +drop table if exists t; diff --git a/tests/queries/0_stateless/03114_analyzer_cte_with_join.sql b/tests/queries/0_stateless/03114_analyzer_cte_with_join.sql index 140197c7104..65dd3cb66b6 100644 --- a/tests/queries/0_stateless/03114_analyzer_cte_with_join.sql +++ b/tests/queries/0_stateless/03114_analyzer_cte_with_join.sql @@ -1,7 +1,7 @@ -- Tags: no-replicated-database -- https://github.com/ClickHouse/ClickHouse/issues/58500 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; drop table if exists t; diff --git a/tests/queries/0_stateless/03115_alias_exists_column.sql b/tests/queries/0_stateless/03115_alias_exists_column.sql index 654cdd71175..65fc0d8f9a9 100644 --- a/tests/queries/0_stateless/03115_alias_exists_column.sql +++ b/tests/queries/0_stateless/03115_alias_exists_column.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44412 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT EXISTS(SELECT 1) AS mycheck FORMAT TSVWithNames; diff --git a/tests/queries/0_stateless/03116_analyzer_explicit_alias_as_column_name.sql b/tests/queries/0_stateless/03116_analyzer_explicit_alias_as_column_name.sql index 16035b8e72a..d3e3a29077a 100644 --- a/tests/queries/0_stateless/03116_analyzer_explicit_alias_as_column_name.sql +++ b/tests/queries/0_stateless/03116_analyzer_explicit_alias_as_column_name.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/39923 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT errors.name AS labels, diff --git a/tests/queries/0_stateless/03117_analyzer_same_column_name_as_func.sql b/tests/queries/0_stateless/03117_analyzer_same_column_name_as_func.sql index 43f428a1bf9..a3f4da89525 100644 --- a/tests/queries/0_stateless/03117_analyzer_same_column_name_as_func.sql +++ b/tests/queries/0_stateless/03117_analyzer_same_column_name_as_func.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/39855 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table x( a UInt64, diff --git a/tests/queries/0_stateless/03118_analyzer_multi_join_prewhere.sql b/tests/queries/0_stateless/03118_analyzer_multi_join_prewhere.sql index 8680e9215c3..84f89c2c647 100644 --- a/tests/queries/0_stateless/03118_analyzer_multi_join_prewhere.sql +++ b/tests/queries/0_stateless/03118_analyzer_multi_join_prewhere.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/4596 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE a1 ( ANIMAL Nullable(String) ) engine = MergeTree order by tuple(); insert into a1 values('CROCO'); diff --git a/tests/queries/0_stateless/03119_analyzer_window_function_in_CTE_alias.sql b/tests/queries/0_stateless/03119_analyzer_window_function_in_CTE_alias.sql index 2eb2c66b551..edbb324bda2 100644 --- a/tests/queries/0_stateless/03119_analyzer_window_function_in_CTE_alias.sql +++ b/tests/queries/0_stateless/03119_analyzer_window_function_in_CTE_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/47422 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; DROP TEMPORARY TABLE IF EXISTS test; CREATE TEMPORARY TABLE test (a Float32, id UInt64); diff --git a/tests/queries/0_stateless/03120_analyzer_dist_join.sql b/tests/queries/0_stateless/03120_analyzer_dist_join.sql index 624da39c69b..e40df56c5ac 100644 --- a/tests/queries/0_stateless/03120_analyzer_dist_join.sql +++ b/tests/queries/0_stateless/03120_analyzer_dist_join.sql @@ -1,6 +1,6 @@ -- Tags: no-replicated-database -- https://github.com/ClickHouse/ClickHouse/issues/8547 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET distributed_foreground_insert=1; CREATE TABLE a1_replicated ON CLUSTER test_shard_localhost ( diff --git a/tests/queries/0_stateless/03120_analyzer_param_in_CTE_alias.sql b/tests/queries/0_stateless/03120_analyzer_param_in_CTE_alias.sql index 16fffae4737..d1e3d5a2ffa 100644 --- a/tests/queries/0_stateless/03120_analyzer_param_in_CTE_alias.sql +++ b/tests/queries/0_stateless/03120_analyzer_param_in_CTE_alias.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/33000 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET param_test_a=30; diff --git a/tests/queries/0_stateless/03121_analyzer_filed_redefenition_in_subquery.sql b/tests/queries/0_stateless/03121_analyzer_filed_redefenition_in_subquery.sql index fe05259a320..891eb7aac1c 100644 --- a/tests/queries/0_stateless/03121_analyzer_filed_redefenition_in_subquery.sql +++ b/tests/queries/0_stateless/03121_analyzer_filed_redefenition_in_subquery.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/14739 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; drop table if exists test_subquery; diff --git a/tests/queries/0_stateless/03122_analyzer_collate_in_window_function.sql b/tests/queries/0_stateless/03122_analyzer_collate_in_window_function.sql index 707c5d889f7..efd2e2fc873 100644 --- a/tests/queries/0_stateless/03122_analyzer_collate_in_window_function.sql +++ b/tests/queries/0_stateless/03122_analyzer_collate_in_window_function.sql @@ -1,6 +1,6 @@ -- Tags: no-fasttest -- https://github.com/ClickHouse/ClickHouse/issues/44039 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table test_window_collate(c1 String, c2 String) engine=MergeTree order by c1; insert into test_window_collate values('1', '上海'); diff --git a/tests/queries/0_stateless/03123_analyzer_dist_join_CTE.sql b/tests/queries/0_stateless/03123_analyzer_dist_join_CTE.sql index 4fb8e0b91c4..4d9f5e2971a 100644 --- a/tests/queries/0_stateless/03123_analyzer_dist_join_CTE.sql +++ b/tests/queries/0_stateless/03123_analyzer_dist_join_CTE.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/22923 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SET prefer_localhost_replica=0; create table "t0" (a Int64, b Int64) engine = MergeTree() partition by a order by a; diff --git a/tests/queries/0_stateless/03124_analyzer_nested_CTE_dist_in.sql b/tests/queries/0_stateless/03124_analyzer_nested_CTE_dist_in.sql index 406a50c6d16..be5346efa1c 100644 --- a/tests/queries/0_stateless/03124_analyzer_nested_CTE_dist_in.sql +++ b/tests/queries/0_stateless/03124_analyzer_nested_CTE_dist_in.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23865 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table table_local engine = Memory AS select * from numbers(10); diff --git a/tests/queries/0_stateless/03125_analyzer_CTE_two_joins.sql b/tests/queries/0_stateless/03125_analyzer_CTE_two_joins.sql index f7d5bb5f195..934e2bc3656 100644 --- a/tests/queries/0_stateless/03125_analyzer_CTE_two_joins.sql +++ b/tests/queries/0_stateless/03125_analyzer_CTE_two_joins.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29748 -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; create table events ( distinct_id String ) engine = Memory; diff --git a/tests/queries/0_stateless/03126_column_not_under_group_by.sql b/tests/queries/0_stateless/03126_column_not_under_group_by.sql index 890a4e823a9..516126c899c 100644 --- a/tests/queries/0_stateless/03126_column_not_under_group_by.sql +++ b/tests/queries/0_stateless/03126_column_not_under_group_by.sql @@ -1,7 +1,6 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT v.x, r.a, sum(c) FROM (select 1 x, 2 c) AS v ANY LEFT JOIN (SELECT 1 x, 2 a) AS r ON v.x = r.x GROUP BY v.x; -- { serverError NOT_AN_AGGREGATE} - diff --git a/tests/queries/0_stateless/03129_cte_with_final.sql b/tests/queries/0_stateless/03129_cte_with_final.sql index 01e1ca6dfb0..2a0714ec571 100644 --- a/tests/queries/0_stateless/03129_cte_with_final.sql +++ b/tests/queries/0_stateless/03129_cte_with_final.sql @@ -12,7 +12,7 @@ ORDER BY key; INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); INSERT INTO t Values (1, 'first', '2024-04-19 01:01:01'); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE passes=1 WITH merged_test AS( diff --git a/tests/queries/0_stateless/03130_analyzer_self_join_group_by.sql b/tests/queries/0_stateless/03130_analyzer_self_join_group_by.sql index 66b6b99981b..81af10c4a64 100644 --- a/tests/queries/0_stateless/03130_analyzer_self_join_group_by.sql +++ b/tests/queries/0_stateless/03130_analyzer_self_join_group_by.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t1; CREATE TABLE t1 (x Int32) ENGINE = MergeTree ORDER BY x; INSERT INTO t1 VALUES (1), (2), (3); -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT t2.x FROM t1 JOIN t1 as t2 ON t1.x = t2.x GROUP BY t1.x; -- { serverError NOT_AN_AGGREGATE } SELECT t2.number FROM numbers(10) as t1 JOIN numbers(10) as t2 ON t1.number = t2.number GROUP BY t1.number; -- { serverError NOT_AN_AGGREGATE } 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..4e42d94fd79 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 @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET join_algorithm = 'hash'; DROP TABLE IF EXISTS test_table_1; diff --git a/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.sql b/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.sql index 4e8096fbed1..7b7237ea7d1 100644 --- a/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.sql +++ b/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- { echoOn } SELECT concat(1, sum(if(0, toUInt128(concat('%', toLowCardinality(toNullable(1)), toUInt256(1))), materialize(0)))); SELECT any(if((number % 10) = 5, number, CAST(NULL, 'Nullable(Int128)'))) AS a, toTypeName(a) FROM numbers(100) AS a; diff --git a/tests/queries/0_stateless/03132_sqlancer_union_all.sql b/tests/queries/0_stateless/03132_sqlancer_union_all.sql index 576da9e53d5..2502ce31ec3 100644 --- a/tests/queries/0_stateless/03132_sqlancer_union_all.sql +++ b/tests/queries/0_stateless/03132_sqlancer_union_all.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t0; DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/03142_untuple_crash.sql b/tests/queries/0_stateless/03142_untuple_crash.sql index ac5dbba0de1..45106973271 100644 --- a/tests/queries/0_stateless/03142_untuple_crash.sql +++ b/tests/queries/0_stateless/03142_untuple_crash.sql @@ -1,2 +1,2 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT untuple(x -> 0) -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03142_window_function_limit_by.sql b/tests/queries/0_stateless/03142_window_function_limit_by.sql index 44dd890db41..a3f6b56aee5 100644 --- a/tests/queries/0_stateless/03142_window_function_limit_by.sql +++ b/tests/queries/0_stateless/03142_window_function_limit_by.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; -- https://github.com/ClickHouse/ClickHouse/issues/55965 @@ -8,7 +8,7 @@ CREATE TABLE error_win_func `k` String, `in` UInt64, `out` UInt64 -) +) ENGINE = MergeTree ORDER BY k AS SELECT * from VALUES (('a', 2, 4), ('a', 4, 2), ('a', 6, 3), ('a', 8, 4)); @@ -16,7 +16,7 @@ SELECT * from VALUES (('a', 2, 4), ('a', 4, 2), ('a', 6, 3), ('a', 8, 4)); SELECT k, in / out AS ratio, - count(*) OVER w AS count_rows_w + count(*) OVER w AS count_rows_w FROM error_win_func WINDOW w AS (ROWS BETWEEN CURRENT ROW AND 3 FOLLOWING) @@ -38,4 +38,3 @@ WHERE st IN ('x', 'y') LIMIT 1 BY m; DROP TABLE t; - diff --git a/tests/queries/0_stateless/03143_cte_scope.sql b/tests/queries/0_stateless/03143_cte_scope.sql index 1b1d9444651..cf4478e2ff9 100644 --- a/tests/queries/0_stateless/03143_cte_scope.sql +++ b/tests/queries/0_stateless/03143_cte_scope.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/56287 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tmp_a; DROP TABLE IF EXISTS tmp_b; diff --git a/tests/queries/0_stateless/03143_group_by_constant_secondary.sql b/tests/queries/0_stateless/03143_group_by_constant_secondary.sql index 030e3212748..099160dd522 100644 --- a/tests/queries/0_stateless/03143_group_by_constant_secondary.sql +++ b/tests/queries/0_stateless/03143_group_by_constant_secondary.sql @@ -3,6 +3,6 @@ SELECT count() FROM remote(test_cluster_two_shards, system, one) GROUP BY 'hi' SETTINGS - allow_experimental_analyzer = 1, + enable_analyzer = 1, group_by_two_level_threshold = 1, group_by_two_level_threshold_bytes = 33950592; diff --git a/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql b/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql index 97ed29802c7..02a8a2f3ce0 100644 --- a/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql +++ b/tests/queries/0_stateless/03143_parallel_replicas_mat_view_bug.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS mv_table; DROP TABLE IF EXISTS null_table; SET cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=4, allow_experimental_parallel_reading_from_replicas=1; -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE TABLE null_table (str String) ENGINE = Null; CREATE MATERIALIZED VIEW mv_table (str String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03143_parallel_replicas_mat_view_bug', '{replica}') ORDER BY str AS SELECT str AS str FROM null_table; diff --git a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql index 5bb4a8c9c9c..4445d5b8449 100644 --- a/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql +++ b/tests/queries/0_stateless/03144_aggregate_states_with_different_types.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; select * APPLY hex from ( diff --git a/tests/queries/0_stateless/03144_invalid_filter.sql b/tests/queries/0_stateless/03144_invalid_filter.sql index deb8d7b96b3..5b434972cc7 100644 --- a/tests/queries/0_stateless/03144_invalid_filter.sql +++ b/tests/queries/0_stateless/03144_invalid_filter.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/48049 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE test_table (`id` UInt64, `value` String) ENGINE = TinyLog() AS Select number, number::String from numbers(10); diff --git a/tests/queries/0_stateless/03146_bug47862.sql b/tests/queries/0_stateless/03146_bug47862.sql index 918f2316bea..0f411400618 100644 --- a/tests/queries/0_stateless/03146_bug47862.sql +++ b/tests/queries/0_stateless/03146_bug47862.sql @@ -9,4 +9,4 @@ FROM ( WHERE val_idx != 0 ) WHERE cast_res > 0 -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03146_tpc_ds_grouping.sql b/tests/queries/0_stateless/03146_tpc_ds_grouping.sql index f48c40e9bc4..cb290086b51 100644 --- a/tests/queries/0_stateless/03146_tpc_ds_grouping.sql +++ b/tests/queries/0_stateless/03146_tpc_ds_grouping.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/46335 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT key_a + key_b AS d, rank() OVER () AS f diff --git a/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql index 1b647a7ee62..75b7489ae5b 100644 --- a/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql +++ b/tests/queries/0_stateless/03148_query_log_used_dictionaries.sql @@ -15,7 +15,7 @@ SELECT dictGet('03148_dictionary', 'name', number) as dict_value FROM numbers(1) SETTINGS - allow_experimental_analyzer = 1, + enable_analyzer = 1, log_comment = 'simple_with_analyzer' FORMAT Null; @@ -34,7 +34,7 @@ FROM ( FROM numbers(1) ) t SETTINGS - allow_experimental_analyzer = 1, + enable_analyzer = 1, log_comment = 'nested_with_analyzer' FORMAT Null; @@ -50,7 +50,7 @@ SELECT dictGet('03148_dictionary', 'name', number) as dict_value FROM numbers(1) SETTINGS - allow_experimental_analyzer = 0, + enable_analyzer = 0, log_comment = 'simple_without_analyzer' FORMAT Null; @@ -69,7 +69,7 @@ FROM ( FROM numbers(1) ) t SETTINGS - allow_experimental_analyzer = 0, + enable_analyzer = 0, log_comment = 'nested_without_analyzer' FORMAT Null; diff --git a/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql b/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql index af326c15bd8..38f25f60e09 100644 --- a/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql +++ b/tests/queries/0_stateless/03148_setting_max_streams_to_max_threads_ratio_overflow.sql @@ -7,8 +7,8 @@ CREATE TABLE test_table INSERT INTO test_table VALUES (0, 'Value_0'); -SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, allow_experimental_analyzer = 1; -- { serverError PARAMETER_OUT_OF_BOUND } +SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, enable_analyzer = 1; -- { serverError PARAMETER_OUT_OF_BOUND } -SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, allow_experimental_analyzer = 0; -- { serverError PARAMETER_OUT_OF_BOUND } +SELECT * FROM test_table SETTINGS max_threads = 1025, max_streams_to_max_threads_ratio = -9223372036854775808, enable_analyzer = 0; -- { serverError PARAMETER_OUT_OF_BOUND } DROP TABLE test_table; diff --git a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql index a0bd1381351..c39143216d4 100644 --- a/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql +++ b/tests/queries/0_stateless/03150_grouping_sets_use_nulls_pushdown.sql @@ -21,21 +21,21 @@ SELECT * FROM ( SELECT day_ FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) WHERE day_ = '2023-01-05' ORDER BY * -SETTINGS allow_experimental_analyzer=1; +SETTINGS enable_analyzer=1; SELECT * FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) WHERE day_ = '2023-01-05' GROUP BY * ORDER BY ALL -SETTINGS allow_experimental_analyzer=1; +SETTINGS enable_analyzer=1; SELECT * FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (*), (day_) ) ) WHERE day_ = '2023-01-05' GROUP BY GROUPING SETS (*) ORDER BY type_1 -SETTINGS allow_experimental_analyzer=1; +SETTINGS enable_analyzer=1; SELECT * FROM ( SELECT day_, COUNT(*) FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) diff --git a/tests/queries/0_stateless/03151_analyzer_view_read_only_necessary_columns.sql b/tests/queries/0_stateless/03151_analyzer_view_read_only_necessary_columns.sql index 40204b5cd03..ac86a8705d5 100644 --- a/tests/queries/0_stateless/03151_analyzer_view_read_only_necessary_columns.sql +++ b/tests/queries/0_stateless/03151_analyzer_view_read_only_necessary_columns.sql @@ -8,7 +8,7 @@ CREATE TABLE test_table DROP VIEW IF EXISTS test_view; CREATE VIEW test_view AS SELECT id, value FROM test_table; -EXPLAIN header = 1 SELECT sum(id) FROM test_view settings allow_experimental_analyzer=1; +EXPLAIN header = 1 SELECT sum(id) FROM test_view settings enable_analyzer=1; DROP VIEW test_view; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.sql b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.sql index 645e89034d7..ea8a9e1f8d7 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.sql +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS users; CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree order by (uid, name); diff --git a/tests/queries/0_stateless/03154_recursive_cte_distributed.sql b/tests/queries/0_stateless/03154_recursive_cte_distributed.sql index b8c3356d5d6..47e0b9aad0b 100644 --- a/tests/queries/0_stateless/03154_recursive_cte_distributed.sql +++ b/tests/queries/0_stateless/03154_recursive_cte_distributed.sql @@ -1,6 +1,6 @@ -- Tags: shard -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql index 30423cb86ff..42c5f5ef65f 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.sql +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/62464 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT n, [number] AS inter FROM ( SELECT toFloat32(number % 10) AS n, number diff --git a/tests/queries/0_stateless/03155_in_nested_subselects.sql b/tests/queries/0_stateless/03155_in_nested_subselects.sql index 4f5ccd30aa3..faecb73040d 100644 --- a/tests/queries/0_stateless/03155_in_nested_subselects.sql +++ b/tests/queries/0_stateless/03155_in_nested_subselects.sql @@ -1,5 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/63833 -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; create table Example (id Int32) engine = MergeTree ORDER BY id; INSERT INTO Example SELECT number AS id FROM numbers(2); @@ -8,10 +8,10 @@ create table Null engine=Null as Example ; --create table Null engine=MergeTree order by id as Example ; create materialized view Transform to Example as -select * from Null -join ( select * FROM Example +select * from Null +join ( select * FROM Example WHERE id IN (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM Null))))) - ) as old + ) as old using id; INSERT INTO Null SELECT number AS id FROM numbers(2); diff --git a/tests/queries/0_stateless/03161_cnf_reduction.reference b/tests/queries/0_stateless/03161_cnf_reduction.reference index 5e39c0f3223..41051af362f 100644 --- a/tests/queries/0_stateless/03161_cnf_reduction.reference +++ b/tests/queries/0_stateless/03161_cnf_reduction.reference @@ -2,7 +2,7 @@ SELECT id FROM `03161_table` WHERE f -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 1 +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 1 -- Expected result with analyzer: 1 @@ -11,7 +11,7 @@ SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experim SELECT id FROM `03161_table` WHERE f -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 0 +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 0 -- Expected result w/o analyzer: 1 diff --git a/tests/queries/0_stateless/03161_cnf_reduction.sql b/tests/queries/0_stateless/03161_cnf_reduction.sql index b34e9171d45..c232823e9cd 100644 --- a/tests/queries/0_stateless/03161_cnf_reduction.sql +++ b/tests/queries/0_stateless/03161_cnf_reduction.sql @@ -10,7 +10,7 @@ EXPLAIN SYNTAX SELECT id FROM 03161_table WHERE f AND (NOT(f) OR f) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 1; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 1; SELECT ''; @@ -19,7 +19,7 @@ SELECT '-- Expected result with analyzer:'; SELECT id FROM 03161_table WHERE f AND (NOT(f) OR f) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 1; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 1; SELECT ''; @@ -29,7 +29,7 @@ EXPLAIN SYNTAX SELECT id FROM 03161_table WHERE f AND (NOT(f) OR f) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 0; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 0; SELECT ''; @@ -38,7 +38,7 @@ SELECT '-- Expected result w/o analyzer:'; SELECT id FROM 03161_table WHERE f AND (NOT(f) OR f) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 0; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 0; DROP TABLE IF EXISTS 03161_table; @@ -58,7 +58,7 @@ SELECT '-- Reproducer from the issue with analyzer'; SELECT count() FROM 03161_reproducer WHERE ((NOT c2) AND c2 AND (NOT c1)) OR ((NOT c2) AND c3 AND (NOT c5)) OR ((NOT c7) AND (NOT c8)) OR (c9 AND c6 AND c8 AND (NOT c8) AND (NOT c7)) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 1; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 1; SELECT ''; @@ -67,6 +67,6 @@ SELECT '-- Reproducer from the issue w/o analyzer'; SELECT count() FROM 03161_reproducer WHERE ((NOT c2) AND c2 AND (NOT c1)) OR ((NOT c2) AND c3 AND (NOT c5)) OR ((NOT c7) AND (NOT c8)) OR (c9 AND c6 AND c8 AND (NOT c8) AND (NOT c7)) -SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, allow_experimental_analyzer = 0; +SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, enable_analyzer = 0; DROP TABLE IF EXISTS 03161_reproducer; diff --git a/tests/queries/0_stateless/03164_analyzer_global_in_alias.sql b/tests/queries/0_stateless/03164_analyzer_global_in_alias.sql index 00c293334ee..ccfacd12d98 100644 --- a/tests/queries/0_stateless/03164_analyzer_global_in_alias.sql +++ b/tests/queries/0_stateless/03164_analyzer_global_in_alias.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT 1 GLOBAL IN (SELECT 1) AS s, s FROM remote('127.0.0.{2,3}', system.one) GROUP BY 1; SELECT 1 GLOBAL IN (SELECT 1) AS s FROM remote('127.0.0.{2,3}', system.one) GROUP BY 1; diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql index dbffbc1af71..b1018d00082 100644 --- a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql @@ -27,4 +27,4 @@ ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, insert into checks select * from generateRandom() limit 1; -select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0; +select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS enable_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0; diff --git a/tests/queries/0_stateless/03164_materialize_skip_index.sql b/tests/queries/0_stateless/03164_materialize_skip_index.sql index 4e59ef6b6cd..0443872596d 100644 --- a/tests/queries/0_stateless/03164_materialize_skip_index.sql +++ b/tests/queries/0_stateless/03164_materialize_skip_index.sql @@ -9,7 +9,7 @@ CREATE TABLE t_skip_index_insert ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 4; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SET materialize_skip_indexes_on_insert = 0; SYSTEM STOP MERGES t_skip_index_insert; diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.reference b/tests/queries/0_stateless/03165_order_by_duplicate.reference index 5d5e7a33f4a..93d312d5d02 100644 --- a/tests/queries/0_stateless/03165_order_by_duplicate.reference +++ b/tests/queries/0_stateless/03165_order_by_duplicate.reference @@ -36,4 +36,4 @@ QUERY id: 0 LIMIT BY LIST id: 16, nodes: 1 COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 - SETTINGS allow_experimental_analyzer=1 + SETTINGS enable_analyzer=1 diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.sql b/tests/queries/0_stateless/03165_order_by_duplicate.sql index 0054cbc36a6..b8bcc10e119 100644 --- a/tests/queries/0_stateless/03165_order_by_duplicate.sql +++ b/tests/queries/0_stateless/03165_order_by_duplicate.sql @@ -13,4 +13,4 @@ WHERE id IN ( ) ORDER BY id ASC LIMIT 1 BY id -SETTINGS allow_experimental_analyzer = 1; +SETTINGS enable_analyzer = 1; diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql index e32d23920dd..e27e8645466 100644 --- a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql @@ -3,5 +3,5 @@ create table dst (s String, lc LowCardinality(String)) engine MergeTree order by create materialized view mv to dst (s String, lc String) as select 'a' as s, toLowCardinality('b') as lc from src; insert into src values (1); -select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=0; -select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=1; +select s, lc from mv where not ignore(lc) settings enable_analyzer=0; +select s, lc from mv where not ignore(lc) settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index d3e3b38a3cb..96221f27e73 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS t_ind_merge_1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; CREATE TABLE t_ind_merge_1 (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) ENGINE = MergeTree diff --git a/tests/queries/0_stateless/03167_parametrized_view_with_cte.sql b/tests/queries/0_stateless/03167_parametrized_view_with_cte.sql index 1ac5540047a..ae6ab586415 100644 --- a/tests/queries/0_stateless/03167_parametrized_view_with_cte.sql +++ b/tests/queries/0_stateless/03167_parametrized_view_with_cte.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; CREATE OR REPLACE VIEW param_test AS SELECT {test_str:String} as s_result; WITH 'OK' AS s SELECT * FROM param_test(test_str=s); WITH (SELECT 123) AS s SELECT * FROM param_test(test_str=s); diff --git a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql index 50d99b851a6..5ab32415f1c 100644 --- a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql +++ b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql @@ -7,7 +7,7 @@ FROM FROM numbers(4096 * 100) GROUP BY k ) -SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=0; +SETTINGS enable_analyzer = 1, optimize_injective_functions_inside_uniq=0; SELECT sum(u) FROM @@ -18,4 +18,4 @@ FROM FROM numbers(4096 * 100) GROUP BY k ) -SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=1; +SETTINGS enable_analyzer = 1, optimize_injective_functions_inside_uniq=1; diff --git a/tests/queries/0_stateless/03170_part_offset_as_table_column.sql b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql index 36cbc156744..7711457f23f 100644 --- a/tests/queries/0_stateless/03170_part_offset_as_table_column.sql +++ b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql @@ -9,12 +9,12 @@ ORDER BY key; INSERT INTO test_table (key) SELECT number FROM numbers(10); -set allow_experimental_analyzer=0; +set enable_analyzer=0; SELECT * FROM test_table; -set allow_experimental_analyzer=1; +set enable_analyzer=1; SELECT * FROM test_table; diff --git a/tests/queries/0_stateless/03171_condition_pushdown.sql b/tests/queries/0_stateless/03171_condition_pushdown.sql index 9cfe41ce921..fcf5db886e9 100644 --- a/tests/queries/0_stateless/03171_condition_pushdown.sql +++ b/tests/queries/0_stateless/03171_condition_pushdown.sql @@ -1,5 +1,5 @@ -- This query succeeds only if it is correctly optimized. -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (123, 456) AND t2.number = t1.number ORDER BY ALL; -- Still TODO: diff --git a/tests/queries/0_stateless/03171_function_to_subcolumns_fuzzer.sql b/tests/queries/0_stateless/03171_function_to_subcolumns_fuzzer.sql index f10019a78dd..53476c5bdd1 100644 --- a/tests/queries/0_stateless/03171_function_to_subcolumns_fuzzer.sql +++ b/tests/queries/0_stateless/03171_function_to_subcolumns_fuzzer.sql @@ -1,5 +1,5 @@ SET optimize_functions_to_subcolumns = 1; -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS t_func_to_subcolumns_map_2; diff --git a/tests/queries/0_stateless/03173_forbid_qualify.sql b/tests/queries/0_stateless/03173_forbid_qualify.sql index d8cb2bad2ea..0a41385c52f 100644 --- a/tests/queries/0_stateless/03173_forbid_qualify.sql +++ b/tests/queries/0_stateless/03173_forbid_qualify.sql @@ -4,8 +4,8 @@ create table test_qualify (number Int64) ENGINE = MergeTree ORDER BY (number); insert into test_qualify SELECT * FROM numbers(100); select count() from test_qualify; -- 100 -select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 1; -- 49 -select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS enable_analyzer = 1; -- 49 +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS enable_analyzer = 0; -- { serverError NOT_IMPLEMENTED } delete from test_qualify where number in (select number from test_qualify qualify row_number() over (order by number) = 50); -- { serverError UNFINISHED } select count() from test_qualify; -- 100 diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index 20a29e2734e..7c567c0f58f 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -nq " " $CLICKHOUSE_CLIENT -nq " -SET allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; +SET enable_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; SELECT id, diff --git a/tests/queries/0_stateless/03174_merge_join_bug.sql b/tests/queries/0_stateless/03174_merge_join_bug.sql index ab4cb6cd4a9..ab3c384765d 100644 --- a/tests/queries/0_stateless/03174_merge_join_bug.sql +++ b/tests/queries/0_stateless/03174_merge_join_bug.sql @@ -1,6 +1,6 @@ -- Tags: no-random-settings -SET allow_experimental_analyzer=1, join_algorithm = 'full_sorting_merge'; +SET enable_analyzer=1, join_algorithm = 'full_sorting_merge'; CREATE TABLE xxxx_yyy (key UInt32, key_b ALIAS key) ENGINE=MergeTree() ORDER BY key SETTINGS ratio_of_defaults_for_sparse_serialization=0.0; INSERT INTO xxxx_yyy SELECT number FROM numbers(10); diff --git a/tests/queries/0_stateless/03199_join_with_materialized_column.sql b/tests/queries/0_stateless/03199_join_with_materialized_column.sql index 8c53c5b3e66..fef171cb9d1 100644 --- a/tests/queries/0_stateless/03199_join_with_materialized_column.sql +++ b/tests/queries/0_stateless/03199_join_with_materialized_column.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS table_with_materialized; CREATE TABLE table_with_materialized (col String MATERIALIZED 'A') ENGINE = Memory; diff --git a/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql b/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql index c32d7524492..d400a025f76 100644 --- a/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql +++ b/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer=1; +SET enable_analyzer=1; SELECT *, ngramMinHash(*) AS minhash, mortonEncode(untuple(ngramMinHash(*))) AS z FROM (SELECT toString(number) FROM numbers(10)) @@ -38,4 +38,3 @@ ORDER BY tuple(); INSERT INTO seq VALUES (0), (6), (7); WITH (Select min(number), max(number) from seq) as range Select * from numbers(range.1, range.2); - diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql index a01a595dbb5..25f3bb0f4c8 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql @@ -2,6 +2,5 @@ set allow_experimental_dynamic_type=1; create table test (d Dynamic) engine=Memory; insert into table test select * from numbers(5); alter table test modify column d Dynamic(max_types=1); -select d.UInt64 from test settings allow_experimental_analyzer=1; -select d.UInt64 from test settings allow_experimental_analyzer=0; - +select d.UInt64 from test settings enable_analyzer=1; +select d.UInt64 from test settings enable_analyzer=0; diff --git a/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql index 2dd0a37657d..6777a753490 100644 --- a/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql +++ b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql @@ -8,6 +8,6 @@ SELECT count() FROM (SELECT number FROM numbers(10)) as tbl LEFT JOIN t_subcolumns_join ON number = id WHERE id is null -SETTINGS allow_experimental_analyzer = 1, optimize_functions_to_subcolumns = 1, join_use_nulls = 1; +SETTINGS enable_analyzer = 1, optimize_functions_to_subcolumns = 1, join_use_nulls = 1; DROP TABLE t_subcolumns_join; diff --git a/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql index 24369fd6497..b791f328da4 100644 --- a/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql +++ b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql @@ -1,2 +1,2 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; EXPLAIN QUERY TREE SELECT tuple(sumIf(toInt64(1), 1)) FROM numbers(100) settings optimize_rewrite_sum_if_to_count_if=1; diff --git a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql index 03e9e0feb40..1d10a2b9212 100644 --- a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql +++ b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql @@ -1,4 +1,4 @@ -SET allow_experimental_analyzer = 1; +SET enable_analyzer = 1; SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; @@ -14,4 +14,3 @@ SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 170450 SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; - diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index b4b58ba3cb0..343e87a26d1 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -2,5 +2,5 @@ set max_threads=10; set optimize_use_implicit_projections=1; -EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS allow_experimental_analyzer = 0; -EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS allow_experimental_analyzer = 1; +EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS enable_analyzer = 0; +EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS enable_analyzer = 1; diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.reference b/tests/queries/1_stateful/00173_group_by_use_nulls.reference index e82b996ad3c..4f8d10a9221 100644 --- a/tests/queries/1_stateful/00173_group_by_use_nulls.reference +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.reference @@ -9,7 +9,7 @@ 33010362 1336 800784 1336 -- { echoOn } -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; SELECT CounterID AS k, quantileBFloat16(0.5)(ResolutionWidth) diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.sql b/tests/queries/1_stateful/00173_group_by_use_nulls.sql index 8531e9efaf8..ed537bb289a 100644 --- a/tests/queries/1_stateful/00173_group_by_use_nulls.sql +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.sql @@ -21,7 +21,7 @@ LIMIT 10 SETTINGS group_by_use_nulls = 1 FORMAT Null; -- { echoOn } -set allow_experimental_analyzer = 1; +set enable_analyzer = 1; SELECT CounterID AS k, From cc5dd9830e5063d5a6c71cd9c6e406961e78861d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Jul 2024 14:56:47 +0200 Subject: [PATCH 0859/1170] Modify the docs --- docs/en/development/architecture.md | 2 +- docs/ru/development/architecture.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index c5d13ab63a5..23531f742c5 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -118,7 +118,7 @@ And the result of interpreting the `INSERT SELECT` query is a "completed" `Query `InterpreterSelectQuery` uses `ExpressionAnalyzer` and `ExpressionActions` machinery for query analysis and transformations. This is where most rule-based query optimizations are performed. `ExpressionAnalyzer` is quite messy and should be rewritten: various query transformations and optimizations should be extracted into separate classes to allow for modular transformations of the query. -To address current problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` is being developed. It is a new version of `InterpreterSelectQuery` that does not use `ExpressionAnalyzer` and introduces an additional abstraction level between `AST` and `QueryPipeline` called `QueryTree`. It is not production-ready yet, but it can be tested with the `enable_analyzer` flag. +To address problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` has been developed. IThis is a new version of the `Interpreter Select Query`, which does not use the `Expression Analyzer` and introduces an additional layer of abstraction between `AST` and `QueryPipeline`, called `QueryTree'. It is fully ready for use in production, but just in case it can be turned off by setting the value of the `enable_analyzer` setting to `false`. ## Functions {#functions} diff --git a/docs/ru/development/architecture.md b/docs/ru/development/architecture.md index 0701c8f4a51..0833120c34d 100644 --- a/docs/ru/development/architecture.md +++ b/docs/ru/development/architecture.md @@ -115,7 +115,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны `InterpreterSelectQuery` использует `ExpressionAnalyzer` и `ExpressionActions` механизмы для анализа запросов и преобразований. Именно здесь выполняется большинство оптимизаций запросов на основе правил. `ExpressionAnalyzer` написан довольно грязно и должен быть переписан: различные преобразования запросов и оптимизации должны быть извлечены в отдельные классы, чтобы позволить модульные преобразования или запросы. -Для решения текущих проблем, существующих в интерпретаторах, разрабатывается новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он еще не готов к использованию в продакшене, но его можно протестировать с помощью флага `enable_analyzer`. +Для решения проблем, существующих в интерпретаторах, был разработан новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он полностью готов к использованию в продакшене, но на всякий случай его можно выключить, установив значение настройки `enable_analyzer` в `false`. ## Функции {#functions} From 62d5e83f2540d47487846da19d5297794126705e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Jul 2024 15:01:48 +0200 Subject: [PATCH 0860/1170] Double space... --- docs/ru/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/architecture.md b/docs/ru/development/architecture.md index 0833120c34d..d2afbf233b8 100644 --- a/docs/ru/development/architecture.md +++ b/docs/ru/development/architecture.md @@ -115,7 +115,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны `InterpreterSelectQuery` использует `ExpressionAnalyzer` и `ExpressionActions` механизмы для анализа запросов и преобразований. Именно здесь выполняется большинство оптимизаций запросов на основе правил. `ExpressionAnalyzer` написан довольно грязно и должен быть переписан: различные преобразования запросов и оптимизации должны быть извлечены в отдельные классы, чтобы позволить модульные преобразования или запросы. -Для решения проблем, существующих в интерпретаторах, был разработан новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он полностью готов к использованию в продакшене, но на всякий случай его можно выключить, установив значение настройки `enable_analyzer` в `false`. +Для решения проблем, существующих в интерпретаторах, был разработан новый `InterpreterSelectQueryAnalyzer`. Это новая версия `InterpreterSelectQuery`, которая не использует `ExpressionAnalyzer` и вводит дополнительный уровень абстракции между `AST` и `QueryPipeline`, называемый `QueryTree`. Он полностью готов к использованию в продакшене, но на всякий случай его можно выключить, установив значение настройки `enable_analyzer` в `false`. ## Функции {#functions} From aa66203f1732f22f4a0d5ed4c4816e148ffd7861 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jul 2024 18:46:24 +0200 Subject: [PATCH 0861/1170] Update architecture.md --- docs/en/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index 23531f742c5..a1a5901f859 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -118,7 +118,7 @@ And the result of interpreting the `INSERT SELECT` query is a "completed" `Query `InterpreterSelectQuery` uses `ExpressionAnalyzer` and `ExpressionActions` machinery for query analysis and transformations. This is where most rule-based query optimizations are performed. `ExpressionAnalyzer` is quite messy and should be rewritten: various query transformations and optimizations should be extracted into separate classes to allow for modular transformations of the query. -To address problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` has been developed. IThis is a new version of the `Interpreter Select Query`, which does not use the `Expression Analyzer` and introduces an additional layer of abstraction between `AST` and `QueryPipeline`, called `QueryTree'. It is fully ready for use in production, but just in case it can be turned off by setting the value of the `enable_analyzer` setting to `false`. +To address problems that exist in interpreters, a new `InterpreterSelectQueryAnalyzer` has been developed. This is a new version of the `InterpreterSelectQuery`, which does not use the `ExpressionAnalyzer` and introduces an additional layer of abstraction between `AST` and `QueryPipeline`, called `QueryTree'. It is fully ready for use in production, but just in case it can be turned off by setting the value of the `enable_analyzer` setting to `false`. ## Functions {#functions} From ee2589df597cd853259a18212ae979d7e6d65150 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 15 Jul 2024 23:06:03 +0200 Subject: [PATCH 0862/1170] Add a settings to the history changelog --- src/Core/SettingsChangesHistory.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 08fb6dc3301..dc81932f923 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -524,6 +524,10 @@ static std::initializer_list Date: Tue, 16 Jul 2024 13:37:34 +0000 Subject: [PATCH 0863/1170] Fix tests --- .../0_stateless/02995_baseline_24_7_2.tsv | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/tests/queries/0_stateless/02995_baseline_24_7_2.tsv b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv index 10b392f3e04..d3a07ecb644 100644 --- a/tests/queries/0_stateless/02995_baseline_24_7_2.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv @@ -18,7 +18,11 @@ allow_distributed_ddl 1 allow_drop_detached 0 allow_execute_multiif_columnar 1 allow_experimental_alter_materialized_view_structure 1 +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv allow_experimental_analyzer 1 +======= +allow_experimental_analyzer 0 +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv allow_experimental_annoy_index 0 allow_experimental_bigint_types 1 allow_experimental_codecs 0 @@ -159,7 +163,10 @@ cloud_mode 0 cloud_mode_engine 1 cluster_for_parallel_replicas collect_hash_table_stats_during_aggregation 1 +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv collect_hash_table_stats_during_joins 1 +======= +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv column_names_for_schema_inference compatibility compatibility_ignore_auto_increment_in_create_table 0 @@ -329,9 +336,13 @@ format_regexp_escaping_rule Raw format_regexp_skip_unmatched 0 format_schema format_template_resultset +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv format_template_resultset_format format_template_row format_template_row_format +======= +format_template_row +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv format_template_rows_between_delimiter \n format_tsv_null_representation \\N formatdatetime_f_prints_single_zero 0 @@ -388,8 +399,11 @@ iceberg_engine_ignore_schema_evolution 0 idle_connection_timeout 3600 ignore_cold_parts_seconds 0 ignore_data_skipping_indices +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv ignore_drop_queries_probability 0 ignore_materialized_views_with_dropped_target_table 0 +======= +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv ignore_on_cluster_for_replicated_access_entities_queries 0 ignore_on_cluster_for_replicated_named_collections_queries 0 ignore_on_cluster_for_replicated_udf_queries 0 @@ -712,8 +726,13 @@ mutations_execute_subqueries_on_initiator 0 mutations_max_literal_size_to_replace 16384 mutations_sync 0 mysql_datatypes_support_level +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv mysql_map_fixed_string_to_text_in_show_columns 1 mysql_map_string_to_text_in_show_columns 1 +======= +mysql_map_fixed_string_to_text_in_show_columns 0 +mysql_map_string_to_text_in_show_columns 0 +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv mysql_max_rows_to_insert 65536 network_compression_method LZ4 network_zstd_compression_level 1 @@ -780,9 +799,13 @@ os_thread_priority 0 output_format_arrow_compression_method lz4_frame output_format_arrow_fixed_string_as_fixed_byte_array 1 output_format_arrow_low_cardinality_as_dictionary 0 +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv output_format_arrow_string_as_string 1 output_format_arrow_use_64_bit_indexes_for_dictionary 0 output_format_arrow_use_signed_indexes_for_dictionary 1 +======= +output_format_arrow_string_as_string 0 +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv output_format_avro_codec output_format_avro_rows_in_file 1 output_format_avro_string_column_pattern @@ -1045,7 +1068,10 @@ totals_mode after_having_exclusive trace_profile_events 0 transfer_overflow_mode throw transform_null_in 0 +<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv traverse_shadow_remote_data_paths 0 +======= +>>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv union_default_mode unknown_packet_in_send_data 0 update_insert_deduplication_token_in_dependent_materialized_views 0 From a8ca5ad50b8998d9e13d81b66bc89434c1364704 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 29 Jul 2024 17:02:11 +0000 Subject: [PATCH 0864/1170] Fixed build and made setting an alias --- src/Client/HedgedConnections.cpp | 4 ++-- src/Client/MultiplexedConnections.cpp | 4 ++-- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/ActionsVisitor.cpp | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 4 ++-- .../ClusterProxy/executeQuery.cpp | 4 ++-- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 10 +++++----- src/Interpreters/InterpreterFactory.cpp | 6 +++--- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 6 +++--- src/Interpreters/executeQuery.cpp | 6 +++--- .../getHeaderForProcessingStage.cpp | 2 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 4 ++-- src/Server/TCPHandler.cpp | 6 +++--- src/Storages/AlterCommands.cpp | 2 +- src/Storages/IStorageCluster.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 18 ++++++++--------- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 6 +++--- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageMerge.cpp | 20 +++++++++---------- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- src/Storages/StorageView.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- src/TableFunctions/TableFunctionView.cpp | 2 +- .../TableFunctionViewIfPermitted.cpp | 2 +- 34 files changed, 82 insertions(+), 81 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 1c7f222aa78..dd8348ea04f 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -196,11 +196,11 @@ void HedgedConnections::sendQuery( modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; } - /// FIXME: Remove once we will make `enable_analyzer` obsolete setting. + /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. /// Make the analyzer being set, so it will be effectively applied on the remote server. /// In other words, the initiator always controls whether the analyzer enabled or not for /// all servers involved in the distributed query processing. - modified_settings.set("enable_analyzer", static_cast(modified_settings.enable_analyzer)); + modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); replica.connection->sendQuery( timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 7ca22ae4c81..244eccf1ed9 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -150,11 +150,11 @@ void MultiplexedConnections::sendQuery( client_info.number_of_current_replica = replica_info->number_of_current_replica; } - /// FIXME: Remove once we will make `enable_analyzer` obsolete setting. + /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. /// Make the analyzer being set, so it will be effectively applied on the remote server. /// In other words, the initiator always controls whether the analyzer enabled or not for /// all servers involved in the distributed query processing. - modified_settings.set("enable_analyzer", static_cast(modified_settings.enable_analyzer)); + modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d6c0dc223b2..ac24c087946 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -638,7 +638,7 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ - M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `enable_analyzer`", 0) \ + M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `allow_experimental_analyzer`", 0) \ M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \ M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \ M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ @@ -943,7 +943,7 @@ class IColumn; \ M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \ \ - M(Bool, enable_analyzer, true, "Allow new query analyzer.", IMPORTANT) ALIAS(allow_experimental_analyzer) \ + M(Bool, allow_experimental_analyzer, true, "Allow new query analyzer.", IMPORTANT) ALIAS(enable_analyzer) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ \ M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index dc81932f923..71f7c940e2c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -528,6 +528,7 @@ static std::initializer_listgetSettingsRef().enable_analyzer && !identifier) + if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { /// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter. /// This is a hacky way to allow reusing cache for prepared sets. diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0948f24eca0..e35d31d2350 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -68,7 +68,7 @@ ASTPtr rewriteSelectQuery( // are written into the query context and will be sent by the query pipeline. select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {}); - if (!context->getSettingsRef().enable_analyzer) + if (!context->getSettingsRef().allow_experimental_analyzer) { if (table_function_ptr) select_query.addTableFunction(table_function_ptr); @@ -165,7 +165,7 @@ void SelectStreamFactory::createForShardImpl( auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) { Block shard_header; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze()); else shard_header = header; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 6c8ab11bfc9..d04a73e384e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -300,7 +300,7 @@ void executeQuery( const size_t shards = cluster->getShardCount(); - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i) { @@ -581,7 +581,7 @@ void executeQueryWithParallelReplicasCustomKey( /// Return directly (with correct header) if no shard to query. if (query_info.getCluster()->getShardsInfo().empty()) { - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) return; Pipe pipe(std::make_shared(header)); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ea631ef01d5..971f90bd3cd 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -834,7 +834,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti Block as_select_sample; - if (getContext()->getSettingsRef().enable_analyzer) + if (getContext()->getSettingsRef().allow_experimental_analyzer) { as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); } @@ -1327,7 +1327,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { Block input_block; - if (getContext()->getSettingsRef().enable_analyzer) + if (getContext()->getSettingsRef().allow_experimental_analyzer) { input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 4a061f02c2b..39fc85a5e23 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -129,7 +129,7 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression auto select_query = table_expression.subquery->children.at(0); auto current_context = getContext(); - if (settings.enable_analyzer) + if (settings.allow_experimental_analyzer) { SelectQueryOptions select_query_options; sample_block = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock(); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 2fbfbf3a809..bedd9cb4a80 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -394,9 +394,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl() } case ASTExplainQuery::QueryTree: { - if (!getContext()->getSettingsRef().enable_analyzer) + if (!getContext()->getSettingsRef().allow_experimental_analyzer) throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "EXPLAIN QUERY TREE is only supported with a new analyzer. Set enable_analyzer = 1."); + "EXPLAIN QUERY TREE is only supported with a new analyzer. Set allow_experimental_analyzer = 1."); if (ast.getExplainedQuery()->as() == nullptr) throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERY TREE query"); @@ -453,7 +453,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() ContextPtr context; - if (getContext()->getSettingsRef().enable_analyzer) + if (getContext()->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options); context = interpreter.getContext(); @@ -499,7 +499,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() QueryPlan plan; ContextPtr context; - if (getContext()->getSettingsRef().enable_analyzer) + if (getContext()->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options); context = interpreter.getContext(); @@ -558,7 +558,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() QueryPlan plan; ContextPtr context = getContext(); - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); context = interpreter.getContext(); diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index a909c4e602d..12b3b510098 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -118,7 +118,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte if (query->as()) { - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) interpreter_name = "InterpreterSelectQueryAnalyzer"; /// This is internal part of ASTSelectWithUnionQuery. /// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child. @@ -129,7 +129,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { ProfileEvents::increment(ProfileEvents::SelectQuery); - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) interpreter_name = "InterpreterSelectQueryAnalyzer"; else interpreter_name = "InterpreterSelectWithUnionQuery"; @@ -222,7 +222,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { const auto kind = query->as()->getKind(); if (kind == ASTExplainQuery::ParsedAST || kind == ASTExplainQuery::AnalyzedSyntax) - context->setSetting("enable_analyzer", false); + context->setSetting("allow_experimental_analyzer", false); interpreter_name = "InterpreterExplainQuery"; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 0213e2a2c42..c97593a1781 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -95,7 +95,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) Block header_block; auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - if (current_context->getSettingsRef().enable_analyzer) + if (current_context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter_select(query.select, current_context, select_query_options); header_block = interpreter_select.getSampleBlock(); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index c049dbc9cc1..57ad5caa4c7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -189,7 +189,7 @@ bool isStorageTouchedByMutations( std::optional interpreter_select_query; BlockIO io; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits()); @@ -415,9 +415,9 @@ MutationsInterpreter::MutationsInterpreter( , logger(getLogger("MutationsInterpreter(" + source.getStorage()->getStorageID().getFullTableName() + ")")) { auto new_context = Context::createCopy(context_); - if (new_context->getSettingsRef().enable_analyzer) + if (new_context->getSettingsRef().allow_experimental_analyzer) { - new_context->setSetting("enable_analyzer", false); + new_context->setSetting("allow_experimental_analyzer", false); LOG_DEBUG(logger, "Will use old analyzer to prepare mutation"); } context = std::move(new_context); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 7476915ab8a..ce58f7f922c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -685,10 +685,10 @@ void validateAnalyzerSettings(ASTPtr ast, bool context_value) if (auto * set_query = node->as()) { - if (auto * value = set_query->changes.tryGet("enable_analyzer")) + if (auto * value = set_query->changes.tryGet("allow_experimental_analyzer")) { if (top_level != value->safeGet()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'enable_analyzer' is changed in the subquery. Top level value: {}", top_level); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); } } @@ -912,7 +912,7 @@ static std::tuple executeQueryImpl( /// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter), /// to allow settings to take effect. InterpreterSetQuery::applySettingsFromQuery(ast, context); - validateAnalyzerSettings(ast, context->getSettingsRef().enable_analyzer); + validateAnalyzerSettings(ast, context->getSettingsRef().allow_experimental_analyzer); if (auto * insert_query = ast->as()) insert_query->tail = istr; diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index c4a791e85e1..cf18cbbb54a 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -141,7 +141,7 @@ Block getHeaderForProcessingStage( Block result; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { auto storage = std::make_shared(storage_snapshot->storage.getStorageID(), storage_snapshot->getAllColumnsDescription(), diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index dc4b7fd733b..d8624a1c99b 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -65,7 +65,7 @@ std::unique_ptr createLocalPlan( .setShardInfo(static_cast(shard_num), static_cast(shard_count)) .ignoreASTOptimizations(); - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index a2d5ec5d1cb..98d66ed77c3 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -319,7 +319,7 @@ std::optional generateViewChain( Block header; /// Get list of columns we get from select query. - if (select_context->getSettingsRef().enable_analyzer) + if (select_context->getSettingsRef().allow_experimental_analyzer) header = InterpreterSelectQueryAnalyzer::getSampleBlock(query, select_context); else header = InterpreterSelectQuery(query, select_context, SelectQueryOptions()).getSampleBlock(); @@ -613,7 +613,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat QueryPipelineBuilder pipeline; - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(view.query, local_context, local_context->getViewSource(), SelectQueryOptions().ignoreAccessCheck()); pipeline = interpreter.buildQueryPipeline(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4262716b406..c5dfe3e6e5f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1904,14 +1904,14 @@ void TCPHandler::receiveQuery() /// Settings /// - /// FIXME: Remove when enable_analyzer will become obsolete. + /// FIXME: Remove when allow_experimental_analyzer will become obsolete. /// Analyzer became Beta in 24.3 and started to be enabled by default. /// We have to disable it for ourselves to make sure we don't have different settings on /// different servers. if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && client_info.getVersionNumber() < VersionNumber(23, 3, 0) - && !passed_settings.enable_analyzer.changed) - passed_settings.set("enable_analyzer", false); + && !passed_settings.allow_experimental_analyzer.changed) + passed_settings.set("allow_experimental_analyzer", false); auto settings_changes = passed_settings.changes(); query_kind = query_context->getClientInfo().query_kind; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 2843ff5a14e..7891042bb96 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -806,7 +806,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) metadata.select = SelectQueryDescription::getSelectQueryFromASTForMatView(select, metadata.refresh != nullptr, context); Block as_select_sample; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(select->clone(), context); } diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index b485ab9cbb5..63467603d16 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -125,7 +125,7 @@ void IStorageCluster::read( Block sample_block; ASTPtr query_to_send = query_info.query; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index c93da7ca512..71b1a0a73c9 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -378,7 +378,7 @@ void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Ch QueryPipelineBuilder builder; - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) { auto select_description = buildSelectQueryTreeDescription(select_query_description.inner_query, local_context); if (select_description.dependent_table_node) @@ -475,7 +475,7 @@ Block StorageLiveView::getHeader() const if (!sample_block) { - if (live_view_context->getSettingsRef().enable_analyzer) + if (live_view_context->getSettingsRef().allow_experimental_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(select_query_description.select_query, live_view_context, @@ -519,7 +519,7 @@ ASTPtr StorageLiveView::getInnerBlocksQuery() auto & select_with_union_query = select_query_description.select_query->as(); auto blocks_query = select_with_union_query.list_of_selects->children.at(0)->clone(); - if (!live_view_context->getSettingsRef().enable_analyzer) + if (!live_view_context->getSettingsRef().allow_experimental_analyzer) { /// Rewrite inner query with right aliases for JOIN. /// It cannot be done in constructor or startup() because InterpreterSelectQuery may access table, @@ -543,7 +543,7 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont QueryPipelineBuilder builder; - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(select_query_description.inner_query, local_context, @@ -599,7 +599,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) QueryPipelineBuilder builder; - if (block_context->getSettingsRef().enable_analyzer) + if (block_context->getSettingsRef().allow_experimental_analyzer) { auto select_description = buildSelectQueryTreeDescription(select_query_description.select_query, block_context); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 01ef0a409b0..ce27ad24e10 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7097,7 +7097,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( SelectQueryInfo &) const { /// with new analyzer, Planner make decision regarding parallel replicas usage, and so about processing stage on reading - if (!query_context->getSettingsRef().enable_analyzer) + if (!query_context->getSettingsRef().allow_experimental_analyzer) { const auto & settings = query_context->getSettingsRef(); if (query_context->canUseParallelReplicasCustomKey()) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index fccb20c2b0a..6e963066f39 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes namespace { -void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool enable_analyzer, bool legacy = false) +void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool allow_experimental_analyzer, bool legacy = false) { switch (node.type) { @@ -45,18 +45,18 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o /// If it was created from ASTLiteral, then result_name can be an alias. /// We need to convert value back to string here. const auto * column_const = typeid_cast(node.column.get()); - if (column_const && !enable_analyzer) + if (column_const && !allow_experimental_analyzer) writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out); else writeString(node.result_name, out); break; } case ActionsDAG::ActionType::ALIAS: - appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); break; case ActionsDAG::ActionType::ARRAY_JOIN: writeCString("arrayJoin(", out); - appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy); + appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy); writeChar(')', out); break; case ActionsDAG::ActionType::FUNCTION: @@ -75,17 +75,17 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o writeCString(", ", out); first = false; - appendColumnNameWithoutAlias(*arg, out, enable_analyzer, legacy); + appendColumnNameWithoutAlias(*arg, out, allow_experimental_analyzer, legacy); } writeChar(')', out); } } } -String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool enable_analyzer, bool legacy = false) +String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool allow_experimental_analyzer, bool legacy = false) { WriteBufferFromOwnString out; - appendColumnNameWithoutAlias(node, out, enable_analyzer, legacy); + appendColumnNameWithoutAlias(node, out, allow_experimental_analyzer, legacy); return std::move(out.str()); } @@ -131,7 +131,7 @@ std::string RPNBuilderTreeNode::getColumnName() const if (ast_node) return ast_node->getColumnNameWithoutAlias(); else - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer); } std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const @@ -144,7 +144,7 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const } else { - return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer, true /*legacy*/); + return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer, true /*legacy*/); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index da427ca4a6a..04e6d6676d1 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -397,7 +397,7 @@ void StorageBuffer::read( /// TODO: Find a way to support projections for StorageBuffer if (processed_stage > QueryProcessingStage::FetchColumns) { - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) { auto storage = std::make_shared( getStorageID(), diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f1fe70b4594..3e38ddf830a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -833,7 +833,7 @@ void StorageDistributed::read( const auto & settings = local_context->getSettingsRef(); - if (settings.enable_analyzer) + if (settings.allow_experimental_analyzer) { StorageID remote_storage_id = StorageID::createEmpty(); if (!remote_table_function_ptr) @@ -1057,7 +1057,7 @@ static std::optional getFilterFromQuery(const ASTPtr & ast, ContextP QueryPlan plan; SelectQueryOptions options; options.only_analyze = true; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast, context, options); plan = std::move(interpreter).extractQueryPlan(); @@ -1611,7 +1611,7 @@ ClusterPtr StorageDistributed::skipUnusedShards( const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) return skipUnusedShardsWithAnalyzer(cluster, query_info, storage_snapshot, local_context); const auto & select = query_info.query->as(); diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 27bfa6f854c..0094723e3fd 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -150,7 +150,7 @@ void StorageExecutable::read( for (auto & input_query : input_queries) { QueryPipelineBuilder builder; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) builder = InterpreterSelectQueryAnalyzer(input_query, context, {}).buildQueryPipeline(); else builder = InterpreterSelectWithUnionQuery(input_query, context, {}).buildQueryPipeline(); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 613317b2564..7c268d36a7b 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -590,7 +590,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ auto modified_query_info = getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases); - if (!context->getSettingsRef().enable_analyzer) + if (!context->getSettingsRef().allow_experimental_analyzer) { auto storage_columns = storage_metadata_snapshot->getColumns(); auto syntax_result = TreeRewriter(context).analyzeSelect( @@ -1047,13 +1047,13 @@ void ReadFromMerge::addVirtualColumns( const StorageWithLockAndName & storage_with_lock) const { const auto & [database_name, _, storage, table_name] = storage_with_lock; - bool enable_analyzer = context->getSettingsRef().enable_analyzer; + bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; /// Add virtual columns if we don't already have them. Block plan_header = child.plan.getCurrentDataStream().header; - if (enable_analyzer) + if (allow_experimental_analyzer) { String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); @@ -1133,8 +1133,8 @@ QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( if (!builder->initialized()) return builder; - bool enable_analyzer = context->getSettingsRef().enable_analyzer; - if (processed_stage > child.stage || (enable_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) + bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; + if (processed_stage > child.stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, @@ -1168,7 +1168,7 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( modified_select.setFinal(); } - bool enable_analyzer = modified_context->getSettingsRef().enable_analyzer; + bool allow_experimental_analyzer = modified_context->getSettingsRef().allow_experimental_analyzer; auto storage_stage = storage->getQueryProcessingStage(modified_context, processed_stage, @@ -1201,13 +1201,13 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( row_policy_data_opt->addStorageFilter(source_step_with_filter); } } - else if (processed_stage > storage_stage || enable_analyzer) + else if (processed_stage > storage_stage || allow_experimental_analyzer) { /// Maximum permissible parallelism is streams_num modified_context->setSetting("max_threads", streams_num); modified_context->setSetting("max_streams_to_max_threads_ratio", 1); - if (enable_analyzer) + if (allow_experimental_analyzer) { /// Converting query to AST because types might be different in the source table. /// Need to resolve types again. @@ -1479,7 +1479,7 @@ void ReadFromMerge::convertAndFilterSourceStream( auto storage_sample_block = snapshot->metadata->getSampleBlock(); auto pipe_columns = before_block_header.getNamesAndTypesList(); - if (local_context->getSettingsRef().enable_analyzer) + if (local_context->getSettingsRef().allow_experimental_analyzer) { for (const auto & alias : aliases) { @@ -1522,7 +1522,7 @@ void ReadFromMerge::convertAndFilterSourceStream( ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; - if (local_context->getSettingsRef().enable_analyzer + if (local_context->getSettingsRef().allow_experimental_analyzer && (child.stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ebc88993ee4..f55f672fe5e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -208,7 +208,7 @@ void StorageMergeTree::read( const auto & settings = local_context->getSettingsRef(); /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree - && !settings.enable_analyzer) + && !settings.allow_experimental_analyzer) { ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); @@ -216,7 +216,7 @@ void StorageMergeTree::read( } if (local_context->canUseParallelReplicasCustomKey() && settings.parallel_replicas_for_non_replicated_merge_tree - && !settings.enable_analyzer && local_context->getClientInfo().distributed_depth == 0) + && !settings.allow_experimental_analyzer && local_context->getClientInfo().distributed_depth == 0) { if (auto cluster = local_context->getClusterForParallelReplicas(); local_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) @@ -244,7 +244,7 @@ void StorageMergeTree::read( const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree - && (!local_context->getSettingsRef().enable_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); + && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); if (auto plan = reader.read( column_names, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a3965e7a6d4..2d826c6c2df 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5480,13 +5480,13 @@ void StorageReplicatedMergeTree::read( return; } /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here - if (local_context->canUseParallelReplicasOnInitiator() && !settings.enable_analyzer) + if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) { readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); return; } - if (local_context->canUseParallelReplicasCustomKey() && !settings.enable_analyzer + if (local_context->canUseParallelReplicasCustomKey() && !settings.allow_experimental_analyzer && local_context->getClientInfo().distributed_depth == 0) { if (auto cluster = local_context->getClusterForParallelReplicas(); @@ -5555,7 +5555,7 @@ void StorageReplicatedMergeTree::readLocalImpl( const size_t num_streams) { const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() - && (!local_context->getSettingsRef().enable_analyzer + && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas); auto plan = reader.read( diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index dcb5ef2ae77..878998ebf12 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -164,7 +164,7 @@ void StorageView::read( auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done); - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(current_inner_query, getViewContext(context, storage_snapshot), options, column_names); interpreter.addStorageLimits(*query_info.storage_limits); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 16eccfd7343..d674f054632 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -172,7 +172,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType /// with subqueries it's possible that new analyzer will be enabled in ::read method /// of underlying storage when all other parts of infra are not ready for it /// (built with old analyzer). - context_copy->setSetting("enable_analyzer", false); + context_copy->setSetting("allow_experimental_analyzer", false); auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a2b1704f24b..65bf6768b1b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1197,7 +1197,7 @@ StorageWindowView::StorageWindowView( , fire_signal_timeout_s(context_->getSettingsRef().wait_for_window_view_fire_signal_timeout.totalSeconds()) , clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds()) { - if (context_->getSettingsRef().enable_analyzer) + if (context_->getSettingsRef().allow_experimental_analyzer) disabled_due_to_analyzer = true; if (mode <= LoadingStrictnessLevel::CREATE) @@ -1753,9 +1753,9 @@ StoragePtr StorageWindowView::getTargetTable() const void StorageWindowView::throwIfWindowViewIsDisabled(ContextPtr local_context) const { - if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().enable_analyzer)) + if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().allow_experimental_analyzer)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Experimental WINDOW VIEW feature is not supported " - "in the current infrastructure for query analysis (the setting 'enable_analyzer')"); + "in the current infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); } void registerStorageWindowView(StorageFactory & factory) diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 02a278cf590..57501df6d4d 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -50,7 +50,7 @@ ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context Block sample_block; - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context); else sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context); diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index 7bae2731525..935be6c1987 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -114,7 +114,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const try { - if (context->getSettingsRef().enable_analyzer) + if (context->getSettingsRef().allow_experimental_analyzer) { sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context); } From ddcad048de8862392194d649788c56466a432b0e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 30 Jul 2024 14:44:56 +0000 Subject: [PATCH 0865/1170] Fix build --- src/Core/SettingsChangesHistory.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 71f7c940e2c..bd7330ac6f8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -500,6 +500,8 @@ static std::initializer_list Date: Tue, 30 Jul 2024 21:29:07 +0000 Subject: [PATCH 0866/1170] Minor --- src/Core/SettingsChangesHistory.cpp | 4 +--- tests/queries/0_stateless/00202_cross_join.sql | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index bd7330ac6f8..84c0ef2b127 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -528,9 +528,7 @@ static std::initializer_list Date: Wed, 31 Jul 2024 14:21:12 +0000 Subject: [PATCH 0867/1170] Build --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 84c0ef2b127..c5d47fcdc4b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -525,7 +525,6 @@ static std::initializer_list Date: Wed, 31 Jul 2024 15:32:32 +0000 Subject: [PATCH 0868/1170] Remove all changes --- src/Core/SettingsChangesHistory.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c5d47fcdc4b..75bc15358ab 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -525,9 +525,7 @@ static std::initializer_list Date: Wed, 31 Jul 2024 16:00:04 +0000 Subject: [PATCH 0869/1170] Add to settings changes --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 75bc15358ab..893394adb61 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -501,6 +501,7 @@ static std::initializer_list Date: Wed, 31 Jul 2024 22:45:18 +0200 Subject: [PATCH 0870/1170] Validate also alias --- src/Interpreters/executeQuery.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ce58f7f922c..fe87eed5570 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -690,6 +690,12 @@ void validateAnalyzerSettings(ASTPtr ast, bool context_value) if (top_level != value->safeGet()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level); } + + if (auto * value = set_query->changes.tryGet("enable_analyzer")) + { + if (top_level != value->safeGet()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'enable_analyzer' is changed in the subquery. Top level value: {}", top_level); + } } for (auto child : node->children) From ef811fd25c36855e1c39029962999609ef6fb4de Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Aug 2024 12:50:56 +0000 Subject: [PATCH 0871/1170] Fix test --- .../01049_join_low_card_bug_long.reference.j2 | 60 +++++++++---------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 index 872bb448027..341e77b1d78 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 @@ -19,17 +19,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -49,17 +49,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -79,17 +79,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l -- @@ -109,17 +109,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -333,17 +333,17 @@ str_r LowCardinality(String) str_l LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -- @@ -363,17 +363,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) +String String str_r str_r LowCardinality(String) LowCardinality(String) String String LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String String String String String str_l str_l -- @@ -393,17 +393,17 @@ str_r String str_l String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) str_r str_r String String LowCardinality(String) LowCardinality(String) String String str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; String String str str String String str str -String String str_r str_r String String +String String str_r str_r String String String String String String str_l str_l -- @@ -423,13 +423,13 @@ str_r Nullable(String) str_l Nullable(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str From 0301af99e5c6eda72a379a0d048903a3ecb9e0e0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Aug 2024 15:53:08 +0000 Subject: [PATCH 0872/1170] Fixed a bunch of tests --- tests/config/users.d/analyzer.xml | 2 +- .../integration/helpers/0_common_enable_old_analyzer.xml | 2 +- tests/integration/test_analyzer_compatibility/test.py | 6 ++++-- .../02998_analyzer_secret_args_tree_node.reference | 8 ++++---- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/tests/config/users.d/analyzer.xml b/tests/config/users.d/analyzer.xml index edba8b8578e..4b9764526fa 100644 --- a/tests/config/users.d/analyzer.xml +++ b/tests/config/users.d/analyzer.xml @@ -1,7 +1,7 @@ - 0 + 0 diff --git a/tests/integration/helpers/0_common_enable_old_analyzer.xml b/tests/integration/helpers/0_common_enable_old_analyzer.xml index edba8b8578e..4b9764526fa 100644 --- a/tests/integration/helpers/0_common_enable_old_analyzer.xml +++ b/tests/integration/helpers/0_common_enable_old_analyzer.xml @@ -1,7 +1,7 @@ - 0 + 0 diff --git a/tests/integration/test_analyzer_compatibility/test.py b/tests/integration/test_analyzer_compatibility/test.py index 2c840154eb5..6eeba1f1274 100644 --- a/tests/integration/test_analyzer_compatibility/test.py +++ b/tests/integration/test_analyzer_compatibility/test.py @@ -78,10 +78,12 @@ WHERE initial_query_id = '{query_id}';""" current.query("SYSTEM FLUSH LOGS") backward.query("SYSTEM FLUSH LOGS") + # The old version doesn't know about the alias. + # For this we will ask about the old experimental name. assert ( backward.query( """ -SELECT hostname() AS h, getSetting('enable_analyzer') +SELECT hostname() AS h, getSetting('allow_experimental_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) ORDER BY h;""" ) @@ -92,7 +94,7 @@ ORDER BY h;""" analyzer_enabled = backward.query( f""" SELECT -DISTINCT Settings['enable_analyzer'] +DISTINCT Settings['allow_experimental_analyzer'] FROM clusterAllReplicas('test_cluster_mixed', system.query_log) WHERE initial_query_id = '{query_id}';""" ) diff --git a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference index ccd9540cb49..67ef38093d4 100644 --- a/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference +++ b/tests/queries/0_stateless/02998_analyzer_secret_args_tree_node.reference @@ -7,7 +7,7 @@ QUERY id: 0 encrypt(\'aes-256-ofb\', [HIDDEN id: 3], [HIDDEN id: 2]) Nullable(String) PROJECTION LIST id: 1, nodes: 2 - CONSTANT id: 2, constant_value: \'\\n��&\', constant_value_type: Nullable(String) + CONSTANT id: 2, constant_value: \'\\n&\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -15,7 +15,7 @@ QUERY id: 0 CONSTANT id: 5, constant_value: \'aes-256-ofb\', constant_value_type: String CONSTANT id: 6, constant_value: [HIDDEN id: 1], constant_value_type: Nullable(String) CONSTANT id: 7, constant_value: [HIDDEN id: 2], constant_value_type: String - CONSTANT id: 8, constant_value: \'��\', constant_value_type: Nullable(String) + CONSTANT id: 8, constant_value: \'\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 9, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -33,7 +33,7 @@ QUERY id: 0 encrypt(\'aes-256-ofb\', _subquery_2, \'12345678901234567890123456789012\') Nullable(String) PROJECTION LIST id: 1, nodes: 2 - CONSTANT id: 2, constant_value: \'\\n��&\', constant_value_type: Nullable(String) + CONSTANT id: 2, constant_value: \'\\n&\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 3, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 JOIN TREE TABLE id: 10, table_name: system.one CONSTANT id: 11, constant_value: \'12345678901234567890123456789012\', constant_value_type: String - CONSTANT id: 12, constant_value: \'��\', constant_value_type: Nullable(String) + CONSTANT id: 12, constant_value: \'\', constant_value_type: Nullable(String) EXPRESSION FUNCTION id: 13, function_name: encrypt, function_type: ordinary, result_type: Nullable(String) ARGUMENTS From 330aae951706e4c3fd6ddada231d5d4bc23e37a0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Aug 2024 16:51:44 +0000 Subject: [PATCH 0873/1170] Even better test --- .../test_analyzer_compatibility/test.py | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_analyzer_compatibility/test.py b/tests/integration/test_analyzer_compatibility/test.py index 6eeba1f1274..505d1629cd2 100644 --- a/tests/integration/test_analyzer_compatibility/test.py +++ b/tests/integration/test_analyzer_compatibility/test.py @@ -1,4 +1,5 @@ import uuid +import time import pytest from helpers.cluster import ClickHouseCluster @@ -51,19 +52,19 @@ def test_two_new_versions(start_cluster): assert ( current.query( """ -SELECT hostname() AS h, getSetting('enable_analyzer') +SELECT hostname() AS h, getSetting('allow_experimental_analyzer') FROM clusterAllReplicas('test_cluster_mixed', system.one) ORDER BY h;""" ) == TSV([["backward", "true"], ["current", "true"]]) ) - # Should be enabled everywhere - analyzer_enabled = current.query( + # Should be enabled explicitly on the old instance. + analyzer_enabled = backward.query( f""" SELECT -DISTINCT Settings['enable_analyzer'] -FROM clusterAllReplicas('test_cluster_mixed', system.query_log) +DISTINCT Settings['allow_experimental_analyzer'] +FROM system.query_log WHERE initial_query_id = '{query_id}';""" ) @@ -100,3 +101,26 @@ WHERE initial_query_id = '{query_id}';""" ) assert TSV(analyzer_enabled) == TSV("0") + + # Only new version knows about the alias + # and it will send the old setting `allow_experimental_analyzer` + # to the remote server. + query_id = str(uuid.uuid4()) + current.query( + "SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables) SETTINGS enable_analyzer = 1;", + query_id=query_id, + ) + + current.query("SYSTEM FLUSH LOGS") + backward.query("SYSTEM FLUSH LOGS") + + # Should be disabled explicitly everywhere. + analyzer_enabled = current.query( + f""" +SELECT +DISTINCT Settings['allow_experimental_analyzer'] +FROM system.query_log +WHERE initial_query_id = '{query_id}';""" + ) + + assert TSV(analyzer_enabled) == TSV("1") From a2020224750f9861f1571d4aa8e139560b3a1dfc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Aug 2024 11:29:56 +0000 Subject: [PATCH 0874/1170] Fixed performance tests --- tests/performance/storage_join_direct_join.xml | 2 +- tests/performance/uniq_to_count.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 987500bb4f0..867108ac2b7 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -15,5 +15,5 @@ SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS - enable_analyzer=1 + allow_experimental_analyzer=1 diff --git a/tests/performance/uniq_to_count.xml b/tests/performance/uniq_to_count.xml index 57b0085d8fa..64e4cf1cc0d 100644 --- a/tests/performance/uniq_to_count.xml +++ b/tests/performance/uniq_to_count.xml @@ -3,6 +3,6 @@ select uniq(number) from (select number from numbers(1000000) group by number) - select uniq(number) from (select DISTINCT number from numbers(1000000)) SETTINGS enable_analyzer=1 - select uniq(number) from (select number from numbers(1000000) group by number) SETTINGS enable_analyzer=1 + select uniq(number) from (select DISTINCT number from numbers(1000000)) SETTINGS allow_experimental_analyzer=1 + select uniq(number) from (select number from numbers(1000000) group by number) SETTINGS allow_experimental_analyzer=1 From a9204c5da3179ca7c5e78fe537f4e57bf129dbab Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Aug 2024 11:34:57 +0000 Subject: [PATCH 0875/1170] Delete changes --- .../0_stateless/02995_baseline_24_7_2.tsv | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/tests/queries/0_stateless/02995_baseline_24_7_2.tsv b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv index d3a07ecb644..10b392f3e04 100644 --- a/tests/queries/0_stateless/02995_baseline_24_7_2.tsv +++ b/tests/queries/0_stateless/02995_baseline_24_7_2.tsv @@ -18,11 +18,7 @@ allow_distributed_ddl 1 allow_drop_detached 0 allow_execute_multiif_columnar 1 allow_experimental_alter_materialized_view_structure 1 -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv allow_experimental_analyzer 1 -======= -allow_experimental_analyzer 0 ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv allow_experimental_annoy_index 0 allow_experimental_bigint_types 1 allow_experimental_codecs 0 @@ -163,10 +159,7 @@ cloud_mode 0 cloud_mode_engine 1 cluster_for_parallel_replicas collect_hash_table_stats_during_aggregation 1 -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv collect_hash_table_stats_during_joins 1 -======= ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv column_names_for_schema_inference compatibility compatibility_ignore_auto_increment_in_create_table 0 @@ -336,13 +329,9 @@ format_regexp_escaping_rule Raw format_regexp_skip_unmatched 0 format_schema format_template_resultset -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv format_template_resultset_format format_template_row format_template_row_format -======= -format_template_row ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv format_template_rows_between_delimiter \n format_tsv_null_representation \\N formatdatetime_f_prints_single_zero 0 @@ -399,11 +388,8 @@ iceberg_engine_ignore_schema_evolution 0 idle_connection_timeout 3600 ignore_cold_parts_seconds 0 ignore_data_skipping_indices -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv ignore_drop_queries_probability 0 ignore_materialized_views_with_dropped_target_table 0 -======= ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv ignore_on_cluster_for_replicated_access_entities_queries 0 ignore_on_cluster_for_replicated_named_collections_queries 0 ignore_on_cluster_for_replicated_udf_queries 0 @@ -726,13 +712,8 @@ mutations_execute_subqueries_on_initiator 0 mutations_max_literal_size_to_replace 16384 mutations_sync 0 mysql_datatypes_support_level -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv mysql_map_fixed_string_to_text_in_show_columns 1 mysql_map_string_to_text_in_show_columns 1 -======= -mysql_map_fixed_string_to_text_in_show_columns 0 -mysql_map_string_to_text_in_show_columns 0 ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv mysql_max_rows_to_insert 65536 network_compression_method LZ4 network_zstd_compression_level 1 @@ -799,13 +780,9 @@ os_thread_priority 0 output_format_arrow_compression_method lz4_frame output_format_arrow_fixed_string_as_fixed_byte_array 1 output_format_arrow_low_cardinality_as_dictionary 0 -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv output_format_arrow_string_as_string 1 output_format_arrow_use_64_bit_indexes_for_dictionary 0 output_format_arrow_use_signed_indexes_for_dictionary 1 -======= -output_format_arrow_string_as_string 0 ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv output_format_avro_codec output_format_avro_rows_in_file 1 output_format_avro_string_column_pattern @@ -1068,10 +1045,7 @@ totals_mode after_having_exclusive trace_profile_events 0 transfer_overflow_mode throw transform_null_in 0 -<<<<<<< HEAD:tests/queries/0_stateless/02995_baseline_24_7_2.tsv traverse_shadow_remote_data_paths 0 -======= ->>>>>>> Fix tests:tests/queries/0_stateless/02995_baseline_23_12_1.tsv union_default_mode unknown_packet_in_send_data 0 update_insert_deduplication_token_in_dependent_materialized_views 0 From d998bf55f1a592df93f6c839dd647ff437962076 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Aug 2024 11:37:08 +0000 Subject: [PATCH 0876/1170] Fix build --- src/Core/SettingsChangesHistory.cpp | 263 +--------------------------- 1 file changed, 1 insertion(+), 262 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 893394adb61..2ff392fcb84 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,6 +78,7 @@ static std::initializer_list 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"}}}, }; From 011910a59409b3b2ec5430097d88c40e091c6b30 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 5 Aug 2024 15:28:43 +0000 Subject: [PATCH 0877/1170] Fixed the integration test --- ...allel_replicas_crash_after_refactoring.sql | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql diff --git a/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql new file mode 100644 index 00000000000..cae4fa0f0df --- /dev/null +++ b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql @@ -0,0 +1,33 @@ +-- Tags: disabled + +DROP TABLE IF EXISTS t1__fuzz_5; + +CREATE TABLE t1__fuzz_5 +( + `k` Int16, + `v` Nullable(UInt8) +) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity = 10; + +INSERT INTO t1__fuzz_5 SELECT + number, + number +FROM numbers(1000); + +INSERT INTO t1__fuzz_5 SELECT + number, + number +FROM numbers(1000, 1000); + +INSERT INTO t1__fuzz_5 SELECT + number, + number +FROM numbers(2000, 1000); + +SET receive_timeout = 10., receive_data_timeout_ms = 10000, allow_suspicious_low_cardinality_types = true, parallel_distributed_insert_select = 2, log_queries = true, table_function_remote_max_addresses = 200, max_execution_time = 10., max_memory_usage = 10000000000, log_comment = '/workspace/ch/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql', send_logs_level = 'warning', prefer_localhost_replica = false, allow_introspection_functions = true, use_parallel_replicas = 257, max_parallel_replicas = 65535, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0., parallel_replicas_for_non_replicated_merge_tree = true; + +SELECT max(k) IGNORE NULLS FROM t1__fuzz_5 WITH TOTALS SETTINGS use_parallel_replicas = 257, max_parallel_replicas = 65535, prefer_localhost_replica = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0; + +DROP TABLE IF EXISTS t1__fuzz_5; From e4c71aec4dfa40d7e9b75c626b0229a33c59f315 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 5 Aug 2024 16:12:25 +0000 Subject: [PATCH 0878/1170] Update version_date.tsv and changelogs after v24.5.5.78-stable --- docs/changelogs/v24.5.5.78-stable.md | 55 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 56 insertions(+) create mode 100644 docs/changelogs/v24.5.5.78-stable.md diff --git a/docs/changelogs/v24.5.5.78-stable.md b/docs/changelogs/v24.5.5.78-stable.md new file mode 100644 index 00000000000..415ea165101 --- /dev/null +++ b/docs/changelogs/v24.5.5.78-stable.md @@ -0,0 +1,55 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.5.78-stable (0138248cb62) FIXME as compared to v24.5.4.49-stable (63b760955a0) + +#### Improvement +* Backported in [#66768](https://github.com/ClickHouse/ClickHouse/issues/66768): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#66884](https://github.com/ClickHouse/ClickHouse/issues/66884): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66691](https://github.com/ClickHouse/ClickHouse/issues/66691): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67814](https://github.com/ClickHouse/ClickHouse/issues/67814): 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 [#67501](https://github.com/ClickHouse/ClickHouse/issues/67501): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67850](https://github.com/ClickHouse/ClickHouse/issues/67850): 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)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65350](https://github.com/ClickHouse/ClickHouse/issues/65350): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65621](https://github.com/ClickHouse/ClickHouse/issues/65621): Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65933](https://github.com/ClickHouse/ClickHouse/issues/65933): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#66301](https://github.com/ClickHouse/ClickHouse/issues/66301): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66328](https://github.com/ClickHouse/ClickHouse/issues/66328): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66155](https://github.com/ClickHouse/ClickHouse/issues/66155): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66454](https://github.com/ClickHouse/ClickHouse/issues/66454): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66226](https://github.com/ClickHouse/ClickHouse/issues/66226): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66680](https://github.com/ClickHouse/ClickHouse/issues/66680): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66604](https://github.com/ClickHouse/ClickHouse/issues/66604): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66360](https://github.com/ClickHouse/ClickHouse/issues/66360): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66972](https://github.com/ClickHouse/ClickHouse/issues/66972): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66969](https://github.com/ClickHouse/ClickHouse/issues/66969): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66720](https://github.com/ClickHouse/ClickHouse/issues/66720): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66951](https://github.com/ClickHouse/ClickHouse/issues/66951): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66757](https://github.com/ClickHouse/ClickHouse/issues/66757): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66948](https://github.com/ClickHouse/ClickHouse/issues/66948): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67633](https://github.com/ClickHouse/ClickHouse/issues/67633): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67481](https://github.com/ClickHouse/ClickHouse/issues/67481): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67197](https://github.com/ClickHouse/ClickHouse/issues/67197): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67379](https://github.com/ClickHouse/ClickHouse/issues/67379): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67576](https://github.com/ClickHouse/ClickHouse/issues/67576): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66387](https://github.com/ClickHouse/ClickHouse/issues/66387): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66426](https://github.com/ClickHouse/ClickHouse/issues/66426): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66544](https://github.com/ClickHouse/ClickHouse/issues/66544): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66859](https://github.com/ClickHouse/ClickHouse/issues/66859): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66875](https://github.com/ClickHouse/ClickHouse/issues/66875): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67059](https://github.com/ClickHouse/ClickHouse/issues/67059): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66945](https://github.com/ClickHouse/ClickHouse/issues/66945): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67252](https://github.com/ClickHouse/ClickHouse/issues/67252): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67412](https://github.com/ClickHouse/ClickHouse/issues/67412): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cb6b8f588da..75c10fa67b8 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.5.78-stable 2024-08-05 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From 4e8d11c48e9ac0107940d730ccf8d35b7ac3573e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 5 Aug 2024 16:46:42 +0000 Subject: [PATCH 0879/1170] Add fuzzers to 03208_buffer_over_distributed_type_mismatch --- ...208_buffer_over_distributed_type_mismatch.sql | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql index 5a7c89074cf..333c445403d 100644 --- a/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql +++ b/tests/queries/0_stateless/03208_buffer_over_distributed_type_mismatch.sql @@ -58,3 +58,19 @@ SELECT amount FROM realtimebuff__fuzz_19 t1 JOIN realtimebuff__fuzz_19 t2 ON t1.amount = t2.amount JOIN realtimebuff__fuzz_19 t3 ON t1.amount = t3.amount ; -- { serverError NOT_IMPLEMENTED,AMBIGUOUS_COLUMN_NAME } + + +-- fuzzers: + +SELECT + toLowCardinality(1) + materialize(toLowCardinality(2)) +FROM realtimebuff__fuzz_19 +GROUP BY toLowCardinality(1) +FORMAT Null +; + +SELECT intDivOrZero(intDivOrZero(toLowCardinality(-128), toLowCardinality(-1)) = 0, materialize(toLowCardinality(4))) +FROM realtimebuff__fuzz_19 GROUP BY materialize(toLowCardinality(-127)), intDivOrZero(0, 0) = toLowCardinality(toLowCardinality(0)) +WITH TOTALS ORDER BY ALL DESC NULLS FIRST +FORMAT Null +; From 05395ac7bd93949d07f163bfc6cabf9c3e85adc5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Aug 2024 17:01:15 +0000 Subject: [PATCH 0880/1170] Un-flake 02524_fuzz_and_fuss_2 (hopefully) https://s3.amazonaws.com/clickhouse-test-reports/0/3221b651da16cd868350a1aff022ba71a1a15f3c/stateless_tests__tsan__s3_storage__[2_5].html https://s3.amazonaws.com/clickhouse-test-reports/0/1fde5b7bfa1b1a6a0d67258be5e7ef855b730559/stateless_tests__tsan__s3_storage__[2_4].html --- tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql index 7b49378d4da..a38fb0bd471 100644 --- a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql +++ b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql @@ -9,6 +9,6 @@ ENGINE = Memory; INSERT INTO data_a_02187 SELECT * FROM system.one -SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', distributed_foreground_insert = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; +SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', distributed_foreground_insert = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '9', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; DROP TABLE data_a_02187; From c8805fbcedcce226a82d78cff79d6afbdddca0a7 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Mon, 5 Aug 2024 17:41:47 +0000 Subject: [PATCH 0881/1170] Misc fixes --- docker/test/fasttest/run.sh | 2 +- src/Functions/bitSlice.cpp | 4 ++-- src/Storages/MergeTree/KeyCondition.h | 3 ++- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 26283afc86a..394d31addb1 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -41,7 +41,7 @@ export FASTTEST_WORKSPACE export FASTTEST_SOURCE export FASTTEST_BUILD export FASTTEST_DATA -export FASTTEST_OUT +export FASTTEST_OUTPUT export PATH function ccache_status diff --git a/src/Functions/bitSlice.cpp b/src/Functions/bitSlice.cpp index f24473351ae..908c534b228 100644 --- a/src/Functions/bitSlice.cpp +++ b/src/Functions/bitSlice.cpp @@ -42,11 +42,11 @@ public: { FunctionArgumentDescriptors mandatory_args{ {"s", static_cast(&isStringOrFixedString), nullptr, "String"}, - {"offset", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + {"offset", static_cast(&isNativeNumber), nullptr, "(U)Int8/16/32/64 or Float"}, }; FunctionArgumentDescriptors optional_args{ - {"length", static_cast(&isNativeNumber), nullptr, "(U)Int8, (U)Int16, (U)Int32, (U)Int64 or Float"}, + {"length", static_cast(&isNativeNumber), nullptr, "(U)Int8/16/32/64 or Float"}, }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index e9343ec08ea..8bbb86aba43 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -6,6 +6,8 @@ #include #include +#include + #include #include @@ -14,7 +16,6 @@ #include #include -#include "DataTypes/Serializations/ISerialization.h" namespace DB From 457686c379cb884a579b86cbfd6a9abb3ec16d1a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 10 Jul 2024 19:39:58 +0200 Subject: [PATCH 0882/1170] Simplify StorageDistributed ctors Signed-off-by: Azat Khuzhin --- src/Storages/StorageDistributed.cpp | 32 ---------------------- src/Storages/StorageDistributed.h | 14 ---------- src/TableFunctions/TableFunctionRemote.cpp | 19 ++----------- 3 files changed, 3 insertions(+), 62 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3e38ddf830a..eb9483127fc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -373,38 +373,6 @@ StorageDistributed::StorageDistributed( } -StorageDistributed::StorageDistributed( - const StorageID & id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ASTPtr remote_table_function_ptr_, - const String & cluster_name_, - ContextPtr context_, - const ASTPtr & sharding_key_, - const String & storage_policy_name_, - const String & relative_data_path_, - const DistributedSettings & distributed_settings_, - LoadingStrictnessLevel mode, - ClusterPtr owned_cluster_) - : StorageDistributed( - id_, - columns_, - constraints_, - String{}, - String{}, - String{}, - cluster_name_, - context_, - sharding_key_, - storage_policy_name_, - relative_data_path_, - distributed_settings_, - mode, - std::move(owned_cluster_), - remote_table_function_ptr_) -{ -} - QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ContextPtr local_context, QueryProcessingStage::Enum to_stage, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 85a8de86953..a9e10dd14ba 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -63,20 +63,6 @@ public: ClusterPtr owned_cluster_ = {}, ASTPtr remote_table_function_ptr_ = {}); - StorageDistributed( - const StorageID & id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ASTPtr remote_table_function_ptr_, - const String & cluster_name_, - ContextPtr context_, - const ASTPtr & sharding_key_, - const String & storage_policy_name_, - const String & relative_data_path_, - const DistributedSettings & distributed_settings_, - LoadingStrictnessLevel mode, - ClusterPtr owned_cluster_ = {}); - ~StorageDistributed() override; std::string getName() const override { return "Distributed"; } diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index e60c31b2d77..5d136993018 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -306,21 +306,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con cached_columns = getActualTableStructure(context, is_insert_query); assert(cluster); - StoragePtr res = remote_table_function_ptr - ? std::make_shared( - StorageID(getDatabaseName(), table_name), - cached_columns, - ConstraintsDescription{}, - remote_table_function_ptr, - String{}, - context, - sharding_key, - String{}, - String{}, - DistributedSettings{}, - LoadingStrictnessLevel::CREATE, - cluster) - : std::make_shared( + StoragePtr res = std::make_shared( StorageID(getDatabaseName(), table_name), cached_columns, ConstraintsDescription{}, @@ -334,7 +320,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con String{}, DistributedSettings{}, LoadingStrictnessLevel::CREATE, - cluster); + cluster, + remote_table_function_ptr); res->startup(); return res; From 9d7710684b91a9a5b0d7c2a34b709232979ebe65 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 10 Jul 2024 19:57:09 +0200 Subject: [PATCH 0883/1170] tests/test_distributed_inter_server_secret: get_query_user_info return list Signed-off-by: Azat Khuzhin --- .../test_distributed_inter_server_secret/test.py | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 7ecb2cda257..9389b901a30 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -116,10 +116,10 @@ def start_cluster(): cluster.shutdown() -# @return -- [user, initial_user] +# @return -- [[user, initial_user]] def get_query_user_info(node, query_pattern): node.query("SYSTEM FLUSH LOGS") - return ( + lines = ( node.query( """ SELECT user, initial_user @@ -133,8 +133,10 @@ def get_query_user_info(node, query_pattern): ) ) .strip() - .split("\t") + .split("\n") ) + lines = map(lambda x: x.split("\t"), lines) + return list(lines) # @return -- [user, initial_user] @@ -331,19 +333,19 @@ def test_secure_disagree_insert(): def test_user_insecure_cluster(user, password): id_ = "query-dist_insecure-" + user n1.query(f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password) - assert get_query_user_info(n1, id_) == [ + assert get_query_user_info(n1, id_)[0] == [ user, user, ] # due to prefer_localhost_replica - assert get_query_user_info(n2, id_) == ["default", user] + assert get_query_user_info(n2, id_)[0] == ["default", user] @users def test_user_secure_cluster(user, password): id_ = "query-dist_secure-" + user n1.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) - assert get_query_user_info(n1, id_) == [user, user] - assert get_query_user_info(n2, id_) == [user, user] + assert get_query_user_info(n1, id_)[0] == [user, user] + assert get_query_user_info(n2, id_)[0] == [user, user] @users From 0dccc34a2004b075e290ebfd5ecc9fea3077a205 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 10 Jul 2024 19:23:34 +0200 Subject: [PATCH 0884/1170] Fix cluster() for inter-server secret (preserve initial user as before) The behaviour of cluster() with inter-server secret had been changed in #63013, after it started to use "default" user, and this introduces a regression. The intention of that patch was to adjust only remote(), since it only it accept custom user, which should be ignored. Fixes: https://github.com/ClickHouse/ClickHouse/issues/66287 Fixes: https://github.com/ClickHouse/ClickHouse/issues/66352 Signed-off-by: Azat Khuzhin --- src/Storages/StorageDistributed.cpp | 6 ++++-- src/Storages/StorageDistributed.h | 5 ++++- src/TableFunctions/TableFunctionRemote.cpp | 3 ++- .../test_distributed_inter_server_secret/test.py | 15 ++++++++++++++- 4 files changed, 24 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index eb9483127fc..e146e95f89f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -308,7 +308,8 @@ StorageDistributed::StorageDistributed( const DistributedSettings & distributed_settings_, LoadingStrictnessLevel mode, ClusterPtr owned_cluster_, - ASTPtr remote_table_function_ptr_) + ASTPtr remote_table_function_ptr_, + bool is_remote_function_) : IStorage(id_) , WithContext(context_->getGlobalContext()) , remote_database(remote_database_) @@ -322,6 +323,7 @@ StorageDistributed::StorageDistributed( , relative_data_path(relative_data_path_) , distributed_settings(distributed_settings_) , rng(randomSeed()) + , is_remote_function(is_remote_function_) { if (!distributed_settings.flush_on_detach && distributed_settings.background_insert_batch) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Settings flush_on_detach=0 and background_insert_batch=1 are incompatible"); @@ -869,7 +871,7 @@ void StorageDistributed::read( sharding_key_column_name, distributed_settings, shard_filter_generator, - /* is_remote_function= */ static_cast(owned_cluster)); + is_remote_function); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index a9e10dd14ba..8a5585e9fd0 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -61,7 +61,8 @@ public: const DistributedSettings & distributed_settings_, LoadingStrictnessLevel mode, ClusterPtr owned_cluster_ = {}, - ASTPtr remote_table_function_ptr_ = {}); + ASTPtr remote_table_function_ptr_ = {}, + bool is_remote_function_ = false); ~StorageDistributed() override; @@ -273,6 +274,8 @@ private: // For random shard index generation mutable std::mutex rng_mutex; pcg64 rng; + + bool is_remote_function; }; } diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 5d136993018..8a877ff0802 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -321,7 +321,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con DistributedSettings{}, LoadingStrictnessLevel::CREATE, cluster, - remote_table_function_ptr); + remote_table_function_ptr, + !is_cluster_function); res->startup(); return res; diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 9389b901a30..36d7e044f1c 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -418,7 +418,7 @@ def test_per_user_protocol_settings_secure_cluster(user, password): ) -def test_secure_cluster_distributed_over_distributed_different_users(): +def test_secure_cluster_distributed_over_distributed_different_users_remote(): # This works because we will have initial_user='default' n1.query( "SELECT * FROM remote('n1', currentDatabase(), dist_secure)", user="new_user" @@ -433,3 +433,16 @@ def test_secure_cluster_distributed_over_distributed_different_users(): # and stuff). with pytest.raises(QueryRuntimeException): n1.query("SELECT * FROM dist_over_dist_secure", user="new_user") + + +def test_secure_cluster_distributed_over_distributed_different_users_cluster(): + id_ = "cluster-user" + n1.query( + f"SELECT *, '{id_}' FROM cluster(secure, currentDatabase(), dist_secure)", + user="nopass", + settings={ + "prefer_localhost_replica": 0, + }, + ) + assert get_query_user_info(n1, id_) == [["nopass", "nopass"]] * 4 + assert get_query_user_info(n2, id_) == [["nopass", "nopass"]] * 3 From 3593f740a8793339c5657a0bcc6f785e1198e510 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 12:16:23 +0200 Subject: [PATCH 0885/1170] Revert "Remove bad tests @azat" (reverts #66823) This reverts commit 0c2c027af63fcbababffbe3a39ed2631884e1938. Signed-off-by: Azat Khuzhin --- ...2_part_log_rmt_fetch_merge_error.reference | 10 +++++ .../03002_part_log_rmt_fetch_merge_error.sql | 35 ++++++++++++++++ ..._part_log_rmt_fetch_mutate_error.reference | 10 +++++ .../03002_part_log_rmt_fetch_mutate_error.sql | 41 +++++++++++++++++++ 4 files changed, 96 insertions(+) create mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.reference create mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql create mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.reference create mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.reference b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.reference new file mode 100644 index 00000000000..b19d389d8d0 --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.reference @@ -0,0 +1,10 @@ +before +rmt_master NewPart 0 1 +rmt_master MergeParts 0 1 +rmt_slave MergeParts 1 0 +rmt_slave DownloadPart 0 1 +after +rmt_master NewPart 0 1 +rmt_master MergeParts 0 1 +rmt_slave MergeParts 1 0 +rmt_slave DownloadPart 0 2 diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql new file mode 100644 index 00000000000..548a8e5570a --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql @@ -0,0 +1,35 @@ +-- Tags: no-replicated-database, no-parallel, no-shared-merge-tree +-- SMT: The merge process is completely different from RMT + +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; +-- 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; + +insert into rmt_master values (1); + +system sync replica rmt_master; +system sync replica rmt_slave; +system stop replicated sends rmt_master; +optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1; + +select sleep(3) format Null; + +system flush logs; +select 'before'; +select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + +system start replicated sends rmt_master; +-- sleep few seconds to try rmt_slave to fetch the part and reflect this error +-- in system.part_log +select sleep(3) format Null; +system sync replica rmt_slave; + +system flush logs; +select 'after'; +select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + +drop table rmt_master; +drop table rmt_slave; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.reference b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.reference new file mode 100644 index 00000000000..aac9e7527d1 --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.reference @@ -0,0 +1,10 @@ +before +rmt_master NewPart 0 1 +rmt_master MutatePart 0 1 +rmt_slave DownloadPart 0 1 +rmt_slave MutatePart 1 0 +after +rmt_master NewPart 0 1 +rmt_master MutatePart 0 1 +rmt_slave DownloadPart 0 2 +rmt_slave MutatePart 1 0 diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql new file mode 100644 index 00000000000..d8b5ebb3148 --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql @@ -0,0 +1,41 @@ +-- Tags: no-replicated-database, no-parallel, no-shared-merge-tree +-- SMT: The merge process is completely different from RMT + +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 tuple() settings always_fetch_merged_part=0, old_parts_lifetime=600; +-- prefer_fetch_merged_part_*_threshold=0, consider this table as a "slave" +create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by tuple() settings prefer_fetch_merged_part_time_threshold=0, prefer_fetch_merged_part_size_threshold=0, old_parts_lifetime=600; + +insert into rmt_master values (1); + +system sync replica rmt_master; +system sync replica rmt_slave; +system stop replicated sends rmt_master; +system stop pulling replication log rmt_slave; +alter table rmt_master update key=key+100 where 1 settings alter_sync=1; + +-- first we need to make the rmt_master execute mutation so that it will have +-- the part, and rmt_slave will consider it instead of performing mutation on +-- it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored +select sleep(3) format Null; +system start pulling replication log rmt_slave; +-- and sleep few more seconds to try rmt_slave to fetch the part and reflect +-- this error in system.part_log +select sleep(3) format Null; + +system flush logs; +select 'before'; +select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + +system start replicated sends rmt_master; +select sleep(3) format Null; +system sync replica rmt_slave; + +system flush logs; +select 'after'; +select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + +drop table rmt_master; +drop table rmt_slave; From 50aebcfc28c5fa98dca92d92c5233adea92801f4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 13:18:53 +0200 Subject: [PATCH 0886/1170] Rewrite 03002_part_log_rmt_fetch_*_error tests from sql to sh with retries Except for this patch should fix the flakiness of this test it also reduces the execution time from 15.4s to 3.3s (5x!) Signed-off-by: Azat Khuzhin --- .../03002_part_log_rmt_fetch_merge_error.sh | 54 ++++++++++++++++ .../03002_part_log_rmt_fetch_merge_error.sql | 35 ---------- .../03002_part_log_rmt_fetch_mutate_error.sh | 64 +++++++++++++++++++ .../03002_part_log_rmt_fetch_mutate_error.sql | 41 ------------ 4 files changed, 118 insertions(+), 76 deletions(-) create mode 100755 tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh delete mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql create mode 100755 tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh delete mode 100644 tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql 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 new file mode 100755 index 00000000000..25d946b325d --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# SMT: The merge process is completely different from RMT + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +function wait_until() +{ + local q=$1 && shift + while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + # too frequent FLUSH LOGS is too costly + sleep 2 + done +} + +$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; + -- 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; + + insert into rmt_master values (1); + + system sync replica rmt_master; + system sync replica rmt_slave; + system stop replicated sends rmt_master; + optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1; +" + +$CLICKHOUSE_CLIENT -nm -q " + system flush logs; + select 'before'; + select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + + system start replicated sends rmt_master; +" +# wait until rmt_slave will fetch the part and reflect this error in system.part_log +wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +$CLICKHOUSE_CLIENT -nm -q " + system sync replica rmt_slave; + + system flush logs; + select 'after'; + select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + + drop table rmt_master; + drop table rmt_slave; +" diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql deleted file mode 100644 index 548a8e5570a..00000000000 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sql +++ /dev/null @@ -1,35 +0,0 @@ --- Tags: no-replicated-database, no-parallel, no-shared-merge-tree --- SMT: The merge process is completely different from RMT - -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; --- 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; - -insert into rmt_master values (1); - -system sync replica rmt_master; -system sync replica rmt_slave; -system stop replicated sends rmt_master; -optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1; - -select sleep(3) format Null; - -system flush logs; -select 'before'; -select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; - -system start replicated sends rmt_master; --- sleep few seconds to try rmt_slave to fetch the part and reflect this error --- in system.part_log -select sleep(3) format Null; -system sync replica rmt_slave; - -system flush logs; -select 'after'; -select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; - -drop table rmt_master; -drop table rmt_slave; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh new file mode 100755 index 00000000000..cc8f53aafb9 --- /dev/null +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash +# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# SMT: The merge process is completely different from RMT + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# shellcheck source=./mergetree_mutations.lib +. "$CUR_DIR"/mergetree_mutations.lib + +set -e + +function wait_until() +{ + local q=$1 && shift + while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + # too frequent FLUSH LOGS is too costly + sleep 2 + done +} + +$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 tuple() settings always_fetch_merged_part=0, old_parts_lifetime=600; + -- prefer_fetch_merged_part_*_threshold=0, consider this table as a 'slave' + create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by tuple() settings prefer_fetch_merged_part_time_threshold=0, prefer_fetch_merged_part_size_threshold=0, old_parts_lifetime=600; + + insert into rmt_master values (1); + + system sync replica rmt_master; + system sync replica rmt_slave; + system stop replicated sends rmt_master; + system stop pulling replication log rmt_slave; + alter table rmt_master update key=key+100 where 1 settings alter_sync=1; +" + +# first we need to make the rmt_master execute mutation so that it will have +# the part, and rmt_slave will consider it instead of performing mutation on +# it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored +wait_for_mutation rmt_master 0000000000 +$CLICKHOUSE_CLIENT -nm -q "system start pulling replication log rmt_slave" +# and wait until rmt_slave to fetch the part and reflect this error in system.part_log +wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +$CLICKHOUSE_CLIENT -nm -q " + system flush logs; + select 'before'; + select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + + system start replicated sends rmt_master; +" +wait_for_mutation rmt_slave 0000000000 +$CLICKHOUSE_CLIENT -nm -q " + system sync replica rmt_slave; + + system flush logs; + select 'after'; + select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; + + drop table rmt_master; + drop table rmt_slave; +" diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql deleted file mode 100644 index d8b5ebb3148..00000000000 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sql +++ /dev/null @@ -1,41 +0,0 @@ --- Tags: no-replicated-database, no-parallel, no-shared-merge-tree --- SMT: The merge process is completely different from RMT - -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 tuple() settings always_fetch_merged_part=0, old_parts_lifetime=600; --- prefer_fetch_merged_part_*_threshold=0, consider this table as a "slave" -create table rmt_slave (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', 'slave') order by tuple() settings prefer_fetch_merged_part_time_threshold=0, prefer_fetch_merged_part_size_threshold=0, old_parts_lifetime=600; - -insert into rmt_master values (1); - -system sync replica rmt_master; -system sync replica rmt_slave; -system stop replicated sends rmt_master; -system stop pulling replication log rmt_slave; -alter table rmt_master update key=key+100 where 1 settings alter_sync=1; - --- first we need to make the rmt_master execute mutation so that it will have --- the part, and rmt_slave will consider it instead of performing mutation on --- it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored -select sleep(3) format Null; -system start pulling replication log rmt_slave; --- and sleep few more seconds to try rmt_slave to fetch the part and reflect --- this error in system.part_log -select sleep(3) format Null; - -system flush logs; -select 'before'; -select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; - -system start replicated sends rmt_master; -select sleep(3) format Null; -system sync replica rmt_slave; - -system flush logs; -select 'after'; -select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; - -drop table rmt_master; -drop table rmt_slave; From 0310f52d3a73fa3df61ef218ce643e448edfd012 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Aug 2024 10:49:09 +0200 Subject: [PATCH 0887/1170] Fix 01811_storage_buffer_flush_parameters flakiness Signed-off-by: Azat Khuzhin --- .../01811_storage_buffer_flush_parameters.sh | 47 +++++++++++++++++++ .../01811_storage_buffer_flush_parameters.sql | 22 --------- 2 files changed, 47 insertions(+), 22 deletions(-) create mode 100755 tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh delete mode 100644 tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sql diff --git a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh new file mode 100755 index 00000000000..6a5949741ab --- /dev/null +++ b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function wait_with_limit() +{ + local limit=$1 && shift + local expr=$1 && shift + + for ((i = 0; i < limit; ++i)); do + if eval "$expr"; then + break + fi + sleep 1 + done +} + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data_01811; + drop table if exists buffer_01811; + + + create table data_01811 (key Int) Engine=Memory(); + /* Buffer with flush_rows=1000 */ + create table buffer_01811 (key Int) Engine=Buffer(currentDatabase(), data_01811, + /* num_layers= */ 1, + /* min_time= */ 1, /* max_time= */ 86400, + /* min_rows= */ 1e9, /* max_rows= */ 1e6, + /* min_bytes= */ 0, /* max_bytes= */ 4e6, + /* flush_time= */ 86400, /* flush_rows= */ 10, /* flush_bytes= */0 + ); + + insert into buffer_01811 select * from numbers(10); + insert into buffer_01811 select * from numbers(10); +" + +# wait for background buffer flush +wait_with_limit 30 '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01811") -gt 0 ]]' + +$CLICKHOUSE_CLIENT -nm -q "select count() from data_01811" + +$CLICKHOUSE_CLIENT -nm -q " + drop table buffer_01811; + drop table data_01811; +" diff --git a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sql b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sql deleted file mode 100644 index dac68ad4ae8..00000000000 --- a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sql +++ /dev/null @@ -1,22 +0,0 @@ -drop table if exists data_01811; -drop table if exists buffer_01811; - -create table data_01811 (key Int) Engine=Memory(); -/* Buffer with flush_rows=1000 */ -create table buffer_01811 (key Int) Engine=Buffer(currentDatabase(), data_01811, - /* num_layers= */ 1, - /* min_time= */ 1, /* max_time= */ 86400, - /* min_rows= */ 1e9, /* max_rows= */ 1e6, - /* min_bytes= */ 0, /* max_bytes= */ 4e6, - /* flush_time= */ 86400, /* flush_rows= */ 10, /* flush_bytes= */0 -); - -insert into buffer_01811 select * from numbers(10); -insert into buffer_01811 select * from numbers(10); - --- wait for background buffer flush -select sleep(3) format Null; -select count() from data_01811; - -drop table buffer_01811; -drop table data_01811; From 618789d1960182ff91d614545a16d1332768e008 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Jun 2024 20:42:27 +0200 Subject: [PATCH 0888/1170] Do not pass logs for keeper explicitly in the image to allow overriding Right now, if you will start keeper like this: $ cat /tmp/local.yaml --- logger: log: "@remove": remove errorlog: "@remove": remove console: 1 $ docker run --name keeper -v /tmp/local.yaml:/etc/clickhouse-keeper/conf.d/local.yaml --rm -it clickhouse/clickhouse-keeper This will still not work, because the --log-file/--errorlog-file passed explicitly. So this patch removes this in case config is valid, but, if keeper is launched with embedded config, this had been left as-is, to keep previous behavior. Signed-off-by: Azat Khuzhin --- docker/keeper/entrypoint.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index 1390ad9ce74..68bd0ef9d87 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -40,8 +40,6 @@ fi DATA_DIR="${CLICKHOUSE_DATA_DIR:-/var/lib/clickhouse}" LOG_DIR="${LOG_DIR:-/var/log/clickhouse-keeper}" -LOG_PATH="${LOG_DIR}/clickhouse-keeper.log" -ERROR_LOG_PATH="${LOG_DIR}/clickhouse-keeper.err.log" COORDINATION_DIR="${DATA_DIR}/coordination" COORDINATION_LOG_DIR="${DATA_DIR}/coordination/log" COORDINATION_SNAPSHOT_DIR="${DATA_DIR}/coordination/snapshots" @@ -84,7 +82,7 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then # There is a config file. It is already tested with gosu (if it is readably by keeper user) if [ -f "$KEEPER_CONFIG" ]; then - exec $gosu /usr/bin/clickhouse-keeper --config-file="$KEEPER_CONFIG" --log-file="$LOG_PATH" --errorlog-file="$ERROR_LOG_PATH" "$@" + exec $gosu /usr/bin/clickhouse-keeper --config-file="$KEEPER_CONFIG" "$@" fi # There is no config file. Will use embedded one From 205303c2868645e2aebf76f9b49ab77960579c56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 5 Aug 2024 20:37:43 +0200 Subject: [PATCH 0889/1170] Disable more bad tests --- .../Nodes/tests/gtest_resource_class_fair.cpp | 12 +++++++----- .../Nodes/tests/gtest_resource_class_priority.cpp | 10 ++++++---- .../Nodes/tests/gtest_throttler_constraint.cpp | 2 +- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp index 4f0e8c80734..16cce309c2a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp @@ -8,7 +8,9 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerFairPolicy, Factory) +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerFairPolicy, Factory) { ResourceTest t; @@ -17,7 +19,7 @@ TEST(SchedulerFairPolicy, Factory) EXPECT_TRUE(dynamic_cast(fair.get()) != nullptr); } -TEST(SchedulerFairPolicy, FairnessWeights) +TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) { ResourceTest t; @@ -41,7 +43,7 @@ TEST(SchedulerFairPolicy, FairnessWeights) t.consumed("B", 20); } -TEST(SchedulerFairPolicy, Activation) +TEST(DISABLED_SchedulerFairPolicy, Activation) { ResourceTest t; @@ -77,7 +79,7 @@ TEST(SchedulerFairPolicy, Activation) t.consumed("B", 10); } -TEST(SchedulerFairPolicy, FairnessMaxMin) +TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) { ResourceTest t; @@ -101,7 +103,7 @@ TEST(SchedulerFairPolicy, FairnessMaxMin) t.consumed("A", 20); } -TEST(SchedulerFairPolicy, HierarchicalFairness) +TEST(DISABLED_SchedulerFairPolicy, HierarchicalFairness) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp index a447b7f6780..d3d38aae048 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp @@ -8,7 +8,9 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerPriorityPolicy, Factory) +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerPriorityPolicy, Factory) { ResourceTest t; @@ -17,7 +19,7 @@ TEST(SchedulerPriorityPolicy, Factory) EXPECT_TRUE(dynamic_cast(prio.get()) != nullptr); } -TEST(SchedulerPriorityPolicy, Priorities) +TEST(DISABLED_SchedulerPriorityPolicy, Priorities) { ResourceTest t; @@ -51,7 +53,7 @@ TEST(SchedulerPriorityPolicy, Priorities) t.consumed("C", 0); } -TEST(SchedulerPriorityPolicy, Activation) +TEST(DISABLED_SchedulerPriorityPolicy, Activation) { ResourceTest t; @@ -92,7 +94,7 @@ TEST(SchedulerPriorityPolicy, Activation) t.consumed("C", 0); } -TEST(SchedulerPriorityPolicy, SinglePriority) +TEST(DISABLED_SchedulerPriorityPolicy, SinglePriority) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 363e286c91c..2bc24cdb292 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -10,7 +10,7 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Test disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) { From 529f21c6b8dac22235b92c8a443abfb4b95c0f52 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 25 Jul 2024 11:32:59 +0200 Subject: [PATCH 0890/1170] CI: CheckReadyForMerge fixes --- .github/workflows/backport_branches.yml | 7 ++++++- .github/workflows/jepsen.yml | 1 + .github/workflows/merge_queue.yml | 7 ++++++- .github/workflows/nightly.yml | 1 + .github/workflows/pull_request.yml | 6 +++++- .github/workflows/release_branches.yml | 4 +--- tests/ci/ci_config.py | 28 +++++++++++++++++++++++++ tests/ci/ci_utils.py | 9 ++++---- tests/ci/merge_pr.py | 24 ++++++++++++--------- 9 files changed, 67 insertions(+), 20 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 322946ac77b..23744dc7f8f 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -260,13 +260,18 @@ jobs: - name: Finish label if: ${{ !failure() }} run: | + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ toJson(needs) }} + EOF cd "$GITHUB_WORKSPACE/tests/ci" # update mergeable check - python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} + python3 merge_pr.py --set-ci-status # update overall ci report python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} python3 merge_pr.py - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat > "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 035ba2e5b98..2280b1a7305 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -64,6 +64,7 @@ jobs: - name: Check out repository code uses: ClickHouse/checkout@v1 - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index 64083668719..629cf79770e 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -103,9 +103,14 @@ jobs: - name: Check and set merge status if: ${{ needs.StyleCheck.result == 'success' }} run: | + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ toJson(needs) }} + EOF cd "$GITHUB_WORKSPACE/tests/ci" - python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} + python3 merge_pr.py --set-ci-status - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat > "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index ea9c125db70..36fea39686f 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -52,6 +52,7 @@ jobs: - name: Check out repository code uses: ClickHouse/checkout@v1 - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat > "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 071f0f1e20a..dbc740ebc1b 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -170,7 +170,11 @@ jobs: if: ${{ needs.StyleCheck.result == 'success' }} run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} + export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' + ${{ toJson(needs) }} + EOF + python3 merge_pr.py --set-ci-status - name: Check Workflow results uses: ./.github/actions/check_workflow with: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index b79208b03a6..a5cd6321e8c 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -481,12 +481,10 @@ jobs: - name: Finish label if: ${{ !failure() }} run: | - cd "$GITHUB_WORKSPACE/tests/ci" - # update mergeable check - python3 merge_pr.py --set-ci-status --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} # update overall ci report python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} - name: Check Workflow results + if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" cat > "$WORKFLOW_RESULT_FILE" << 'EOF' diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c031ca9b805..d34405e7692 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -687,6 +687,34 @@ class CI: assert res, f"not a build [{build_name}] or invalid JobConfig" return res + @classmethod + def is_workflow_ok(cls) -> bool: + # TODO: temporary method to make Mergeable check working + res = cls.GH.get_workflow_results() + if not res: + print("ERROR: no workflow results found") + return False + for workflow_job, workflow_data in res.items(): + status = workflow_data["result"] + if status in ( + cls.GH.ActionStatuses.SUCCESS, + cls.GH.ActionStatuses.SKIPPED, + ): + print(f"Workflow status for [{workflow_job}] is [{status}] - continue") + elif status in (cls.GH.ActionStatuses.FAILURE,): + if workflow_job in ( + WorkflowStages.TESTS_2, + WorkflowStages.TESTS_2_WW, + ): + print( + f"Failed Workflow status for [{workflow_job}], it's not required - continue" + ) + continue + + print(f"Failed Workflow status for [{workflow_job}]") + return False + return True + if __name__ == "__main__": parser = ArgumentParser( diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index dae1520afb6..0a2dd5e35ba 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -94,9 +94,10 @@ class GH: FAILURE = "failure" PENDING = "pending" SUCCESS = "success" + SKIPPED = "skipped" @classmethod - def _get_workflow_results(cls): + def get_workflow_results(cls): if not Path(Envs.WORKFLOW_RESULT_FILE).exists(): print( f"ERROR: Failed to get workflow results from file [{Envs.WORKFLOW_RESULT_FILE}]" @@ -115,13 +116,13 @@ class GH: @classmethod def print_workflow_results(cls): - res = cls._get_workflow_results() + res = cls.get_workflow_results() results = [f"{job}: {data['result']}" for job, data in res.items()] cls.print_in_group("Workflow results", results) @classmethod def is_workflow_ok(cls) -> bool: - res = cls._get_workflow_results() + res = cls.get_workflow_results() for _job, data in res.items(): if data["result"] == "failure": return False @@ -129,7 +130,7 @@ class GH: @classmethod def get_workflow_job_result(cls, wf_job_name: str) -> Optional[str]: - res = cls._get_workflow_results() + res = cls.get_workflow_results() if wf_job_name in res: return res[wf_job_name]["result"] # type: ignore else: diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 13c7537a84b..ddeb76adf7e 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -23,7 +23,7 @@ from commit_status_helper import ( from get_robot_token import get_best_robot_token from github_helper import GitHub, NamedUser, PullRequest, Repository from pr_info import PRInfo -from report import SUCCESS, FAILURE +from report import SUCCESS from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY from synchronizer_utils import SYNC_BRANCH_PREFIX from ci_config import CI @@ -248,23 +248,27 @@ def main(): repo = gh.get_repo(args.repo) if args.set_ci_status: + CI.GH.print_workflow_results() # set Mergeable check status and exit - assert args.wf_status in (FAILURE, SUCCESS) commit = get_commit(gh, args.pr_info.sha) statuses = get_commit_filtered_statuses(commit) has_failed_statuses = False - has_native_failed_status = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") - if CI.is_required(status.context) and status.state != SUCCESS: - print(f"WARNING: Failed status [{status.context}], [{status.state}]") + if ( + CI.is_required(status.context) + and status.state != SUCCESS + and status.context != CI.StatusNames.SYNC + ): + print( + f"WARNING: Not success status [{status.context}], [{status.state}]" + ) has_failed_statuses = True - if status.context != CI.StatusNames.SYNC: - has_native_failed_status = True - if args.wf_status == SUCCESS or has_failed_statuses: - # set Mergeable check if workflow is successful (green) + workflow_ok = CI.is_workflow_ok() + if workflow_ok or has_failed_statuses: + # set Mergeable Check if workflow is successful (green) # or if we have GH statuses with failures (red) # to avoid false-green on a died runner state = trigger_mergeable_check( @@ -283,7 +287,7 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - if args.wf_status == SUCCESS and not has_native_failed_status: + if workflow_ok and not has_failed_statuses: sys.exit(0) else: sys.exit(1) From bf2e679a51d843e03b52e0dc2853667c8b03978d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 19:40:25 +0000 Subject: [PATCH 0891/1170] no-msan 00314_sample_factor_virtual_column --- .../0_stateless/00314_sample_factor_virtual_column.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql b/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql index 6e3dc019069..b8ac5e733ed 100644 --- a/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql +++ b/tests/queries/0_stateless/00314_sample_factor_virtual_column.sql @@ -1,3 +1,7 @@ +-- Tags: no-msan +-- ^ +-- makes SELECTs extremely slow sometimes for some reason: "Aggregated. 1000000 to 1 rows (from 7.63 MiB) in 242.829221645 sec." + DROP TABLE IF EXISTS sample_00314_1; DROP TABLE IF EXISTS sample_00314_2; DROP TABLE IF EXISTS sample_merge_00314; From cb6baefa948ef1270ce9454f72075ac10bf6e729 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Aug 2024 21:49:36 +0100 Subject: [PATCH 0892/1170] 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 4829ba31d3e8f874beb9e626df801021673008fd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 5 Aug 2024 21:18:53 +0000 Subject: [PATCH 0893/1170] Actually fix a test --- .../test_distributed_type_object/test.py | 3 ++ ...allel_replicas_crash_after_refactoring.sql | 33 ------------------- 2 files changed, 3 insertions(+), 33 deletions(-) delete mode 100644 tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index e774876bc8b..e274bd6b774 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -31,6 +31,9 @@ def started_cluster(): def test_distributed_type_object(started_cluster): + node1.query("TRUNCATE TABLE local_table") + node2.query("TRUNCATE TABLE local_table") + node1.query( 'INSERT INTO local_table FORMAT JSONEachRow {"id": 1, "data": {"k1": 10}}' ) diff --git a/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql b/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql deleted file mode 100644 index cae4fa0f0df..00000000000 --- a/tests/queries/0_stateless/03215_parallel_replicas_crash_after_refactoring.sql +++ /dev/null @@ -1,33 +0,0 @@ --- Tags: disabled - -DROP TABLE IF EXISTS t1__fuzz_5; - -CREATE TABLE t1__fuzz_5 -( - `k` Int16, - `v` Nullable(UInt8) -) -ENGINE = MergeTree -ORDER BY k -SETTINGS index_granularity = 10; - -INSERT INTO t1__fuzz_5 SELECT - number, - number -FROM numbers(1000); - -INSERT INTO t1__fuzz_5 SELECT - number, - number -FROM numbers(1000, 1000); - -INSERT INTO t1__fuzz_5 SELECT - number, - number -FROM numbers(2000, 1000); - -SET receive_timeout = 10., receive_data_timeout_ms = 10000, allow_suspicious_low_cardinality_types = true, parallel_distributed_insert_select = 2, log_queries = true, table_function_remote_max_addresses = 200, max_execution_time = 10., max_memory_usage = 10000000000, log_comment = '/workspace/ch/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql', send_logs_level = 'warning', prefer_localhost_replica = false, allow_introspection_functions = true, use_parallel_replicas = 257, max_parallel_replicas = 65535, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0., parallel_replicas_for_non_replicated_merge_tree = true; - -SELECT max(k) IGNORE NULLS FROM t1__fuzz_5 WITH TOTALS SETTINGS use_parallel_replicas = 257, max_parallel_replicas = 65535, prefer_localhost_replica = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = -0; - -DROP TABLE IF EXISTS t1__fuzz_5; From 5cdf12a9277993f7adf97c5083c7e92f914a5ae7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 4 Aug 2024 22:31:17 +0000 Subject: [PATCH 0894/1170] Automatic style fix From 29f06e8037316d61034700eff8e41c9b8730f317 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 15:17:32 +0200 Subject: [PATCH 0895/1170] Update 00002_log_and_exception_messages_formatting.sql --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index b806aff3a00..efd961a0fda 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel, no-fasttest, no-ubsan, no-batch, no-flaky-check -- no-parallel because we want to run this test when most of the other tests already passed +-- This is not a regular test. It is intended to run once after other tests to validate certain statistics about the whole test runs. +-- TODO: I advice to put in inside clickhouse-test instead. -- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log From 61aff5efae873bde8631919386ba7f5c9db50af1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 15:17:48 +0200 Subject: [PATCH 0896/1170] Update 00002_log_and_exception_messages_formatting.sql --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index efd961a0fda..53321afc94c 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel, no-fasttest, no-ubsan, no-batch, no-flaky-check -- no-parallel because we want to run this test when most of the other tests already passed -- This is not a regular test. It is intended to run once after other tests to validate certain statistics about the whole test runs. --- TODO: I advice to put in inside clickhouse-test instead. +-- TODO: I advise to put in inside clickhouse-test instead. -- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log From cfcd9fe096de7e136093e20ac283569c23721714 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 21:57:23 +0000 Subject: [PATCH 0897/1170] Conflicts --- src/Interpreters/ExpressionActions.h | 2 +- src/Storages/StorageMergeTreeIndex.cpp | 15 +++++++-------- src/Storages/StorageMergeTreeIndex.h | 2 +- src/Storages/System/IStorageSystemOneBlock.cpp | 4 ++-- src/Storages/System/StorageSystemColumns.cpp | 6 +++--- .../System/StorageSystemDataSkippingIndices.cpp | 11 +++++------ src/Storages/System/StorageSystemReplicas.cpp | 11 +++++------ src/Storages/VirtualColumnUtils.h | 7 +++---- 8 files changed, 27 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 6ff39ee07f7..7652fe49eab 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -102,7 +102,7 @@ public: /// /// @param allow_duplicates_in_input - actions are allowed to have /// duplicated input (that will refer into the block). This is needed for - /// preliminary query filtering (filterBlockWithDAG()), because they just + /// preliminary query filtering (filterBlockWithExpression()), because they just /// pass available virtual columns, which cannot be moved in case they are /// used multiple times. void execute(Block & block, size_t & num_rows, bool dry_run = false, bool allow_duplicates_in_input = false) const; diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index 90d01d356e9..15728290f19 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -275,7 +275,7 @@ public: private: std::shared_ptr storage; Poco::Logger * log; - ActionsDAGPtr virtual_columns_filter; + ExpressionActionsPtr virtual_columns_filter; }; void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) @@ -289,10 +289,9 @@ void ReadFromMergeTreeIndex::applyFilters(ActionDAGNodes added_filter_nodes) { {}, std::make_shared(), StorageMergeTreeIndex::part_name_column.name }, }; - virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); - - if (virtual_columns_filter) - VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + if (dag) + virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context); } } @@ -345,7 +344,7 @@ void StorageMergeTreeIndex::read( void ReadFromMergeTreeIndex::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto filtered_parts = storage->getFilteredDataParts(virtual_columns_filter, context); + auto filtered_parts = storage->getFilteredDataParts(virtual_columns_filter); LOG_DEBUG(log, "Reading index{}from {} parts of table {}", storage->with_marks ? " with marks " : " ", @@ -355,7 +354,7 @@ void ReadFromMergeTreeIndex::initializePipeline(QueryPipelineBuilder & pipeline, pipeline.init(Pipe(std::make_shared(getOutputStream().header, storage->key_sample_block, std::move(filtered_parts), context, storage->with_marks))); } -MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(ActionsDAGPtr virtual_columns_filter, const ContextPtr & context) const +MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(const ExpressionActionsPtr & virtual_columns_filter) const { if (!virtual_columns_filter) return data_parts; @@ -365,7 +364,7 @@ MergeTreeData::DataPartsVector StorageMergeTreeIndex::getFilteredDataParts(Actio all_part_names->insert(part->name); Block filtered_block{{std::move(all_part_names), std::make_shared(), part_name_column.name}}; - VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, filtered_block, context); + VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, filtered_block); if (!filtered_block.rows()) return {}; diff --git a/src/Storages/StorageMergeTreeIndex.h b/src/Storages/StorageMergeTreeIndex.h index 652a2d6eeaf..ed8274d7d92 100644 --- a/src/Storages/StorageMergeTreeIndex.h +++ b/src/Storages/StorageMergeTreeIndex.h @@ -36,7 +36,7 @@ public: private: friend class ReadFromMergeTreeIndex; - MergeTreeData::DataPartsVector getFilteredDataParts(ActionsDAGPtr virtual_columns_filter, const ContextPtr & context) const; + MergeTreeData::DataPartsVector getFilteredDataParts(const ExpressionActionsPtr & virtual_columns_filter) const; StoragePtr source_table; bool with_marks; diff --git a/src/Storages/System/IStorageSystemOneBlock.cpp b/src/Storages/System/IStorageSystemOneBlock.cpp index 308b34510ea..b8f32fcdb83 100644 --- a/src/Storages/System/IStorageSystemOneBlock.cpp +++ b/src/Storages/System/IStorageSystemOneBlock.cpp @@ -45,7 +45,7 @@ public: private: std::shared_ptr storage; std::vector columns_mask; - ActionsDAGPtr filter; + std::optional filter; }; void IStorageSystemOneBlock::read( @@ -106,7 +106,7 @@ void ReadFromSystemOneBlock::applyFilters(ActionDAGNodes added_filter_nodes) /// Must prepare sets here, initializePipeline() would be too late, see comment on FutureSetFromSubquery. if (filter) - VirtualColumnUtils::buildSetsForDAG(filter, context); + VirtualColumnUtils::buildSetsForDAG(*filter, context); } } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6b3e0094562..03c569303c5 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -338,7 +338,7 @@ private: std::shared_ptr storage; std::vector columns_mask; const size_t max_block_size; - ActionsDAGPtr virtual_columns_filter; + std::optional virtual_columns_filter; }; void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) @@ -355,7 +355,7 @@ void ReadFromSystemColumns::applyFilters(ActionDAGNodes added_filter_nodes) /// Must prepare sets here, initializePipeline() would be too late, see comment on FutureSetFromSubquery. if (virtual_columns_filter) - VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + VirtualColumnUtils::buildSetsForDAG(*virtual_columns_filter, context); } } @@ -468,7 +468,7 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, /// Filter block with `database` and `table` columns. if (virtual_columns_filter) - VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, block_to_filter, context); + VirtualColumnUtils::filterBlockWithPredicate(virtual_columns_filter->getOutputs().at(0), block_to_filter, context); if (!block_to_filter.rows()) { diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index a6bba44e257..a41771df406 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -214,7 +214,7 @@ private: std::shared_ptr storage; std::vector columns_mask; const size_t max_block_size; - ActionsDAGPtr virtual_columns_filter; + ExpressionActionsPtr virtual_columns_filter; }; void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter_nodes) @@ -228,10 +228,9 @@ void ReadFromSystemDataSkippingIndices::applyFilters(ActionDAGNodes added_filter { ColumnString::create(), std::make_shared(), "database" }, }; - virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); - - if (virtual_columns_filter) - VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + if (dag) + virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context); } } @@ -279,7 +278,7 @@ void ReadFromSystemDataSkippingIndices::initializePipeline(QueryPipelineBuilder /// Condition on "database" in a query acts like an index. Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; if (virtual_columns_filter) - VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, block, context); + VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, block); ColumnPtr & filtered_databases = block.getByPosition(0).column; pipeline.init(Pipe(std::make_shared( diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 9fb4dc5ed6f..724e4bd3f77 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -285,7 +285,7 @@ private: const bool with_zk_fields; const size_t max_block_size; std::shared_ptr impl; - ActionsDAGPtr virtual_columns_filter; + ExpressionActionsPtr virtual_columns_filter; }; void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) @@ -301,10 +301,9 @@ void ReadFromSystemReplicas::applyFilters(ActionDAGNodes added_filter_nodes) { ColumnString::create(), std::make_shared(), "engine" }, }; - virtual_columns_filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); - - if (virtual_columns_filter) - VirtualColumnUtils::buildSetsForDAG(virtual_columns_filter, context); + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + if (dag) + virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context); } } @@ -443,7 +442,7 @@ void ReadFromSystemReplicas::initializePipeline(QueryPipelineBuilder & pipeline, }; if (virtual_columns_filter) - VirtualColumnUtils::filterBlockWithDAG(virtual_columns_filter, filtered_block, context); + VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, filtered_block); if (!filtered_block.rows()) { diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 3f3f93eccf7..abf46dc23a4 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -20,15 +20,14 @@ namespace VirtualColumnUtils /// The filtering functions are tricky to use correctly. /// There are 2 ways: -/// 1. Call filterBlockWithPredicate() or filterBlockWithDAG() inside SourceStepWithFilter::applyFilters(). +/// 1. Call filterBlockWithPredicate() or filterBlockWithExpression() inside SourceStepWithFilter::applyFilters(). /// 2. Call splitFilterDagForAllowedInputs() and buildSetsForDAG() inside SourceStepWithFilter::applyFilters(). -/// Then call filterBlockWithPredicate() or filterBlockWithDAG() in initializePipeline(). +/// Then call filterBlockWithPredicate() or filterBlockWithExpression() in initializePipeline(). /// /// Otherwise calling filter*() outside applyFilters() will throw "Not-ready Set is passed" /// if there are subqueries. -/// Similar to filterBlockWithQuery, but uses ActionsDAG as a predicate. -/// Basically it is filterBlockWithDAG(splitFilterDagForAllowedInputs). +/// 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. From a05ead3f77d52f6f168c0bb483633a63ffbb3997 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 22:04:53 +0000 Subject: [PATCH 0898/1170] Conflict --- src/Core/SettingsChangesHistory.cpp | 260 ---------------------------- 1 file changed, 260 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b80e4bf3086..5b94391bade 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +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.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."}, - {"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_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe 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."}, - {"restore_replace_external_table_functions_to_null", false, false, "New setting."}, - {"restore_replace_external_engines_to_null", false, false, "New setting."} - }}, - {"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", { } From 4e8bd0654cd378fd03985170a4df8f49b583bd01 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 22:06:36 +0000 Subject: [PATCH 0899/1170] Remove from history --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5b94391bade..03de54f2780 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -89,7 +89,6 @@ static std::initializer_list Date: Mon, 5 Aug 2024 22:36:30 +0000 Subject: [PATCH 0900/1170] Apply https://github.com/ClickHouse/zlib-ng/pull/18 --- contrib/zlib-ng | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/zlib-ng b/contrib/zlib-ng index c19ba056b7c..a2fbeffdc30 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit c19ba056b7cc8029bb80f509956090c7ded58032 +Subproject commit a2fbeffdc30a8b0ce6d54ee31208e2688eac4c9f From 087d014f2ae050b7a45f72a7cf87f5de5a8179cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Aug 2024 03:17:17 +0200 Subject: [PATCH 0901/1170] Revert "Revert "Add a test for #47892"" --- .../02968_full_sorting_join_fuzz.reference | 3 +++ .../0_stateless/02968_full_sorting_join_fuzz.sql | 15 +++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02968_full_sorting_join_fuzz.reference create mode 100644 tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql diff --git a/tests/queries/0_stateless/02968_full_sorting_join_fuzz.reference b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.reference new file mode 100644 index 00000000000..0e9f5cc1db1 --- /dev/null +++ b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.reference @@ -0,0 +1,3 @@ +[NULL] [] 100 0 + +[NULL] [] 100 0 diff --git a/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql new file mode 100644 index 00000000000..802eda57df3 --- /dev/null +++ b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql @@ -0,0 +1,15 @@ +SET max_bytes_in_join = 0, join_algorithm = 'full_sorting_merge', max_block_size = 10240; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (`key` UInt32, `s` String) ENGINE = MergeTree ORDER BY key; +CREATE TABLE t2 (`key` UInt32, `s` String) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 SELECT (sipHash64(number, 'x') % 10000000) + 1 AS key, concat('val', toString(number)) AS s FROM numbers_mt(10000000); +INSERT INTO t2 SELECT (sipHash64(number, 'y') % 1000000) + 1 AS key, concat('val', toString(number)) AS s FROM numbers_mt(1000000); + +SELECT materialize([NULL]), [], 100, count(materialize(NULL)) FROM t1 ALL INNER JOIN t2 ON t1.key = t2.key PREWHERE 10 WHERE t2.key WITH TOTALS; + +DROP TABLE t1; +DROP TABLE t2; From 52f3ad97e86b485419f3f88c5c047d0868d0574a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Aug 2024 03:30:19 +0200 Subject: [PATCH 0902/1170] Fix MaterializedMySQL --- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 2f5477a6b9d..8b3850c4e0c 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -46,6 +46,7 @@ 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 From 15a4d13cdad3fe9181703d5c0a993b8dcb8cb263 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 6 Aug 2024 04:33:02 +0200 Subject: [PATCH 0903/1170] Update 02968_full_sorting_join_fuzz.sql --- tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql index 802eda57df3..85ca740cef2 100644 --- a/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql +++ b/tests/queries/0_stateless/02968_full_sorting_join_fuzz.sql @@ -9,7 +9,7 @@ CREATE TABLE t2 (`key` UInt32, `s` String) ENGINE = MergeTree ORDER BY key; INSERT INTO t1 SELECT (sipHash64(number, 'x') % 10000000) + 1 AS key, concat('val', toString(number)) AS s FROM numbers_mt(10000000); INSERT INTO t2 SELECT (sipHash64(number, 'y') % 1000000) + 1 AS key, concat('val', toString(number)) AS s FROM numbers_mt(1000000); -SELECT materialize([NULL]), [], 100, count(materialize(NULL)) FROM t1 ALL INNER JOIN t2 ON t1.key = t2.key PREWHERE 10 WHERE t2.key WITH TOTALS; +SELECT materialize([NULL]), [], 100, count(materialize(NULL)) FROM t1 ALL INNER JOIN t2 ON t1.key = t2.key PREWHERE 10 WHERE t2.key != 0 WITH TOTALS; DROP TABLE t1; DROP TABLE t2; From 24e1bfdb686bf4846c032eab49bc86c5389865ae Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Aug 2024 07:37:00 +0000 Subject: [PATCH 0904/1170] 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 8e6de033355485a770f86b848e7574728a0eecf4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Aug 2024 09:39:00 +0200 Subject: [PATCH 0905/1170] Remove unsupported files --- .../queries/0_stateless/03215_parsing_archive_name_s3.sql | 8 ++++++++ tests/queries/0_stateless/data_minio/::03215_archive.csv | 1 - .../0_stateless/data_minio/test.zip::03215_archive.csv | 1 - .../0_stateless/data_minio/test::03215_archive.csv | 1 - 4 files changed, 8 insertions(+), 3 deletions(-) delete mode 100644 tests/queries/0_stateless/data_minio/::03215_archive.csv delete mode 100644 tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv delete mode 100644 tests/queries/0_stateless/data_minio/test::03215_archive.csv diff --git a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql index e34be475c5a..e0d63d313fa 100644 --- a/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql +++ b/tests/queries/0_stateless/03215_parsing_archive_name_s3.sql @@ -1,7 +1,15 @@ -- Tags: no-fasttest -- Tag no-fasttest: Depends on AWS +SET s3_truncate_on_insert=1; + +INSERT INTO FUNCTION s3(s3_conn, filename='::03215_archive.csv') SELECT 1; SELECT _file, _path FROM s3(s3_conn, filename='::03215_archive.csv') ORDER BY (_file, _path); + SELECT _file, _path FROM s3(s3_conn, filename='test :: 03215_archive.csv') ORDER BY (_file, _path); -- { serverError S3_ERROR } + +INSERT INTO FUNCTION s3(s3_conn, filename='test::03215_archive.csv') SELECT 1; SELECT _file, _path FROM s3(s3_conn, filename='test::03215_archive.csv') ORDER BY (_file, _path); + +INSERT INTO FUNCTION s3(s3_conn, filename='test.zip::03215_archive.csv') SETTINGS allow_archive_path_syntax=0 SELECT 1; SELECT _file, _path FROM s3(s3_conn, filename='test.zip::03215_archive.csv') ORDER BY (_file, _path) SETTINGS allow_archive_path_syntax=0; diff --git a/tests/queries/0_stateless/data_minio/::03215_archive.csv b/tests/queries/0_stateless/data_minio/::03215_archive.csv deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/data_minio/::03215_archive.csv +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv b/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/data_minio/test.zip::03215_archive.csv +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/data_minio/test::03215_archive.csv b/tests/queries/0_stateless/data_minio/test::03215_archive.csv deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/data_minio/test::03215_archive.csv +++ /dev/null @@ -1 +0,0 @@ -1 From 11fd263be6e24ee4cdc3a51ac497510c82837fa5 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 1 Aug 2024 08:35:05 +0000 Subject: [PATCH 0906/1170] 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 0907/1170] 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 27569815d519c8ad70842798b989874cc04b3271 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 17:01:00 +0200 Subject: [PATCH 0908/1170] Fix query cache randomization in stress tests Right now it fails with [1]: 2024.08.05 12:53:53.659422 [ 3559 ] {123237a2-5f08-47e4-996e-21e20bc4a51a} executeQuery: Code: 704. DB::Exception: The query result was not cached because the query contains a non-deterministic function. Use setting `query_cache_nondeterministic_function_handling = 'save'` or `= 'ignore'` to cache the query result regardless or to omit caching. (QUERY_CACHE_USED_WITH_NONDETERMINISTIC_FUNCTIONS) (version 24.8.1.1356) (from [::1]:42534) (in query: SELECT version(), arrayStringConcat(groupArray(value), ' ') FROM system.build_options WHERE name IN ('GIT_HASH', 'GIT_BRANCH') ), Stack trace (when copying this message, always include the lines below): Uexpected exception, will not retry: HTTPError : Code: 500. Code: 704. DB::Exception: The query result was not cached because the query contains a non-deterministic function. Use setting `query_cache_nondeterministic_function_handling = 'save'` or `= 'ignore'` to cache the query result regardless or to omit caching. (QUERY_CACHE_USED_WITH_NONDETERMINISTIC_FUNCTIONS) (version 24.8.1.1356) [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/e68c9c8d16f37f6c25739076c9b071ed97952269/stress_test__asan_/stress_test_run_10.txt Signed-off-by: Azat Khuzhin --- tests/ci/stress.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 8b8b17dd2a9..a3cdd8b3d6f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -47,6 +47,8 @@ def get_options(i: int, upgrade_check: bool) -> str: if i > 0 and random.random() < 1 / 3: client_options.append("use_query_cache=1") + client_options.append("query_cache_nondeterministic_function_handling='ignore'") + client_options.append("query_cache_system_table_handling='ignore'") if i % 5 == 1: client_options.append("memory_tracker_fault_probability=0.001") From b2987e4f4dd267293b8b4f96f86f2202a7671b84 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 6 Aug 2024 11:42:18 +0200 Subject: [PATCH 0909/1170] Update StorageWindowView.cpp --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index bf934ed00d9..5830c844582 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1450,7 +1450,7 @@ void StorageWindowView::writeIntoWindowView( LOG_TRACE(window_view.log, "New max watermark: {}", window_view.max_watermark); } - Pipe pipe(std::make_shared(std::move(block))); + Pipe pipe(std::make_shared(block)); UInt32 lateness_bound = 0; UInt32 t_max_watermark = 0; From 50ed7da27dc1bcdbb03f5cf7e3d80ae95ecf643f Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Tue, 6 Aug 2024 12:02:53 +0200 Subject: [PATCH 0910/1170] squash! fix for parallel execution --- tests/queries/0_stateless/03215_parquet_index.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03215_parquet_index.sql b/tests/queries/0_stateless/03215_parquet_index.sql index 5b176ff70ba..0afccdf6f3b 100644 --- a/tests/queries/0_stateless/03215_parquet_index.sql +++ b/tests/queries/0_stateless/03215_parquet_index.sql @@ -4,12 +4,14 @@ CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet); INSERT INTO test_parquet SELECT number, toString(number) FROM numbers(100); SELECT col1, col2 FROM test_parquet; + -- Parquet will have indexes in columns. We are not checking that indexes exist here, there is an integration test test_parquet_page_index for that. We just check that a setting doesn't break the SELECT DROP TABLE IF EXISTS test_parquet; CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet) SETTINGS output_format_parquet_use_custom_encoder=false, output_format_parquet_write_page_index=true; INSERT INTO test_parquet SELECT number, toString(number) FROM numbers(100); SELECT col1, col2 FROM test_parquet; + -- Parquet will not have indexes in columns. DROP TABLE IF EXISTS test_parquet; CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet) SETTINGS output_format_parquet_use_custom_encoder=false, output_format_parquet_write_page_index=false; From 4a8be15ca8bffc445a6c66cacb5226946531e105 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:25:33 +0000 Subject: [PATCH 0911/1170] Bump rocksdb to v8.11.4 --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 49ce8a1064d..5c2be544f55 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79 +Subproject commit 5c2be544f5509465957706c955b6d623e889ac4e From 7c15ad3966a982a3597034f8a66d91f2750dd2d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Aug 2024 10:15:53 +0000 Subject: [PATCH 0912/1170] do not try to create azure container if not needed --- src/Common/ProfileEvents.cpp | 4 +- .../AzureBlobStorageCommon.cpp | 45 ++++++++++++++++++- 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 2b090136e2e..c6cf6c04af3 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -457,6 +457,7 @@ The server successfully detected this situation and will download merged part fr M(AzureDeleteObjects, "Number of Azure blob storage API DeleteObject(s) calls.") \ M(AzureListObjects, "Number of Azure blob storage API ListObjects calls.") \ M(AzureGetProperties, "Number of Azure blob storage API GetProperties calls.") \ + M(AzureCreateContainer, "Number of Azure blob storage API CreateContainer calls.") \ \ M(DiskAzureGetObject, "Number of Disk Azure API GetObject calls.") \ M(DiskAzureUpload, "Number of Disk Azure blob storage API Upload calls") \ @@ -464,8 +465,9 @@ The server successfully detected this situation and will download merged part fr M(DiskAzureCommitBlockList, "Number of Disk Azure blob storage API CommitBlockList calls") \ M(DiskAzureCopyObject, "Number of Disk Azure blob storage API CopyObject calls") \ M(DiskAzureListObjects, "Number of Disk Azure blob storage API ListObjects calls.") \ - M(DiskAzureDeleteObjects, "Number of Azure blob storage API DeleteObject(s) calls.") \ + M(DiskAzureDeleteObjects, "Number of Disk Azure blob storage API DeleteObject(s) calls.") \ M(DiskAzureGetProperties, "Number of Disk Azure blob storage API GetProperties calls.") \ + M(DiskAzureCreateContainer, "Number of Disk Azure blob storage API CreateContainer calls.") \ \ M(ReadBufferFromAzureMicroseconds, "Time spent on reading from Azure.") \ M(ReadBufferFromAzureInitMicroseconds, "Time spent initializing connection to Azure.") \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 0aa3b9c40b5..9043edd66a0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -11,6 +11,14 @@ #include #include +namespace ProfileEvents +{ + extern const Event AzureGetProperties; + extern const Event DiskAzureGetProperties; + extern const Event AzureCreateContainer; + extern const Event DiskAzureCreateContainer; +} + namespace DB { @@ -214,20 +222,53 @@ void processURL(const String & url, const String & container_name, Endpoint & en } } +static bool containerExists(const ContainerClient & client) +{ + ProfileEvents::increment(ProfileEvents::AzureGetProperties); + if (client.GetClickhouseOptions().IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); + + try + { + client.GetProperties(); + return true; + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) + return false; + throw; + } +} + std::unique_ptr getContainerClient(const ConnectionParams & params, bool readonly) { if (params.endpoint.container_already_exists.value_or(false) || readonly) + { return params.createForContainer(); + } + + if (!params.endpoint.container_already_exists.has_value()) + { + auto container_client = params.createForContainer(); + if (containerExists(*container_client)) + return container_client; + } try { auto service_client = params.createForService(); + + ProfileEvents::increment(ProfileEvents::AzureCreateContainer); + if (params.client_options.ClickhouseOptions.IsClientForDisk) + ProfileEvents::increment(ProfileEvents::DiskAzureCreateContainer); + return std::make_unique(service_client->CreateBlobContainer(params.endpoint.container_name).Value); } catch (const Azure::Storage::StorageException & e) { - /// If container_already_exists is not set (in config), ignore already exists error. - /// (Conflict - The specified container already exists) + /// If container_already_exists is not set (in config), ignore already exists error. Conflict - The specified container already exists. + /// To avoid race with creation of container handle this error despite that we have already checked the existence of container. if (!params.endpoint.container_already_exists.has_value() && e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) return params.createForContainer(); throw; From 7e87650a001279317ce79c36415160639a83a4da Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:31:13 +0000 Subject: [PATCH 0913/1170] Bump rocksdb to v9.0.1 --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 5c2be544f55..36540d43550 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 5c2be544f5509465957706c955b6d623e889ac4e +Subproject commit 36540d43550da19e96515bd731ee416a8787672c From f0a6ae220530fa9b34d23aa722ed8816842410ce Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:32:40 +0000 Subject: [PATCH 0914/1170] Bump rocksdb to v9.1.1 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 36540d43550..6f7cabeac80 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 36540d43550da19e96515bd731ee416a8787672c +Subproject commit 6f7cabeac80a3a6150be2c8a8369fcecb107bf43 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 57c056532c6..0157f31d5d9 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -150,6 +150,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc ${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc ${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc + ${ROCKSDB_SOURCE_DIR}/db/multi_cf_iterator.cc ${ROCKSDB_SOURCE_DIR}/db/output_validator.cc ${ROCKSDB_SOURCE_DIR}/db/periodic_task_scheduler.cc ${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc From b3f5ddcd658efc8ad84b32481239750e8999ba0a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:37:35 +0000 Subject: [PATCH 0915/1170] Bump rocksdb to 9.2.1 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 14 +++++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 6f7cabeac80..08f93221f50 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 6f7cabeac80a3a6150be2c8a8369fcecb107bf43 +Subproject commit 08f93221f50700f19f11555fb46abfe708a716d1 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 0157f31d5d9..cff8f832f23 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -88,6 +88,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 +105,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 @@ -150,7 +152,6 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc ${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc ${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc - ${ROCKSDB_SOURCE_DIR}/db/multi_cf_iterator.cc ${ROCKSDB_SOURCE_DIR}/db/output_validator.cc ${ROCKSDB_SOURCE_DIR}/db/periodic_task_scheduler.cc ${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc @@ -389,6 +390,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 @@ -419,10 +421,12 @@ 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}/db/db_impl/db_impl_follower.cc + ${ROCKSDB_SOURCE_DIR}/port/port_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/env_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_on_demand.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) From e7110be48c7b2b752cd51fd72a23fe956248077c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:43:21 +0000 Subject: [PATCH 0916/1170] Bump rocksdb to v9.3.1 --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 08f93221f50..c5201abc4a9 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 08f93221f50700f19f11555fb46abfe708a716d1 +Subproject commit c5201abc4a983450f9423435a4405829be23d0a8 From d7803ca621ca511f0bf2abe0fbc332658b5c2919 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 6 Aug 2024 10:48:10 +0000 Subject: [PATCH 0917/1170] 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 2a2ba4e685022d3687565586c06dc56dfc276f10 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 10:47:03 +0000 Subject: [PATCH 0918/1170] Bump rocksdb to 9.4.0 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index c5201abc4a9..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit c5201abc4a983450f9423435a4405829be23d0a8 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index cff8f832f23..b39a739d04c 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -126,6 +126,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 @@ -183,6 +184,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 @@ -370,6 +372,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 @@ -421,10 +424,8 @@ if(HAS_ARMV8_CRC) endif(HAS_ARMV8_CRC) list(APPEND SOURCES - ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_follower.cc ${ROCKSDB_SOURCE_DIR}/port/port_posix.cc ${ROCKSDB_SOURCE_DIR}/env/env_posix.cc - ${ROCKSDB_SOURCE_DIR}/env/fs_on_demand.cc ${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc ${ROCKSDB_SOURCE_DIR}/env/io_posix.cc) From 995187006a8c2500ddb7fa234f3443c75d900be4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 5 Aug 2024 20:23:41 +0200 Subject: [PATCH 0919/1170] 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 0920/1170] 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 df0dac2f5b509438cce28214b78765c46439aa8c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 11:17:41 +0000 Subject: [PATCH 0921/1170] Enable jemalloc and liburing in rocksdb --- contrib/CMakeLists.txt | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 30 +++++++++++++++++----------- 2 files changed, 19 insertions(+), 13 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-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index b39a739d04c..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() @@ -433,5 +435,9 @@ 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 97f1cfc232c331504754db4ba4221a282c690bb6 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 24 Jul 2024 14:32:35 +0000 Subject: [PATCH 0922/1170] 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 0923/1170] 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 523767c0c0725773c074e1ad00229b27865a34f2 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 17:26:28 +0200 Subject: [PATCH 0924/1170] run 01171 test in parallel --- .../01171_mv_select_insert_isolation_long.sh | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 2fb58e4cc57..27f063c34d8 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-ordinary-database -# Test is too heavy, avoid parallel run in Flaky Check +# Tags: long, no-ordinary-database # shellcheck disable=SC2119 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -22,11 +21,6 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt32, nm Int32) ENGINE=MergeTr $CLICKHOUSE_CLIENT --query "INSERT INTO src VALUES (0, 0)" -function get_now() -{ - date +%s -} - is_pid_exist() { local pid=$1 @@ -42,14 +36,14 @@ function run_until_deadline_and_at_least_times() local function_to_run=$1; shift local started_time - started_time=$(get_now) + started_time=$SECONDS local i=0 while true do $function_to_run $i "$@" - [[ $(get_now) -lt $deadline ]] || break + [[ $SECONDS -lt $deadline ]] || break i=$(($i + 1)) done @@ -165,7 +159,7 @@ if [[ $((MAIN_TIME_PART + SECOND_TIME_PART + WAIT_FINISH + LAST_TIME_GAP)) -ge exit 1 fi -START_TIME=$(get_now) +START_TIME=$SECONDS STOP_TIME=$((START_TIME + MAIN_TIME_PART)) SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) MIN_ITERATIONS=20 From 244da490138668882db10770679b940d2c4adadf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 18:41:44 +0200 Subject: [PATCH 0925/1170] add max iteration --- .../01171_mv_select_insert_isolation_long.sh | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 27f063c34d8..fe5bfe529b7 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -27,28 +27,30 @@ is_pid_exist() ps -p $pid > /dev/null } -function run_until_deadline_and_at_least_times() +function run_until_deadline_with_min_max_iterations() { set -e local deadline=$1; shift local min_iterations=$1; shift + local max_iterations=$1; shift local function_to_run=$1; shift local started_time started_time=$SECONDS - local i=0 + local iteration=0 while true do - $function_to_run $i "$@" + $function_to_run $iteration "$@" [[ $SECONDS -lt $deadline ]] || break + [[ $iteration -lt $max_iterations ]] || break - i=$(($i + 1)) + iteration=$(($iteration + 1)) done - [[ $i -gt $min_iterations ]] || echo "$i/$min_iterations : not enough iterations of $function_to_run has been made from $started_time until $deadline" >&2 + [[ $iteration -gt $min_iterations ]] || echo "$iteration/$min_iterations : not enough iterations of $function_to_run has been made from $started_time until $deadline" >&2 } function insert_commit_action() @@ -163,17 +165,18 @@ START_TIME=$SECONDS STOP_TIME=$((START_TIME + MAIN_TIME_PART)) SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) MIN_ITERATIONS=20 +MAX_ITERATIONS=200 -run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 1 & PID_1=$! -run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_commit_action 2 & PID_2=$! -run_until_deadline_and_at_least_times $STOP_TIME $MIN_ITERATIONS insert_rollback_action 3 & PID_3=$! +run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 1 & PID_1=$! +run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 2 & PID_2=$! +run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_rollback_action 3 & PID_3=$! -run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS optimize_action & PID_4=$! -run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_action & PID_5=$! -run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_insert_action & PID_6=$! +run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS optimize_action & PID_4=$! +run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_5=$! +run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_6=$! sleep 0.$RANDOM -run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_action & PID_7=$! -run_until_deadline_and_at_least_times $SECOND_STOP_TIME $MIN_ITERATIONS select_insert_action & PID_8=$! +run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_7=$! +run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_8=$! wait $PID_1 || echo "insert_commit_action has failed with status $?" 2>&1 wait $PID_2 || echo "second insert_commit_action has failed with status $?" 2>&1 From c896165e161b2267ae4d6bc255e028d7c0899bd9 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 Jul 2024 17:49:39 +0200 Subject: [PATCH 0926/1170] add min time --- .../01171_mv_select_insert_isolation_long.sh | 53 +++++++++++-------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index fe5bfe529b7..ff71d37cb32 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -27,15 +27,20 @@ is_pid_exist() ps -p $pid > /dev/null } -function run_until_deadline_with_min_max_iterations() +function run_with_time_and_iterations_limits() { set -e - local deadline=$1; shift + local min_time=$1; shift + local max_time=$1; shift local min_iterations=$1; shift local max_iterations=$1; shift local function_to_run=$1; shift + # if [ "${1:-X}" = "1" ]; then + # set -x + # fi + local started_time started_time=$SECONDS local iteration=0 @@ -44,13 +49,14 @@ function run_until_deadline_with_min_max_iterations() do $function_to_run $iteration "$@" - [[ $SECONDS -lt $deadline ]] || break - [[ $iteration -lt $max_iterations ]] || break + [[ $SECONDS -lt $max_time ]] || break + [[ $SECONDS -lt $min_time ]] || [[ $iteration -lt $max_iterations ]] || break iteration=$(($iteration + 1)) done - [[ $iteration -gt $min_iterations ]] || echo "$iteration/$min_iterations : not enough iterations of $function_to_run has been made from $started_time until $deadline" >&2 + [[ $iteration -gt $min_iterations ]] || echo "$iteration/$min_iterations : not enough iterations of $function_to_run has been made from $started_time until $max_time" >&2 + set +x } function insert_commit_action() @@ -151,43 +157,44 @@ function select_insert_action() ROLLBACK;" } -MAIN_TIME_PART=400 -SECOND_TIME_PART=30 +MIN_SECONDS=5 +MAX_SECONDS=300 WAIT_FINISH=60 -LAST_TIME_GAP=10 -if [[ $((MAIN_TIME_PART + SECOND_TIME_PART + WAIT_FINISH + LAST_TIME_GAP)) -ge 600 ]]; then +if [[ $((MAX_SECONDS + WAIT_FINISH)) -ge 550 ]]; then echo "time sttings are wrong" 2>&1 exit 1 fi START_TIME=$SECONDS -STOP_TIME=$((START_TIME + MAIN_TIME_PART)) -SECOND_STOP_TIME=$((STOP_TIME + SECOND_TIME_PART)) +MIN_TIME=$((START_TIME + MIN_SECONDS)) +MAX_TIME=$((START_TIME + MAX_SECONDS)) MIN_ITERATIONS=20 MAX_ITERATIONS=200 -run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 1 & PID_1=$! -run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 2 & PID_2=$! -run_until_deadline_with_min_max_iterations $STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_rollback_action 3 & PID_3=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 1 & PID_1=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 2 & PID_2=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_rollback_action 3 & PID_3=$! -run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS optimize_action & PID_4=$! -run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_5=$! -run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_6=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS optimize_action & PID_4=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_5=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_6=$! sleep 0.$RANDOM -run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_7=$! -run_until_deadline_with_min_max_iterations $SECOND_STOP_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_8=$! - -wait $PID_1 || echo "insert_commit_action has failed with status $?" 2>&1 -wait $PID_2 || echo "second insert_commit_action has failed with status $?" 2>&1 -wait $PID_3 || echo "insert_rollback_action has failed with status $?" 2>&1 +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_action & PID_7=$! +run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS select_insert_action & PID_8=$! +is_pid_exist $PID_1 || echo "insert_commit_action is not running" 2>&1 +is_pid_exist $PID_2 || echo "second insert_commit_action is not running" 2>&1 +is_pid_exist $PID_3 || echo "insert_rollback_action is not running" 2>&1 is_pid_exist $PID_4 || echo "optimize_action is not running" 2>&1 is_pid_exist $PID_5 || echo "select_action is not running" 2>&1 is_pid_exist $PID_6 || echo "select_insert_action is not running" 2>&1 is_pid_exist $PID_7 || echo "second select_action is not running" 2>&1 is_pid_exist $PID_8 || echo "second select_insert_action is not running" 2>&1 +wait $PID_1 || echo "insert_commit_action has failed with status $?" 2>&1 +wait $PID_2 || echo "second insert_commit_action has failed with status $?" 2>&1 +wait $PID_3 || echo "insert_rollback_action has failed with status $?" 2>&1 wait $PID_4 || echo "optimize_action has failed with status $?" 2>&1 wait $PID_5 || echo "select_action has failed with status $?" 2>&1 wait $PID_6 || echo "select_insert_action has failed with status $?" 2>&1 From 874a7dbe378e83331d6e4fab6aa82a0409fbfc5d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Aug 2024 17:39:36 +0200 Subject: [PATCH 0927/1170] adjust settings --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index ff71d37cb32..ba6fc85ca1a 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -158,7 +158,7 @@ function select_insert_action() } MIN_SECONDS=5 -MAX_SECONDS=300 +MAX_SECONDS=400 WAIT_FINISH=60 if [[ $((MAX_SECONDS + WAIT_FINISH)) -ge 550 ]]; then From 860050eb3d15f26c371a1aeab92fd8c0d166ce09 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Aug 2024 13:30:23 +0200 Subject: [PATCH 0928/1170] Update src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 9043edd66a0..1a0b6157a86 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -268,7 +268,7 @@ std::unique_ptr getContainerClient(const ConnectionParams & par catch (const Azure::Storage::StorageException & e) { /// If container_already_exists is not set (in config), ignore already exists error. Conflict - The specified container already exists. - /// To avoid race with creation of container handle this error despite that we have already checked the existence of container. + /// To avoid race with creation of container, handle this error despite that we have already checked the existence of container. if (!params.endpoint.container_already_exists.has_value() && e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) return params.createForContainer(); throw; From 851d4d3fdfdb682d0f9c9a8b1945608c6bc53086 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 6 Aug 2024 13:56:37 +0200 Subject: [PATCH 0929/1170] adjust MIN_ITERATIONS --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index ba6fc85ca1a..f6a88c205c1 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -169,7 +169,7 @@ fi START_TIME=$SECONDS MIN_TIME=$((START_TIME + MIN_SECONDS)) MAX_TIME=$((START_TIME + MAX_SECONDS)) -MIN_ITERATIONS=20 +MIN_ITERATIONS=15 MAX_ITERATIONS=200 run_with_time_and_iterations_limits $MIN_TIME $MAX_TIME $MIN_ITERATIONS $MAX_ITERATIONS insert_commit_action 1 & PID_1=$! From 913b9028649f668020850de91182d975d2a2916a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 12:51:16 +0200 Subject: [PATCH 0930/1170] 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 0931/1170] 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 0932/1170] 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 5313c9539ef8f988a607da87849eaa1f12e787e1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 6 Aug 2024 14:34:06 +0200 Subject: [PATCH 0933/1170] update documentation for Float32/64 and variants --- .../functions/type-conversion-functions.md | 420 +++++++++++++++++- 1 file changed, 416 insertions(+), 4 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..77dd1628fe4 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -3045,13 +3045,425 @@ toUInt256OrDefault('abc', CAST('0', 'UInt256')): 0 - [`toUInt256OrZero`](#touint256orzero). - [`toUInt256OrNull`](#touint256ornull). -## toFloat(32\|64) +## toFloat32 -## toFloat(32\|64)OrZero +Converts an input value to a value of type [`Float32`](../data-types/float.md). Throws an exception in case of an error. -## toFloat(32\|64)OrNull +**Syntax** -## toFloat(32\|64)OrDefault +```sql +toFloat32(expr) +``` + +**Arguments** + +- `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. +- String representations of (U)Int8/16/32/128/256. +- Values of type Float32/64, including `NaN` and `Inf`. +- String representations of Float32/64, including `NaN` and `Inf`. + +Unsupported arguments: +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat32('0xc0fe');`. + +**Returned value** + +- 32-bit floating point value. [Float32](../data-types/float.md). + +**Example** + +Query: + +```sql +SELECT + toFloat32(42.7), + toFloat32('42.7'), + toFloat32('NaN') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat32(42.7): 42.7 +toFloat32('42.7'): 42.7 +toFloat32('NaN'): nan +``` + +**See also** + +- [`toFloat32OrZero`](#tofloat32orzero). +- [`toFloat32OrNull`](#tofloat32ornull). +- [`toFloat32OrDefault`](#tofloat32ordefault). + +## toFloat32OrZero + +Like [`toFloat32`](#tofloat32), this function converts an input value to a value of type [Float32](../data-types/float.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toFloat32OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256, Float32/64. + +Unsupported arguments (return `0`): +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat32OrZero('0xc0fe');`. + +**Returned value** + +- 32-bit Float value if successful, otherwise `0`. [Float32](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat32OrZero('42.7'), + toFloat32OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat32OrZero('42.7'): 42.7 +toFloat32OrZero('abc'): 0 +``` + +**See also** + +- [`toFloat32`](#tofloat32). +- [`toFloat32OrNull`](#tofloat32ornull). +- [`toFloat32OrDefault`](#tofloat32ordefault). + +## toFloat32OrNull + +Like [`toFloat32`](#tofloat32), this function converts an input value to a value of type [Float32](../data-types/float.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toFloat32OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256, Float32/64. + +Unsupported arguments (return `\N`): +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat32OrNull('0xc0fe');`. + +**Returned value** + +- 32-bit Float value if successful, otherwise `\N`. [Float32](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat32OrNull('42.7'), + toFloat32OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat32OrNull('42.7'): 42.7 +toFloat32OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toFloat32`](#tofloat32). +- [`toFloat32OrZero`](#tofloat32orzero). +- [`toFloat32OrDefault`](#tofloat32ordefault). + +## toFloat32OrDefault + +Like [`toFloat32`](#tofloat32), this function converts an input value to a value of type [Float32](../data-types/float.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. + +**Syntax** + +```sql +toFloat32OrDefault(expr[, default]) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `default` (optional) — The default value to return if parsing to type `Float32` is unsuccessful. [Float32](../data-types/float.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- String representations of (U)Int8/16/32/128/256. +- Values of type Float32/64. +- String representations of Float32/64. + +Arguments for which the default value is returned: +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat32OrDefault('0xc0fe', CAST('0', 'Float32'));`. + +**Returned value** + +- 32-bit Float value if successful, otherwise returns the default value if passed or `0` if not. [Float32](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat32OrDefault('8', CAST('0', 'Float32')), + toFloat32OrDefault('abc', CAST('0', 'Float32')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat32OrDefault('8', CAST('0', 'Float32')): 8 +toFloat32OrDefault('abc', CAST('0', 'Float32')): 0 +``` + +**See also** + +- [`toFloat32`](#tofloat32). +- [`toFloat32OrZero`](#tofloat32orzero). +- [`toFloat32OrNull`](#tofloat32ornull). + +## toFloat64 + +Converts an input value to a value of type [`Float64`](../data-types/float.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toFloat64(expr) +``` + +**Arguments** + +- `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. +- String representations of (U)Int8/16/32/128/256. +- Values of type Float32/64, including `NaN` and `Inf`. +- String representations of type Float32/64, including `NaN` and `Inf`. + +Unsupported arguments: +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat64('0xc0fe');`. + +**Returned value** + +- 64-bit floating point value. [Float64](../data-types/float.md). + +**Example** + +Query: + +```sql +SELECT + toFloat64(42.7), + toFloat64('42.7'), + toFloat64('NaN') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat64(42.7): 42.7 +toFloat64('42.7'): 42.7 +toFloat64('NaN'): nan +``` + +**See also** + +- [`toFloat64OrZero`](#tofloat64orzero). +- [`toFloat64OrNull`](#tofloat64ornull). +- [`toFloat64OrDefault`](#tofloat64ordefault). + +## toFloat64OrZero + +Like [`toFloat64`](#tofloat64), this function converts an input value to a value of type [Float64](../data-types/float.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toFloat64OrZero(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256, Float32/64. + +Unsupported arguments (return `0`): +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat64OrZero('0xc0fe');`. + +**Returned value** + +- 64-bit Float value if successful, otherwise `0`. [Float64](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat64OrZero('42.7'), + toFloat64OrZero('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat64OrZero('42.7'): 42.7 +toFloat64OrZero('abc'): 0 +``` + +**See also** + +- [`toFloat64`](#tofloat64). +- [`toFloat64OrNull`](#tofloat64ornull). +- [`toFloat64OrDefault`](#tofloat64ordefault). + +## toFloat64OrNull + +Like [`toFloat64`](#tofloat64), this function converts an input value to a value of type [Float64](../data-types/float.md) but returns `NULL` in case of an error. + +**Syntax** + +```sql +toFloat64OrNull(x) +``` + +**Arguments** + +- `x` — A String representation of a number. [String](../data-types/string.md). + +Supported arguments: +- String representations of (U)Int8/16/32/128/256, Float32/64. + +Unsupported arguments (return `\N`): +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat64OrNull('0xc0fe');`. + +**Returned value** + +- 64-bit Float value if successful, otherwise `\N`. [Float64](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat64OrNull('42.7'), + toFloat64OrNull('abc') +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat64OrNull('42.7'): 42.7 +toFloat64OrNull('abc'): ᴺᵁᴸᴸ +``` + +**See also** + +- [`toFloat64`](#tofloat64). +- [`toFloat64OrZero`](#tofloat64orzero). +- [`toFloat64OrDefault`](#tofloat64ordefault). + +## toFloat64OrDefault + +Like [`toFloat64`](#tofloat64), this function converts an input value to a value of type [Float64](../data-types/float.md) but returns the default value in case of an error. +If no `default` value is passed then `0` is returned in case of an error. + +**Syntax** + +```sql +toFloat64OrDefault(expr[, default]) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions) / [String](../data-types/string.md). +- `default` (optional) — The default value to return if parsing to type `Float64` is unsuccessful. [Float64](../data-types/float.md). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- String representations of (U)Int8/16/32/128/256. +- Values of type Float32/64. +- String representations of Float32/64. + +Arguments for which the default value is returned: +- String representations of binary and hexadecimal values, e.g. `SELECT toFloat64OrDefault('0xc0fe', CAST('0', 'Float64'));`. + +**Returned value** + +- 64-bit Float value if successful, otherwise returns the default value if passed or `0` if not. [Float64](../data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT + toFloat64OrDefault('8', CAST('0', 'Float64')), + toFloat64OrDefault('abc', CAST('0', 'Float64')) +FORMAT vertical; +``` + +Result: + +```response +Row 1: +────── +toFloat64OrDefault('8', CAST('0', 'Float64')): 8 +toFloat64OrDefault('abc', CAST('0', 'Float64')): 0 +``` + +**See also** + +- [`toFloat64`](#tofloat64). +- [`toFloat64OrZero`](#tofloat64orzero). +- [`toFloat64OrNull`](#tofloat64ornull). ## toDate 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 0934/1170] 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 0935/1170] 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 e2eeb6f1802090c29021977abdbc46d20f22c9a1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Aug 2024 12:54:27 +0000 Subject: [PATCH 0936/1170] Update version_date.tsv and changelogs after v24.6.3.95-stable --- docs/changelogs/v24.6.3.95-stable.md | 67 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 68 insertions(+) create mode 100644 docs/changelogs/v24.6.3.95-stable.md diff --git a/docs/changelogs/v24.6.3.95-stable.md b/docs/changelogs/v24.6.3.95-stable.md new file mode 100644 index 00000000000..b90b7346e40 --- /dev/null +++ b/docs/changelogs/v24.6.3.95-stable.md @@ -0,0 +1,67 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.3.95-stable (8325c920d11) FIXME as compared to v24.6.2.17-stable (5710a8b5c0c) + +#### Improvement +* Backported in [#66770](https://github.com/ClickHouse/ClickHouse/issues/66770): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Backported in [#66885](https://github.com/ClickHouse/ClickHouse/issues/66885): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66693](https://github.com/ClickHouse/ClickHouse/issues/66693): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#67816](https://github.com/ClickHouse/ClickHouse/issues/67816): 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 [#67503](https://github.com/ClickHouse/ClickHouse/issues/67503): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67852](https://github.com/ClickHouse/ClickHouse/issues/67852): 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 [#67838](https://github.com/ClickHouse/ClickHouse/issues/67838): 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)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#66303](https://github.com/ClickHouse/ClickHouse/issues/66303): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66330](https://github.com/ClickHouse/ClickHouse/issues/66330): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66157](https://github.com/ClickHouse/ClickHouse/issues/66157): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66210](https://github.com/ClickHouse/ClickHouse/issues/66210): Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66456](https://github.com/ClickHouse/ClickHouse/issues/66456): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66228](https://github.com/ClickHouse/ClickHouse/issues/66228): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66183](https://github.com/ClickHouse/ClickHouse/issues/66183): Fix rare case with missing data in the result of distributed query, close [#61432](https://github.com/ClickHouse/ClickHouse/issues/61432). [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Backported in [#66271](https://github.com/ClickHouse/ClickHouse/issues/66271): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66682](https://github.com/ClickHouse/ClickHouse/issues/66682): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66587](https://github.com/ClickHouse/ClickHouse/issues/66587): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66362](https://github.com/ClickHouse/ClickHouse/issues/66362): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66613](https://github.com/ClickHouse/ClickHouse/issues/66613): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66577](https://github.com/ClickHouse/ClickHouse/issues/66577): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66721](https://github.com/ClickHouse/ClickHouse/issues/66721): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66670](https://github.com/ClickHouse/ClickHouse/issues/66670): Fix reading of uninitialized memory when hashing empty tuples. This closes [#66559](https://github.com/ClickHouse/ClickHouse/issues/66559). [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#66952](https://github.com/ClickHouse/ClickHouse/issues/66952): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66956](https://github.com/ClickHouse/ClickHouse/issues/66956): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66716](https://github.com/ClickHouse/ClickHouse/issues/66716): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66759](https://github.com/ClickHouse/ClickHouse/issues/66759): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66751](https://github.com/ClickHouse/ClickHouse/issues/66751): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67635](https://github.com/ClickHouse/ClickHouse/issues/67635): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67482](https://github.com/ClickHouse/ClickHouse/issues/67482): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67199](https://github.com/ClickHouse/ClickHouse/issues/67199): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67381](https://github.com/ClickHouse/ClickHouse/issues/67381): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67244](https://github.com/ClickHouse/ClickHouse/issues/67244): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#67578](https://github.com/ClickHouse/ClickHouse/issues/67578): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67808](https://github.com/ClickHouse/ClickHouse/issues/67808): 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)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#66599](https://github.com/ClickHouse/ClickHouse/issues/66599) to 24.6: Fix dropping named collection in local storage"'. [#66922](https://github.com/ClickHouse/ClickHouse/pull/66922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66332](https://github.com/ClickHouse/ClickHouse/issues/66332): Do not raise a NOT_IMPLEMENTED error when getting s3 metrics with a multiple disk configuration. [#65403](https://github.com/ClickHouse/ClickHouse/pull/65403) ([Elena Torró](https://github.com/elenatorro)). +* Backported in [#66142](https://github.com/ClickHouse/ClickHouse/issues/66142): Fix flaky test_storage_s3_queue tests. [#66009](https://github.com/ClickHouse/ClickHouse/pull/66009) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#66389](https://github.com/ClickHouse/ClickHouse/issues/66389): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66428](https://github.com/ClickHouse/ClickHouse/issues/66428): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66546](https://github.com/ClickHouse/ClickHouse/issues/66546): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66861](https://github.com/ClickHouse/ClickHouse/issues/66861): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66877](https://github.com/ClickHouse/ClickHouse/issues/66877): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67061](https://github.com/ClickHouse/ClickHouse/issues/67061): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66940](https://github.com/ClickHouse/ClickHouse/issues/66940): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67254](https://github.com/ClickHouse/ClickHouse/issues/67254): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67414](https://github.com/ClickHouse/ClickHouse/issues/67414): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 75c10fa67b8..e410f31ca5a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 From dbfba5ebc4d011d13ab78f91700fae12b46ce58b Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 12:56:52 +0000 Subject: [PATCH 0937/1170] Validate data types in ALTER ADD/MODIFY COLUMN --- src/Storages/AlterCommands.cpp | 5 +++++ ..._type_in_alter_add_modify_column.reference | 0 ...lidate_type_in_alter_add_modify_column.sql | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.reference create mode 100644 tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..dfb388ffdb2 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -1316,6 +1317,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception(ErrorCodes::BAD_ARGUMENTS, "Data type have to be specified for column {} to add", backQuote(column_name)); + validateDataType(command.data_type, DataTypeValidationSettings(context->getSettingsRef())); + /// FIXME: Adding a new column of type Object(JSON) is broken. /// Looks like there is something around default expression for this column (method `getDefault` is not implemented for the data type Object). /// But after ALTER TABLE ADD COLUMN we need to fill existing rows with something (exactly the default value). @@ -1395,6 +1398,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const /// So we don't allow to do it for now. if (command.data_type) { + validateDataType(command.data_type, DataTypeValidationSettings(context->getSettingsRef())); + const GetColumnsOptions options(GetColumnsOptions::All); const auto old_data_type = all_columns.getColumn(options, column_name).type; diff --git a/tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.reference b/tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.sql b/tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.sql new file mode 100644 index 00000000000..267bc7111f4 --- /dev/null +++ b/tests/queries/0_stateless/03215_validate_type_in_alter_add_modify_column.sql @@ -0,0 +1,19 @@ +set allow_experimental_variant_type = 0; +set allow_experimental_dynamic_type = 0; +set allow_suspicious_low_cardinality_types = 0; +set allow_suspicious_fixed_string_types = 0; + +drop table if exists test; +create table test (id UInt64) engine=MergeTree order by id; +alter table test add column bad Variant(UInt32, String); -- {serverError ILLEGAL_COLUMN} +alter table test add column bad Dynamic; -- {serverError ILLEGAL_COLUMN} +alter table test add column bad LowCardinality(UInt8); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} +alter table test add column bad FixedString(10000); -- {serverError ILLEGAL_COLUMN} + +alter table test modify column id Variant(UInt32, String); -- {serverError ILLEGAL_COLUMN} +alter table test modify column id Dynamic; -- {serverError ILLEGAL_COLUMN} +alter table test modify column id LowCardinality(UInt8); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} +alter table test modify column id FixedString(10000); -- {serverError ILLEGAL_COLUMN} + +drop table test; + From f3ee25036f9c5796a9018699d575f94bf75a50b5 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 2 May 2024 17:39:54 +0000 Subject: [PATCH 0938/1170] 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 e0e32b542560aeefa6863195d0207859ec5f338e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 13:26:12 +0000 Subject: [PATCH 0939/1170] Fix tests --- .../queries/0_stateless/02910_object-json-crash-add-column.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02910_object-json-crash-add-column.sql b/tests/queries/0_stateless/02910_object-json-crash-add-column.sql index b2d64be1676..bda5e958453 100644 --- a/tests/queries/0_stateless/02910_object-json-crash-add-column.sql +++ b/tests/queries/0_stateless/02910_object-json-crash-add-column.sql @@ -1,3 +1,5 @@ +SET allow_experimental_object_type=1; + DROP TABLE IF EXISTS test02910; CREATE TABLE test02910 From f37fcb776800cdb6bff4d89e5814c39bbbc46033 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Aug 2024 14:32:20 +0100 Subject: [PATCH 0940/1170] impl --- src/Compression/fuzzers/CMakeLists.txt | 12 ++++++------ src/Storages/fuzzers/CMakeLists.txt | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Compression/fuzzers/CMakeLists.txt b/src/Compression/fuzzers/CMakeLists.txt index a693faecc14..311f1eb3d35 100644 --- a/src/Compression/fuzzers/CMakeLists.txt +++ b/src/Compression/fuzzers/CMakeLists.txt @@ -5,19 +5,19 @@ # If you want really small size of the resulted binary, just link with fuzz_compression and clickhouse_common_io clickhouse_add_executable (compressed_buffer_fuzzer compressed_buffer_fuzzer.cpp) -target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms) +target_link_libraries (compressed_buffer_fuzzer PRIVATE dbms clickhouse_functions) clickhouse_add_executable (lz4_decompress_fuzzer lz4_decompress_fuzzer.cpp) -target_link_libraries (lz4_decompress_fuzzer PUBLIC dbms ch_contrib::lz4) +target_link_libraries (lz4_decompress_fuzzer PUBLIC dbms ch_contrib::lz4 clickhouse_functions) clickhouse_add_executable (delta_decompress_fuzzer delta_decompress_fuzzer.cpp) -target_link_libraries (delta_decompress_fuzzer PRIVATE dbms) +target_link_libraries (delta_decompress_fuzzer PRIVATE dbms clickhouse_functions) clickhouse_add_executable (double_delta_decompress_fuzzer double_delta_decompress_fuzzer.cpp) -target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms) +target_link_libraries (double_delta_decompress_fuzzer PRIVATE dbms clickhouse_functions) clickhouse_add_executable (encrypted_decompress_fuzzer encrypted_decompress_fuzzer.cpp) -target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms) +target_link_libraries (encrypted_decompress_fuzzer PRIVATE dbms clickhouse_functions) clickhouse_add_executable (gcd_decompress_fuzzer gcd_decompress_fuzzer.cpp) -target_link_libraries (gcd_decompress_fuzzer PRIVATE dbms) +target_link_libraries (gcd_decompress_fuzzer PRIVATE dbms clickhouse_functions) diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index e36fccec8df..ec56b853666 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -1,7 +1,7 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.cpp) # Look at comment around fuzz_compression target declaration -target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms) +target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms clickhouse_functions) clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) target_link_libraries (columns_description_fuzzer PRIVATE dbms clickhouse_functions) From dbcc5cf1333c6a3d8f5f8f1b7dc67d038055be2e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 15:39:08 +0200 Subject: [PATCH 0941/1170] CI: Changelog: Critical Bug Fix to Bug Fix --- tests/ci/changelog.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 929f0f3523a..fe47fe53a9e 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -38,7 +38,7 @@ categories_preferred_order = ( "Experimental Feature", "Performance Improvement", "Improvement", - "Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", + #"Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", "Bug Fix (user-visible misbehavior in an official stable release)", "Build/Testing/Packaging Improvement", "Other", @@ -294,7 +294,8 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri r"(?i)bug\Wfix", category, ) - and "Critical Bug Fix" not in category + # Map "Critical Bug Fix" to "Bug fix" category for changelog + #and "Critical Bug Fix" not in category ): category = "Bug Fix (user-visible misbehavior in an official stable release)" From b99c6c1153224b8d17dc663939b7c621dfe77ceb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Aug 2024 14:02:24 +0000 Subject: [PATCH 0942/1170] fix reading of size column from missed Nested in compact parts --- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 12 +++++++++++- src/Storages/MergeTree/MergeTreeReaderCompact.h | 3 ++- .../MergeTree/MergeTreeReaderCompactSingleBuffer.cpp | 7 ++++++- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index ff0311dc1ca..d49ad61feca 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -147,7 +147,8 @@ void MergeTreeReaderCompact::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, size_t rows_to_read, - const InputStreamGetter & getter) + const InputStreamGetter & getter, + ISerialization::SubstreamsCache & cache) { try { @@ -158,6 +159,13 @@ void MergeTreeReaderCompact::readData( deserialize_settings.getter = getter; deserialize_settings.avg_value_size_hint = avg_value_size_hints[name]; + auto it = cache.find(name); + if (it != cache.end() && it->second != nullptr) + { + column = it->second; + return; + } + if (name_and_type.isSubcolumn()) { const auto & type_in_storage = name_and_type.getTypeInStorage(); @@ -181,6 +189,8 @@ void MergeTreeReaderCompact::readData( serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, deserialize_binary_bulk_state_map[name], nullptr); } + cache.emplace(name, column); + size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column != rows_to_read) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 22eabd47930..1c6bd1474e3 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -44,7 +44,8 @@ protected: const NameAndTypePair & name_and_type, ColumnPtr & column, size_t rows_to_read, - const InputStreamGetter & getter); + const InputStreamGetter & getter, + ISerialization::SubstreamsCache & cache); void readPrefix( const NameAndTypePair & name_and_type, diff --git a/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp b/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp index 2b2cf493bb5..004ba4db028 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp @@ -22,6 +22,10 @@ try checkNumberOfColumns(num_columns); createColumnsForReading(res_columns); + /// Use cache to avoid reading the column with the same name twice. + /// It may happen if there are empty array Nested in the part. + std::unordered_map caches; + while (read_rows < max_rows_to_read) { size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark); @@ -32,6 +36,7 @@ try continue; auto & column = res_columns[pos]; + auto & cache = caches[columns_to_read[pos].name]; stream->adjustRightMark(current_task_last_mark); /// Must go before seek. stream->seekToMarkAndColumn(from_mark, *column_positions[pos]); @@ -52,7 +57,7 @@ try }; readPrefix(columns_to_read[pos], buffer_getter, buffer_getter_for_prefix, columns_for_offsets[pos]); - readData(columns_to_read[pos], column, rows_to_read, buffer_getter); + readData(columns_to_read[pos], column, rows_to_read, buffer_getter, cache); } ++from_mark; From 99b18d31db32a077678661bd9ba84fb52ff49333 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 15:30:38 +0200 Subject: [PATCH 0943/1170] 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 0944/1170] 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 0945/1170] 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 0946/1170] 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 5c4f2c198573ba03b793e4610f4e9854a61a5543 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 6 Aug 2024 14:33:28 +0000 Subject: [PATCH 0947/1170] Fix DateTime64 parsing after constant folding --- src/Analyzer/ConstantNode.cpp | 22 ++++++++++++++++++- ...03217_datetime64_constant_to_ast.reference | 2 ++ .../03217_datetime64_constant_to_ast.sql | 6 +++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference create mode 100644 tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 46c1f7fb1ed..c65090f5b55 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -162,6 +163,7 @@ QueryTreeNodePtr ConstantNode::cloneImpl() const ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const { const auto & constant_value_literal = constant_value->getValue(); + const auto & constant_value_type = constant_value->getType(); auto constant_value_ast = std::make_shared(constant_value_literal); if (!options.add_cast_for_constants) @@ -169,7 +171,25 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const if (requiresCastCall()) { - auto constant_type_name_ast = std::make_shared(constant_value->getType()->getName()); + /** Value for DateTime64 is Decimal64, which is serialized as a string literal. + * If we serialize it as is, DateTime64 would be parsed from that string literal, which can be incorrect. + * For example, DateTime64 cannot be parsed from the short value, like '1', while it's a valid Decimal64 value. + * It could also lead to ambiguous parsing because we don't know if the string literal represents a date or a Decimal64 literal. + * For this reason, we use a string literal representing a date instead of a Decimal64 literal. + */ + if (WhichDataType(constant_value_type->getTypeId()).isDateTime64()) + { + const auto * date_time_type = typeid_cast(constant_value_type.get()); + DecimalField decimal_value; + if (constant_value_literal.tryGet>(decimal_value)) + { + WriteBufferFromOwnString ostr; + writeDateTimeText(decimal_value.getValue(), date_time_type->getScale(), ostr, date_time_type->getTimeZone()); + constant_value_ast = std::make_shared(ostr.str()); + } + } + + auto constant_type_name_ast = std::make_shared(constant_value_type->getName()); return makeASTFunction("_CAST", std::move(constant_value_ast), std::move(constant_type_name_ast)); } diff --git a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference new file mode 100644 index 00000000000..c20baa0d261 --- /dev/null +++ b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference @@ -0,0 +1,2 @@ +1970-01-01 00:00:01.000 +1970-01-01 00:00:01.000 diff --git a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql new file mode 100644 index 00000000000..63334a511c7 --- /dev/null +++ b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql @@ -0,0 +1,6 @@ + +SET session_timezone = 'UTC'; + +SELECT toDateTime64('1970-01-01 00:00:01', 3) +FROM remote('127.0.0.{1,2}', system, one) +; 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 0948/1170] 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 0949/1170] 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 0950/1170] 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 0951/1170] 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 0952/1170] 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 0953/1170] 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 0954/1170] 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 0955/1170] 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 0956/1170] 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 0957/1170] 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 0958/1170] 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 0959/1170] 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 f2591bd1a6401337de0e3bf8d5c1fa93fd1b9394 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 16:55:04 +0200 Subject: [PATCH 0960/1170] CI: CiBuddy with channel dispatcher --- tests/ci/ci_buddy.py | 102 ++++++++++++++++++++++++++++--------- tests/ci/create_release.py | 4 +- 2 files changed, 81 insertions(+), 25 deletions(-) diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index f0e73e925fe..07f318207a4 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -1,7 +1,7 @@ import argparse import json import os -from typing import Union, Dict +from typing import Union, Dict, List import boto3 import requests @@ -9,20 +9,44 @@ from botocore.exceptions import ClientError from pr_info import PRInfo from ci_config import CI +from ci_utils import WithIter + + +class Channels(metaclass=WithIter): + # Channel names must match json keys in ParameterStore + ALERTS = "alerts-channel" + INFO = "info-channel" + DRY_RUN = "dry-ryn-channel" + DEFAULT = "default" class CIBuddy: + Channels = Channels _HEADERS = {"Content-Type": "application/json"} def __init__(self, dry_run=False): self.repo = os.getenv("GITHUB_REPOSITORY", "") self.dry_run = dry_run res = self._get_webhooks() - self.test_channel = "" - self.dev_ci_channel = "" + self.channels = {} if res: - self.test_channel = json.loads(res)["test_channel"] - self.dev_ci_channel = json.loads(res)["ci_channel"] + channels = json.loads(res) + for channel in Channels: + if channel in channels: + self.channels[channel] = channels[channel] + + for channel in Channels: + if channel not in self.channels: + if Channels.DEFAULT in self.channels: + print( + f"ERROR: missing config for channel [{channel}] - will use default channel instead" + ) + self.channels[channel] = self.channels[Channels.DEFAULT] + else: + print( + f"ERROR: missing config for channel [{channel}] - will disable notification" + ) + self.channels[channel] = "" self.job_name = os.getenv("CHECK_NAME", "unknown") pr_info = PRInfo() self.pr_number = pr_info.number @@ -63,22 +87,33 @@ class CIBuddy: return json_string - def post(self, message, dry_run=None): - if dry_run is None: - dry_run = self.dry_run - print(f"Posting slack message, dry_run [{dry_run}]") - if dry_run: - url = self.test_channel + def post(self, message: str, channels: List[str]) -> None: + print(f"Posting slack message, dry_run [{self.dry_run}]") + if self.dry_run: + urls = [self.channels[Channels.DRY_RUN]] else: - url = self.dev_ci_channel + urls = [] + for channel in channels: + url = self.channels[channel] + if url: + urls.append(url) + else: + print(f"WARNING: no channel config for [{channel}] - skip") data = {"text": message} try: - requests.post(url, headers=self._HEADERS, data=json.dumps(data), timeout=10) + for url in urls: + requests.post( + url, headers=self._HEADERS, data=json.dumps(data), timeout=10 + ) except Exception as e: print(f"ERROR: Failed to post message, ex {e}") def _post_formatted( - self, title: str, body: Union[Dict, str], with_wf_link: bool + self, + title: str, + body: Union[Dict, str], + with_wf_link: bool, + channels: Union[List[str], str], ) -> None: message = title if isinstance(body, dict): @@ -96,31 +131,49 @@ class CIBuddy: run_id = os.getenv("GITHUB_RUN_ID", "") if with_wf_link and run_id: message += f" *workflow*: \n" - self.post(message) + self.post( + message, channels=[channels] if isinstance(channels, str) else channels + ) def post_info( - self, title: str, body: Union[Dict, str], with_wf_link: bool = True + self, + title: str, + body: Union[Dict, str], + with_wf_link: bool = True, + channels: Union[List[str], str] = Channels.INFO, ) -> None: title_extended = f":white_circle: *{title}*\n\n" - self._post_formatted(title_extended, body, with_wf_link) + self._post_formatted(title_extended, body, with_wf_link, channels=channels) def post_done( - self, title: str, body: Union[Dict, str], with_wf_link: bool = True + self, + title: str, + body: Union[Dict, str], + with_wf_link: bool = True, + channels: Union[List[str], str] = Channels.INFO, ) -> None: title_extended = f":white_check_mark: *{title}*\n\n" - self._post_formatted(title_extended, body, with_wf_link) + self._post_formatted(title_extended, body, with_wf_link, channels=channels) def post_warning( - self, title: str, body: Union[Dict, str], with_wf_link: bool = True + self, + title: str, + body: Union[Dict, str], + with_wf_link: bool = True, + channels: Union[List[str], str] = Channels.ALERTS, ) -> None: title_extended = f":warning: *{title}*\n\n" - self._post_formatted(title_extended, body, with_wf_link) + self._post_formatted(title_extended, body, with_wf_link, channels=channels) def post_critical( - self, title: str, body: Union[Dict, str], with_wf_link: bool = True + self, + title: str, + body: Union[Dict, str], + with_wf_link: bool = True, + channels: Union[List[str], str] = Channels.ALERTS, ) -> None: title_extended = f":black_circle: *{title}*\n\n" - self._post_formatted(title_extended, body, with_wf_link) + self._post_formatted(title_extended, body, with_wf_link, channels=channels) def post_job_error( self, @@ -129,6 +182,7 @@ class CIBuddy: with_instance_info: bool = True, with_wf_link: bool = True, critical: bool = False, + channel: Union[List[str], str] = Channels.ALERTS, ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: @@ -159,7 +213,7 @@ class CIBuddy: run_id = os.getenv("GITHUB_RUN_ID", "") if with_wf_link and run_id: message += f" *workflow*: \n" - self.post(message) + self.post(message, channels=[channel] if isinstance(channel, str) else channel) def parse_args(): diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index b4e08f29dbe..27eba273ce0 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -821,7 +821,9 @@ if __name__ == "__main__": else: title = "Failed: " + title CIBuddy(dry_run=args.dry_run).post_critical( - title, dataclasses.asdict(release_info) + title, + dataclasses.asdict(release_info), + channels=[CIBuddy.Channels.ALERTS, CIBuddy.Channels.INFO], ) if args.set_progress_started: From e1e298f14e34a519d59844417e31b7a532b0a157 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Aug 2024 14:56:54 +0000 Subject: [PATCH 0961/1170] Automatic style fix --- tests/ci/changelog.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index fe47fe53a9e..39e426945d3 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -38,7 +38,7 @@ categories_preferred_order = ( "Experimental Feature", "Performance Improvement", "Improvement", - #"Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", + # "Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", "Bug Fix (user-visible misbehavior in an official stable release)", "Build/Testing/Packaging Improvement", "Other", @@ -295,7 +295,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri category, ) # Map "Critical Bug Fix" to "Bug fix" category for changelog - #and "Critical Bug Fix" not in category + # and "Critical Bug Fix" not in category ): category = "Bug Fix (user-visible misbehavior in an official stable release)" From bdfaffa9d74b0405ea619330215e6c71f0b6976a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:59:23 +0200 Subject: [PATCH 0962/1170] tests: make test_distributed_inter_server_secret idempotent Signed-off-by: Azat Khuzhin --- .../test.py | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 36d7e044f1c..d74cf97e5c6 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -46,6 +46,10 @@ users = pytest.mark.parametrize( ) +def generate_query_id(): + return str(uuid.uuid4()) + + def bootstrap(): for n in list(cluster.instances.values()): n.query("DROP TABLE IF EXISTS data") @@ -268,13 +272,13 @@ def test_secure_insert_buffer_async(): n1.query("SYSTEM RELOAD CONFIG") # ensure that SELECT creates new connection (we need separate table for # this, so that separate distributed pool will be used) - query_id = uuid.uuid4().hex + query_id = generate_query_id() n1.query("SELECT * FROM dist_secure_from_buffer", user="ro", query_id=query_id) assert n1.contains_in_log( "{" + query_id + "} Connection (n2:9000): Connecting." ) - query_id = uuid.uuid4().hex + query_id = generate_query_id() n1.query( "INSERT INTO dist_secure_buffer SELECT * FROM numbers(2)", query_id=query_id ) @@ -331,7 +335,7 @@ def test_secure_disagree_insert(): @users def test_user_insecure_cluster(user, password): - id_ = "query-dist_insecure-" + user + id_ = "query-dist_insecure-" + user + "-" + generate_query_id() n1.query(f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password) assert get_query_user_info(n1, id_)[0] == [ user, @@ -342,7 +346,7 @@ def test_user_insecure_cluster(user, password): @users def test_user_secure_cluster(user, password): - id_ = "query-dist_secure-" + user + id_ = "query-dist_secure-" + user + "-" + generate_query_id() n1.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) assert get_query_user_info(n1, id_)[0] == [user, user] assert get_query_user_info(n2, id_)[0] == [user, user] @@ -350,7 +354,7 @@ def test_user_secure_cluster(user, password): @users def test_per_user_inline_settings_insecure_cluster(user, password): - id_ = "query-ddl-settings-dist_insecure-" + user + id_ = "query-ddl-settings-dist_insecure-" + user + "-" + generate_query_id() n1.query( f""" SELECT *, '{id_}' FROM dist_insecure @@ -367,7 +371,7 @@ def test_per_user_inline_settings_insecure_cluster(user, password): @users def test_per_user_inline_settings_secure_cluster(user, password): - id_ = "query-ddl-settings-dist_secure-" + user + id_ = "query-ddl-settings-dist_secure-" + user + "-" + generate_query_id() n1.query( f""" SELECT *, '{id_}' FROM dist_secure @@ -386,7 +390,7 @@ def test_per_user_inline_settings_secure_cluster(user, password): @users def test_per_user_protocol_settings_insecure_cluster(user, password): - id_ = "query-protocol-settings-dist_insecure-" + user + id_ = "query-protocol-settings-dist_insecure-" + user + "-" + generate_query_id() n1.query( f"SELECT *, '{id_}' FROM dist_insecure", user=user, @@ -402,7 +406,7 @@ def test_per_user_protocol_settings_insecure_cluster(user, password): @users def test_per_user_protocol_settings_secure_cluster(user, password): - id_ = "query-protocol-settings-dist_secure-" + user + id_ = "query-protocol-settings-dist_secure-" + user + "-" + generate_query_id() n1.query( f"SELECT *, '{id_}' FROM dist_secure", user=user, @@ -436,7 +440,7 @@ def test_secure_cluster_distributed_over_distributed_different_users_remote(): def test_secure_cluster_distributed_over_distributed_different_users_cluster(): - id_ = "cluster-user" + id_ = "cluster-user" + "-" + generate_query_id() n1.query( f"SELECT *, '{id_}' FROM cluster(secure, currentDatabase(), dist_secure)", user="nopass", From df2675fad0d1bcb79c8a2d7edd0c08b1da49a945 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 18 Jul 2024 12:32:16 +0200 Subject: [PATCH 0963/1170] [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 0964/1170] 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 47953da08dbc791d521b4e722c08af08b9072b89 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Aug 2024 15:08:40 +0000 Subject: [PATCH 0965/1170] fix reading of size column from missed Nested in compact parts --- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderCompactSingleBuffer.cpp | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index d49ad61feca..69dc2e4b2bb 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -189,7 +189,7 @@ void MergeTreeReaderCompact::readData( serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, deserialize_binary_bulk_state_map[name], nullptr); } - cache.emplace(name, column); + cache[name] = column; size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column != rows_to_read) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp b/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp index 004ba4db028..649bcce1188 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp @@ -22,21 +22,20 @@ try checkNumberOfColumns(num_columns); createColumnsForReading(res_columns); - /// Use cache to avoid reading the column with the same name twice. - /// It may happen if there are empty array Nested in the part. - std::unordered_map caches; - while (read_rows < max_rows_to_read) { size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark); + /// Use cache to avoid reading the column with the same name twice. + /// It may happen if there are empty array Nested in the part. + ISerialization::SubstreamsCache cache; + for (size_t pos = 0; pos < num_columns; ++pos) { if (!res_columns[pos]) continue; auto & column = res_columns[pos]; - auto & cache = caches[columns_to_read[pos].name]; stream->adjustRightMark(current_task_last_mark); /// Must go before seek. stream->seekToMarkAndColumn(from_mark, *column_positions[pos]); From 0bb8d1a8ede4906a1cdc75af2dbf1e190ff355fc Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 6 Aug 2024 17:10:34 +0200 Subject: [PATCH 0966/1170] optimize formatting by 3% --- .../functions/type-conversion-functions.md | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 77dd1628fe4..a2b6e496319 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -3063,7 +3063,7 @@ Supported arguments: - Values of type (U)Int8/16/32/64/128/256. - String representations of (U)Int8/16/32/128/256. - Values of type Float32/64, including `NaN` and `Inf`. -- String representations of Float32/64, including `NaN` and `Inf`. +- String representations of Float32/64, including `NaN` and `Inf` (case-insensitive). Unsupported arguments: - String representations of binary and hexadecimal values, e.g. `SELECT toFloat32('0xc0fe');`. @@ -3081,7 +3081,7 @@ SELECT toFloat32(42.7), toFloat32('42.7'), toFloat32('NaN') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3128,11 +3128,11 @@ Unsupported arguments (return `0`): Query: -``` sql +```sql SELECT toFloat32OrZero('42.7'), toFloat32OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3178,11 +3178,11 @@ Unsupported arguments (return `\N`): Query: -``` sql +```sql SELECT toFloat32OrNull('42.7'), toFloat32OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3219,8 +3219,8 @@ toFloat32OrDefault(expr[, default]) Supported arguments: - Values of type (U)Int8/16/32/64/128/256. - String representations of (U)Int8/16/32/128/256. -- Values of type Float32/64. -- String representations of Float32/64. +- Values of type Float32/64, including `NaN` and `Inf`. +- String representations of Float32/64, including `NaN` and `Inf` (case-insensitive). Arguments for which the default value is returned: - String representations of binary and hexadecimal values, e.g. `SELECT toFloat32OrDefault('0xc0fe', CAST('0', 'Float32'));`. @@ -3233,11 +3233,11 @@ Arguments for which the default value is returned: Query: -``` sql +```sql SELECT toFloat32OrDefault('8', CAST('0', 'Float32')), toFloat32OrDefault('abc', CAST('0', 'Float32')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3273,7 +3273,7 @@ Supported arguments: - Values of type (U)Int8/16/32/64/128/256. - String representations of (U)Int8/16/32/128/256. - Values of type Float32/64, including `NaN` and `Inf`. -- String representations of type Float32/64, including `NaN` and `Inf`. +- String representations of type Float32/64, including `NaN` and `Inf` (case-insensitive). Unsupported arguments: - String representations of binary and hexadecimal values, e.g. `SELECT toFloat64('0xc0fe');`. @@ -3291,7 +3291,7 @@ SELECT toFloat64(42.7), toFloat64('42.7'), toFloat64('NaN') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3338,11 +3338,11 @@ Unsupported arguments (return `0`): Query: -``` sql +```sql SELECT toFloat64OrZero('42.7'), toFloat64OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3388,11 +3388,11 @@ Unsupported arguments (return `\N`): Query: -``` sql +```sql SELECT toFloat64OrNull('42.7'), toFloat64OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3429,8 +3429,8 @@ toFloat64OrDefault(expr[, default]) Supported arguments: - Values of type (U)Int8/16/32/64/128/256. - String representations of (U)Int8/16/32/128/256. -- Values of type Float32/64. -- String representations of Float32/64. +- Values of type Float32/64, including `NaN` and `Inf`. +- String representations of Float32/64, including `NaN` and `Inf` (case-insensitive). Arguments for which the default value is returned: - String representations of binary and hexadecimal values, e.g. `SELECT toFloat64OrDefault('0xc0fe', CAST('0', 'Float64'));`. @@ -3443,11 +3443,11 @@ Arguments for which the default value is returned: Query: -``` sql +```sql SELECT toFloat64OrDefault('8', CAST('0', 'Float64')), toFloat64OrDefault('abc', CAST('0', 'Float64')) -FORMAT vertical; +FORMAT Vertical; ``` Result: From 5390d1b108956907bf4b038a56fdb2ed8e584308 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:12:44 +0200 Subject: [PATCH 0967/1170] 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 40e763dd8b79b20a56b9a5cbff9571d7d9ed8869 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 6 Aug 2024 17:17:02 +0200 Subject: [PATCH 0968/1170] more logs to debug logical error from async inserts --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index ee3ac4207cc..cb02f1cf5f2 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -240,6 +240,14 @@ std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfo ++offset_idx; } } + if (offset_idx != async_insert_info->offsets.size()) + { + LOG_ERROR( + getLogger("MergeTreeDataWriter"), + "ChunkInfo of async insert offsets doesn't match the selector size {}. Offsets content is ({})", + selector.size(), fmt::join(async_insert_info->offsets.begin(), async_insert_info->offsets.end(), ",")); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error for async deduplicated insert, please check error logs"); + } return result; } @@ -317,8 +325,10 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( { if (async_insert_info_with_partition[i] == nullptr) { - LOG_ERROR(getLogger("MergeTreeDataWriter"), "The {}th element in async_insert_info_with_partition is nullptr. There are totally {} partitions in the insert. Selector content is {}", - i, partitions_count, fmt::join(selector.begin(), selector.end(), ",")); + LOG_ERROR( + getLogger("MergeTreeDataWriter"), + "The {}th element in async_insert_info_with_partition is nullptr. There are totally {} partitions in the insert. Selector content is ({}). Offsets content is ({})", + i, partitions_count, fmt::join(selector.begin(), selector.end(), ","), fmt::join(async_insert_info->offsets.begin(), async_insert_info->offsets.end(), ",")); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error for async deduplicated insert, please check error logs"); } result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); From 54ba7703b1ccd116dceefe8b0e9c6aca5c24d212 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 15:16:43 +0000 Subject: [PATCH 0969/1170] 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 2d93910bd601f7a4f8cc9385a2d191ce806453e3 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Tue, 6 Aug 2024 17:44:29 +0200 Subject: [PATCH 0970/1170] added tags no-fasttest --- tests/queries/0_stateless/03215_parquet_index.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03215_parquet_index.sql b/tests/queries/0_stateless/03215_parquet_index.sql index 0afccdf6f3b..5766f40f021 100644 --- a/tests/queries/0_stateless/03215_parquet_index.sql +++ b/tests/queries/0_stateless/03215_parquet_index.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + -- default settings. DROP TABLE IF EXISTS test_parquet; CREATE TABLE test_parquet (col1 int, col2 String) ENGINE=File(Parquet); From 1e631472f0aabcb1b856942fb7409c5c010315a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:07:49 +0200 Subject: [PATCH 0971/1170] Make 01685_ssd_cache_dictionary_complex_key parallelizable --- .../01685_ssd_cache_dictionary_complex_key.sh | 81 +++++++++---------- 1 file changed, 38 insertions(+), 43 deletions(-) diff --git a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh index 5583a9dd5e7..55061b9a643 100755 --- a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh +++ b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh @@ -1,15 +1,11 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n --query=" - DROP DATABASE IF EXISTS 01685_database_for_cache_dictionary; - CREATE DATABASE 01685_database_for_cache_dictionary; - - CREATE TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table + CREATE TABLE complex_key_simple_attributes_source_table ( id UInt64, id_key String, @@ -18,11 +14,11 @@ $CLICKHOUSE_CLIENT -n --query=" ) ENGINE = TinyLog; - INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); - INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1'); - INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + INSERT INTO complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); + INSERT INTO complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1'); + INSERT INTO complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); - CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes + CREATE DICTIONARY cache_dictionary_complex_key_simple_attributes ( id UInt64, id_key String, @@ -30,32 +26,32 @@ $CLICKHOUSE_CLIENT -n --query=" value_second String DEFAULT 'value_second_default' ) PRIMARY KEY id, id_key - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01685_database_for_cache_dictionary')) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '${CLICKHOUSE_DATABASE}')) LIFETIME(MIN 1 MAX 1000) - LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d')); + LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/${CLICKHOUSE_DATABASE}_dic')); SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes'; SELECT 'dictGet existing value'; - SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, - dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; + SELECT dictGet('cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; SELECT 'dictGet with non existing value'; - SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, - dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; + SELECT dictGet('cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; SELECT 'dictGetOrDefault existing value'; - SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, - dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; + SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; SELECT 'dictGetOrDefault non existing value'; - SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, - dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; + SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; SELECT 'dictHas'; - SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; + SELECT dictHas('cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; SELECT 'select all values as input stream'; - SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id; + SELECT * FROM cache_dictionary_complex_key_simple_attributes ORDER BY id; - DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes; - DROP TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table; + DROP DICTIONARY cache_dictionary_complex_key_simple_attributes; + DROP TABLE complex_key_simple_attributes_source_table; - CREATE TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table + CREATE TABLE complex_key_complex_attributes_source_table ( id UInt64, id_key String, @@ -64,11 +60,11 @@ $CLICKHOUSE_CLIENT -n --query=" ) ENGINE = TinyLog; - INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); - INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL); - INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + INSERT INTO complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); + INSERT INTO complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL); + INSERT INTO complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); - CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes + CREATE DICTIONARY cache_dictionary_complex_key_complex_attributes ( id UInt64, id_key String, @@ -77,29 +73,28 @@ $CLICKHOUSE_CLIENT -n --query=" value_second Nullable(String) DEFAULT 'value_second_default' ) PRIMARY KEY id, id_key - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01685_database_for_cache_dictionary')) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '${CLICKHOUSE_DATABASE}')) LIFETIME(MIN 1 MAX 1000) LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/1d')); SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes'; SELECT 'dictGet existing value'; - SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, - dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; + SELECT dictGet('cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; SELECT 'dictGet with non existing value'; - SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, - dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; + SELECT dictGet('cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; SELECT 'dictGetOrDefault existing value'; - SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, - dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; + SELECT dictGetOrDefault('cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; SELECT 'dictGetOrDefault non existing value'; - SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, - dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; + SELECT dictGetOrDefault('cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; SELECT 'dictHas'; - SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; + SELECT dictHas('cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; SELECT 'select all values as input stream'; - SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id; + SELECT * FROM cache_dictionary_complex_key_complex_attributes ORDER BY id; - DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes; - DROP TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table; - - DROP DATABASE 01685_database_for_cache_dictionary;" + DROP DICTIONARY cache_dictionary_complex_key_complex_attributes; + DROP TABLE complex_key_complex_attributes_source_table; +" From 38d891c6befdacfa835005b522f104e9428a270b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:18:09 +0200 Subject: [PATCH 0972/1170] Make 02015_column_default_dict_get_identifier parallelizable --- ...015_column_default_dict_get_identifier.sql | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02015_column_default_dict_get_identifier.sql b/tests/queries/0_stateless/02015_column_default_dict_get_identifier.sql index 24a3b631388..046d0c42180 100644 --- a/tests/queries/0_stateless/02015_column_default_dict_get_identifier.sql +++ b/tests/queries/0_stateless/02015_column_default_dict_get_identifier.sql @@ -1,9 +1,6 @@ -- Tags: no-parallel -DROP DATABASE IF EXISTS 02015_db; -CREATE DATABASE 02015_db; - -CREATE TABLE 02015_db.test_table +CREATE TABLE test_table ( key_column UInt64, data_column_1 UInt64, @@ -12,9 +9,9 @@ CREATE TABLE 02015_db.test_table ENGINE = MergeTree ORDER BY key_column; -INSERT INTO 02015_db.test_table VALUES (0, 0, 0); +INSERT INTO test_table VALUES (0, 0, 0); -CREATE DICTIONARY 02015_db.test_dictionary +CREATE DICTIONARY test_dictionary ( key_column UInt64 DEFAULT 0, data_column_1 UInt64 DEFAULT 1, @@ -22,20 +19,18 @@ CREATE DICTIONARY 02015_db.test_dictionary ) PRIMARY KEY key_column LAYOUT(DIRECT()) -SOURCE(CLICKHOUSE(DB '02015_db' TABLE 'test_table')); +SOURCE(CLICKHOUSE(DB currentDatabase() TABLE 'test_table')); -CREATE TABLE 02015_db.test_table_default +CREATE TABLE test_table_default ( - data_1 DEFAULT dictGetUInt64('02015_db.test_dictionary', 'data_column_1', toUInt64(0)), - data_2 DEFAULT dictGet(02015_db.test_dictionary, 'data_column_2', toUInt64(0)) + data_1 DEFAULT dictGetUInt64('test_dictionary', 'data_column_1', toUInt64(0)), + data_2 DEFAULT dictGet(test_dictionary, 'data_column_2', toUInt64(0)) ) ENGINE=TinyLog; -INSERT INTO 02015_db.test_table_default(data_1) VALUES (5); -SELECT * FROM 02015_db.test_table_default; +INSERT INTO test_table_default(data_1) VALUES (5); +SELECT * FROM test_table_default; -DROP TABLE 02015_db.test_table_default; -DROP DICTIONARY 02015_db.test_dictionary; -DROP TABLE 02015_db.test_table; - -DROP DATABASE 02015_db; +DROP TABLE test_table_default; +DROP DICTIONARY test_dictionary; +DROP TABLE test_table; From 425fb0f485b1936ebfa46f0ae9db1e7ea134272f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:20:09 +0200 Subject: [PATCH 0973/1170] Make 01600_multiple_left_join_with_aliases parallelizable --- .../01600_multiple_left_join_with_aliases.sql | 25 +++++++------------ 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01600_multiple_left_join_with_aliases.sql b/tests/queries/0_stateless/01600_multiple_left_join_with_aliases.sql index ae296e18560..2945622fbed 100644 --- a/tests/queries/0_stateless/01600_multiple_left_join_with_aliases.sql +++ b/tests/queries/0_stateless/01600_multiple_left_join_with_aliases.sql @@ -1,9 +1,4 @@ --- Tags: no-parallel - -drop database if exists test_01600; -create database test_01600; - -CREATE TABLE test_01600.base +CREATE TABLE base ( `id` UInt64, `id2` UInt64, @@ -14,7 +9,7 @@ ENGINE=MergeTree() PARTITION BY d ORDER BY (id,id2,d); -CREATE TABLE test_01600.derived1 +CREATE TABLE derived1 ( `id1` UInt64, `d1` UInt64, @@ -25,7 +20,7 @@ PARTITION BY d1 ORDER BY (id1, d1) ; -CREATE TABLE test_01600.derived2 +CREATE TABLE derived2 ( `id2` UInt64, `d2` UInt64, @@ -36,19 +31,17 @@ PARTITION BY d2 ORDER BY (id2, d2) ; -select +select base.id as `base.id`, derived2.value2 as `derived2.value2`, derived1.value1 as `derived1.value1` -from test_01600.base as base -left join test_01600.derived2 as derived2 on base.id2 = derived2.id2 -left join test_01600.derived1 as derived1 on base.id = derived1.id1; +from base as base +left join derived2 as derived2 on base.id2 = derived2.id2 +left join derived1 as derived1 on base.id = derived1.id1; SELECT base.id AS `base.id`, derived1.value1 AS `derived1.value1` -FROM test_01600.base AS base -LEFT JOIN test_01600.derived1 AS derived1 ON base.id = derived1.id1; - -drop database test_01600; +FROM base AS base +LEFT JOIN derived1 AS derived1 ON base.id = derived1.id1; From 078e4ca36db15a4f3bab27141d24fe00f89a5a1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:21:58 +0200 Subject: [PATCH 0974/1170] Make 02950_dictionary_ssd_cache_short_circuit parallelizable --- ...2950_dictionary_ssd_cache_short_circuit.sh | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh index 3d2fe5d664d..daa9c571a5d 100755 --- a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh +++ b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh @@ -1,15 +1,11 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n --query=" - DROP DATABASE IF EXISTS 02950_database_for_ssd_cache_dictionary; - CREATE DATABASE 02950_database_for_ssd_cache_dictionary; - - CREATE TABLE 02950_database_for_ssd_cache_dictionary.source_table + CREATE TABLE source_table ( id UInt64, v1 String, @@ -18,9 +14,9 @@ $CLICKHOUSE_CLIENT -n --query=" ) ENGINE = TinyLog; - INSERT INTO 02950_database_for_ssd_cache_dictionary.source_table VALUES (0, 'zero', 'zero', 0), (1, 'one', NULL, 1); + INSERT INTO source_table VALUES (0, 'zero', 'zero', 0), (1, 'one', NULL, 1); - CREATE DICTIONARY 02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary + CREATE DICTIONARY ssd_cache_dictionary ( id UInt64, v1 String, @@ -30,12 +26,11 @@ $CLICKHOUSE_CLIENT -n --query=" PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'source_table')) LIFETIME(MIN 1 MAX 1000) - LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$CLICKHOUSE_USER_FILES/0d')); + LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$CLICKHOUSE_USER_FILES/${CLICKHOUSE_DATABASE}_ssd_dic')); - SELECT dictGetOrDefault('02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary', ('v1', 'v2'), 0, (intDiv(1, id), intDiv(1, id))) FROM 02950_database_for_ssd_cache_dictionary.source_table; - SELECT dictGetOrDefault('02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary', 'v2', id+1, intDiv(NULL, id)) FROM 02950_database_for_ssd_cache_dictionary.source_table; - SELECT dictGetOrDefault('02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary', 'v3', id+1, intDiv(NULL, id)) FROM 02950_database_for_ssd_cache_dictionary.source_table; + SELECT dictGetOrDefault('ssd_cache_dictionary', ('v1', 'v2'), 0, (intDiv(1, id), intDiv(1, id))) FROM source_table; + SELECT dictGetOrDefault('ssd_cache_dictionary', 'v2', id+1, intDiv(NULL, id)) FROM source_table; + SELECT dictGetOrDefault('ssd_cache_dictionary', 'v3', id+1, intDiv(NULL, id)) FROM source_table; - DROP DICTIONARY 02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary; - DROP TABLE 02950_database_for_ssd_cache_dictionary.source_table; - DROP DATABASE 02950_database_for_ssd_cache_dictionary;" + DROP DICTIONARY ssd_cache_dictionary; + DROP TABLE source_table;" From c58643897a95d1e58bf25d65dfaad52aa0eddefc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:26:31 +0200 Subject: [PATCH 0975/1170] Make 02002_row_level_filter_bug parallelizable --- .../0_stateless/02002_row_level_filter_bug.sh | 31 +++++++++++-------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/02002_row_level_filter_bug.sh b/tests/queries/0_stateless/02002_row_level_filter_bug.sh index d15a26f48f5..557aa738217 100755 --- a/tests/queries/0_stateless/02002_row_level_filter_bug.sh +++ b/tests/queries/0_stateless/02002_row_level_filter_bug.sh @@ -1,11 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel -# Tag no-parallel: create user CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +TEST_ROLE="${CLICKHOUSE_DATABASE}_role" +TEST_USER="${CLICKHOUSE_DATABASE}_user" +TEST_POLICY="${CLICKHOUSE_DATABASE}_policy" $CLICKHOUSE_CLIENT --query "drop table if exists test_table" $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table @@ -37,26 +38,26 @@ arrayJoin(['AWD','ZZZ']) as team, arrayJoin([3183,3106,0,3130,3108,3126,3109,3107,3182,3180,3129,3128,3125,3266]) as a FROM numbers(600);" -$CLICKHOUSE_CLIENT --query "DROP ROLE IF exists AWD;" -$CLICKHOUSE_CLIENT --query "create role AWD;" -$CLICKHOUSE_CLIENT --query "REVOKE ALL ON *.* FROM AWD;" +$CLICKHOUSE_CLIENT --query "DROP ROLE IF EXISTS ${TEST_ROLE};" +$CLICKHOUSE_CLIENT --query "create role ${TEST_ROLE};" +$CLICKHOUSE_CLIENT --query "REVOKE ALL ON *.* FROM ${TEST_ROLE};" -$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS AWD_user;" -$CLICKHOUSE_CLIENT --query "CREATE USER AWD_user IDENTIFIED WITH plaintext_password BY 'AWD_pwd' DEFAULT ROLE AWD;" +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS ${TEST_USER};" +$CLICKHOUSE_CLIENT --query "CREATE USER ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'AWD_pwd' DEFAULT ROLE ${TEST_ROLE};" -$CLICKHOUSE_CLIENT --query "GRANT SELECT ON test_table TO AWD;" +$CLICKHOUSE_CLIENT --query "GRANT SELECT ON test_table TO ${TEST_ROLE};" -$CLICKHOUSE_CLIENT --query "DROP ROW POLICY IF EXISTS ttt_bu_test_table_AWD ON test_table;" -$CLICKHOUSE_CLIENT --query "CREATE ROW POLICY ttt_bu_test_table_AWD ON test_table FOR SELECT USING team = 'AWD' TO AWD;" +$CLICKHOUSE_CLIENT --query "DROP ROW POLICY IF EXISTS ${TEST_POLICY} ON test_table;" +$CLICKHOUSE_CLIENT --query "CREATE ROW POLICY ${TEST_POLICY} ON test_table FOR SELECT USING team = 'AWD' TO ${TEST_ROLE};" -$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +$CLICKHOUSE_CLIENT --user=${TEST_USER} --password=AWD_pwd --query " SELECT count() AS count FROM test_table WHERE t_date = '2021-07-15' AND c = 'aur' AND a=3130; " -$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +$CLICKHOUSE_CLIENT --user=${TEST_USER} --password=AWD_pwd --query " SELECT team, a, @@ -70,8 +71,12 @@ GROUP BY t_date; " -$CLICKHOUSE_CLIENT --user=AWD_user --password=AWD_pwd --query " +$CLICKHOUSE_CLIENT --user=${TEST_USER} --password=AWD_pwd --query " SELECT count() AS count FROM test_table WHERE (t_date = '2021-07-15') AND (c = 'aur') AND (a = 313) " + +$CLICKHOUSE_CLIENT --query "DROP ROLE IF EXISTS ${TEST_ROLE};" +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS ${TEST_USER};" +$CLICKHOUSE_CLIENT --query "DROP ROW POLICY IF EXISTS ${TEST_POLICY} ON test_table;" From 57b6d461418d4ad8e2d5cb579d5671abc077117e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Aug 2024 18:28:01 +0200 Subject: [PATCH 0976/1170] 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 5c4f4f85036838a7f145139f36949592720a1289 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 6 Aug 2024 18:52:29 +0200 Subject: [PATCH 0977/1170] 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 f0fdba3bbeb8687e38f89d8379e70cfd39db1252 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 19:02:49 +0200 Subject: [PATCH 0978/1170] CI: Strict job timeout 1.5h for tests, 2h for builds --- tests/ci/ci_config.py | 1 - tests/ci/ci_definitions.py | 6 ++---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7a19eb6f827..f578cd8b559 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -510,7 +510,6 @@ class CI: JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], run_by_label=Tags.libFuzzer, - timeout=10800, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.STYLE_CHECKER, ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 48847b0d7a6..69e7ed259d5 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -331,7 +331,7 @@ class JobConfig: # will be triggered for the job if omitted in CI workflow yml run_command: str = "" # job timeout, seconds - timeout: Optional[int] = None + timeout: Optional[int] = 5400 # sets number of batches for a multi-batch job num_batches: int = 1 # label that enables job in CI, if set digest isn't used @@ -420,7 +420,6 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", @@ -531,7 +530,6 @@ class CommonJobConfigs: docker=["clickhouse/sqllogic-test"], ), run_command="sqllogic_test.py", - timeout=10800, release_only=True, runner_type=Runners.FUNC_TESTER, ) @@ -543,7 +541,6 @@ class CommonJobConfigs: docker=["clickhouse/sqltest"], ), run_command="sqltest.py", - timeout=10800, release_only=True, runner_type=Runners.FUZZER_UNIT_TESTER, ) @@ -613,6 +610,7 @@ class CommonJobConfigs: docker=["clickhouse/binary-builder"], git_submodules=True, ), + timeout=7200, run_command="build_check.py $BUILD_NAME", runner_type=Runners.BUILDER, ) From 9c92c26edcb8ac83191f8a814196aa05a5600730 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Aug 2024 17:51:26 +0000 Subject: [PATCH 0979/1170] Simplify code. --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++--- .../MergeTree/registerStorageMergeTree.cpp | 13 +++++++++ src/Storages/StorageFactory.cpp | 28 ------------------- 3 files changed, 17 insertions(+), 33 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 98e1cddcf4b..dbb3e39c12e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3216,13 +3216,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Block the case of alter table add projection for special merge trees. if (std::any_of(commands.begin(), commands.end(), [](const AlterCommand & c) { return c.type == AlterCommand::ADD_PROJECTION; })) { - const std::unordered_set allowed_storages{"MergeTree", "ReplicatedMergeTree", "SharedMergeTree"}; - if (auto storage_name = getName(); !allowed_storages.contains(storage_name) + if (merging_params.mode != MergingParams::Mode::Ordinary && settings_from_storage->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Projection is fully supported in (Replictaed, Shared)MergeTree, but also allowed in non-throw mode with other" - " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." - " Current storage name is {}.", storage_name); + "Projection is fully supported in {} with deduplicate_merge_projection_mode = throw. " + "Use 'drop' or 'rebuild' option of deduplicate_merge_projection_mode.", + getName()); } commands.apply(new_metadata, local_context); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 3f0603f6900..b7887c35590 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -34,6 +34,7 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int SUPPORT_IS_DISABLED; } @@ -829,6 +830,18 @@ static StoragePtr create(const StorageFactory::Arguments & args) "Floating point partition key is not supported: {}", metadata.partition_key.column_names[i]); } + if (metadata.hasProjections() && args.mode == LoadingStrictnessLevel::CREATE) + { + /// Now let's handle the merge tree family. Note we only handle in the mode of CREATE due to backward compatibility. + /// Otherwise, it would fail to start in the case of existing projections with special mergetree. + if (merging_params.mode != MergeTreeData::MergingParams::Mode::Ordinary + && storage_settings->deduplicate_merge_projection_mode == DeduplicateMergeProjectionMode::THROW) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Projection is fully supported in {}MergeTree with deduplicate_merge_projection_mode = throw. " + "Use 'drop' or 'rebuild' option of deduplicate_merge_projection_mode.", + merging_params.getModeName()); + } + if (arg_num != arg_cnt) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong number of engine arguments."); diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 557f53a9ada..25cb81fa5fa 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -20,7 +20,6 @@ namespace ErrorCodes extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; extern const int BAD_ARGUMENTS; extern const int DATA_TYPE_CANNOT_BE_USED_IN_TABLES; - extern const int SUPPORT_IS_DISABLED; } @@ -201,33 +200,6 @@ StoragePtr StorageFactory::get( check_feature( "projections", [](StorageFeatures features) { return features.supports_projections; }); - - /// Now let's handle the merge tree family. Note we only handle in the mode of CREATE due to backward compatibility. - /// Otherwise, it would fail to start in the case of existing projections with special mergetree. - chassert(query.storage->engine); - const std::unordered_set allowed_engines{"MergeTree", "ReplicatedMergeTree", "SharedMergeTree"}; - if (auto engine_name(query.storage->engine->name); mode == LoadingStrictnessLevel::CREATE - && !allowed_engines.contains(engine_name)) - { - /// default throw mode in deduplicate_merge_projection_mode - bool projection_allowed = false; - if (auto * setting = query.storage->settings; setting != nullptr) - { - for (const auto & change : setting->changes) - { - if (change.name == "deduplicate_merge_projection_mode" && change.value != Field("throw")) - { - projection_allowed = true; - break; - } - } - } - if (!projection_allowed) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Projection is fully supported in (Replictaed, Shared)MergeTree, but also allowed in non-throw mode with other" - " mergetree family members. Consider drop or rebuild option of deduplicate_merge_projection_mode." - " Current storage name is {}.", engine_name); - } } } } From c74460b47e23ced4562d6f5123d61042c42d2896 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Aug 2024 17:53:23 +0000 Subject: [PATCH 0980/1170] Cleanup. --- src/Storages/StorageFactory.cpp | 2 -- .../0_stateless/03206_projection_merge_special_mergetree.sql | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 25cb81fa5fa..060b271d8f4 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -196,11 +196,9 @@ StoragePtr StorageFactory::get( [](StorageFeatures features) { return features.supports_skipping_indices; }); if (query.columns_list && query.columns_list->projections && !query.columns_list->projections->children.empty()) - { check_feature( "projections", [](StorageFeatures features) { return features.supports_projections; }); - } } } diff --git a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql index d3448138396..82684f754b6 100644 --- a/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql +++ b/tests/queries/0_stateless/03206_projection_merge_special_mergetree.sql @@ -103,4 +103,4 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'tp') AND (active = 1); -DROP TABLE tp; \ No newline at end of file +DROP TABLE tp; From bf33aabec412aa2729bfd58f3e717c5b8285acb8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jun 2024 10:39:10 +0200 Subject: [PATCH 0981/1170] 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 0982/1170] 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 0983/1170] 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 0984/1170] 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 0985/1170] 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 0986/1170] 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 0987/1170] 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 0988/1170] 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 0989/1170] 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 0990/1170] 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 0991/1170] 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 302bd5fdc6f077acf05a17c5e700cb44b2e609ab Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 18:37:26 +0000 Subject: [PATCH 0992/1170] Remove usearch submodule --- .gitmodules | 3 --- contrib/usearch | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/usearch diff --git a/.gitmodules b/.gitmodules index 7e0b4df4ad1..c4c93822711 100644 --- a/.gitmodules +++ b/.gitmodules @@ -339,9 +339,6 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git -[submodule "contrib/usearch"] - path = contrib/usearch - url = https://github.com/unum-cloud/usearch.git [submodule "contrib/SimSIMD"] path = contrib/SimSIMD url = https://github.com/ashvardanian/SimSIMD.git diff --git a/contrib/usearch b/contrib/usearch deleted file mode 160000 index 955c6f9c11a..00000000000 --- a/contrib/usearch +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f From 7a5b30d955b85a164097b4c7dd5ce957a47059e5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 18:39:18 +0000 Subject: [PATCH 0993/1170] Re-add forked usearch repo --- .gitmodules | 3 +++ contrib/usearch | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/usearch diff --git a/.gitmodules b/.gitmodules index c4c93822711..7fdfb1103c5 100644 --- a/.gitmodules +++ b/.gitmodules @@ -339,6 +339,9 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git +[submodule "contrib/usearch"] + path = contrib/usearch + url = https://github.com/ClickHouse/usearch.git [submodule "contrib/SimSIMD"] path = contrib/SimSIMD url = https://github.com/ashvardanian/SimSIMD.git diff --git a/contrib/usearch b/contrib/usearch new file mode 160000 index 00000000000..955c6f9c11a --- /dev/null +++ b/contrib/usearch @@ -0,0 +1 @@ +Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f From 9594a9baffb125e21b28cd421b511d26febfc900 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 18:43:44 +0000 Subject: [PATCH 0994/1170] Fix memory corruption in usearch --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index 955c6f9c11a..30810452bec 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f +Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356 From 36b6adbb30efdc0a96a274bdcbaeec2d961a8bed Mon Sep 17 00:00:00 2001 From: "Zhukova, Maria" Date: Tue, 6 Aug 2024 12:15:23 -0700 Subject: [PATCH 0995/1170] qpl-cmake: Better wording on ISA-L copy and removed non-required linking --- contrib/qpl-cmake/CMakeLists.txt | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index b2f263252c2..e62612cff5a 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -24,7 +24,9 @@ message(STATUS "Intel QPL version: ${QPL_VERSION}") # which are then combined into static or shared qpl. # Output ch_contrib::qpl by linking with 8 library targets. -# Note, qpl submodule comes with its own version of isal that is not compatible with upstream isal (e.g., ch_contrib::isal). +# Note, QPL has integrated a customized version of ISA-L to meet specific needs. +# This version has been significantly modified and there are no plans to maintain compatibility with the upstream version +# or upgrade the current copy. ## cmake/CompileOptions.cmake and automatic wrappers generation @@ -733,10 +735,6 @@ target_compile_definitions(_qpl target_link_libraries(_qpl PRIVATE ch_contrib::accel-config) -# C++ filesystem library requires additional linking for older GNU/Clang -target_link_libraries(_qpl PRIVATE $<$,$,9.1>>:stdc++fs>) -target_link_libraries(_qpl PRIVATE $<$,$,9.0>>:c++fs>) - target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include" PUBLIC ${UUID_DIR}) 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 0996/1170] 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 0997/1170] 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 139f5e55d34092969e49e4fd191404cac0ddd05b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Aug 2024 20:31:41 +0000 Subject: [PATCH 0998/1170] Unflake 02099_tsv_raw_format.sh --- ...rence => 02099_tsv_raw_format_1.reference} | 2 -- ...aw_format.sh => 02099_tsv_raw_format_1.sh} | 13 ------------- .../02099_tsv_raw_format_2.reference | 2 ++ .../0_stateless/02099_tsv_raw_format_2.sh | 19 +++++++++++++++++++ 4 files changed, 21 insertions(+), 15 deletions(-) rename tests/queries/0_stateless/{02099_tsv_raw_format.reference => 02099_tsv_raw_format_1.reference} (94%) rename tests/queries/0_stateless/{02099_tsv_raw_format.sh => 02099_tsv_raw_format_1.sh} (69%) create mode 100644 tests/queries/0_stateless/02099_tsv_raw_format_2.reference create mode 100755 tests/queries/0_stateless/02099_tsv_raw_format_2.sh diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.reference b/tests/queries/0_stateless/02099_tsv_raw_format_1.reference similarity index 94% rename from tests/queries/0_stateless/02099_tsv_raw_format.reference rename to tests/queries/0_stateless/02099_tsv_raw_format_1.reference index de46cf8dff7..3ac175e51f6 100644 --- a/tests/queries/0_stateless/02099_tsv_raw_format.reference +++ b/tests/queries/0_stateless/02099_tsv_raw_format_1.reference @@ -109,5 +109,3 @@ UInt64 String Date 2 \N nSome text -b1cad4eb4be08a40387c9de70d02fcc2 - -b1cad4eb4be08a40387c9de70d02fcc2 - diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh similarity index 69% rename from tests/queries/0_stateless/02099_tsv_raw_format.sh rename to tests/queries/0_stateless/02099_tsv_raw_format_1.sh index a69c96ab613..a3468f46ca0 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh @@ -46,16 +46,3 @@ echo 'nSome text' | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_string_0209 $CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_string_02099" $CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_string_02099" - - -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()"; -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" -$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum - -$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099" - -$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" -$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum - -$CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099" diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_2.reference b/tests/queries/0_stateless/02099_tsv_raw_format_2.reference new file mode 100644 index 00000000000..4682749c21f --- /dev/null +++ b/tests/queries/0_stateless/02099_tsv_raw_format_2.reference @@ -0,0 +1,2 @@ +c8ff17885084035ea1aebd95fee2efb6 - +c8ff17885084035ea1aebd95fee2efb6 - diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_2.sh b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh new file mode 100755 index 00000000000..d6034a0616f --- /dev/null +++ b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash +# Tags: long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()"; +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(100000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum + +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099" + +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(100000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099" + 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 0999/1170] 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 1000/1170] 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 1001/1170] 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 1002/1170] 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 1003/1170] 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 1004/1170] 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 1005/1170] 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 1006/1170] 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 1007/1170] 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 1008/1170] 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 1009/1170] 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 1010/1170] 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 1011/1170] 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 1012/1170] 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 1013/1170] 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 1014/1170] 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 1015/1170] 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 1016/1170] 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 1017/1170] 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 1018/1170] 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 1019/1170] 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 1020/1170] 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 1021/1170] 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 1022/1170] 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 ead9dc42d8a75ac99a0e4f538764cba206ad59a7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 08:31:49 +0000 Subject: [PATCH 1023/1170] Make it worse --- ...02310_clickhouse_local_INSERT_progress_profile_events.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect index d5b2a278180..5c731ea6d89 100755 --- a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-debug, no-tsan, no-msan, no-asan, no-ubsan, no-s3-storage +# Tags: no-debug, no-tsan, no-msan, no-asan, no-ubsan, no-s3-storage, no-cpu-aarch64 # ^ it can be slower than 60 seconds # This is the regression for the concurrent access in ProgressIndication, From 5eb896b9f1976feaa423071919e65d22e09da4ea Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 10:43:41 +0200 Subject: [PATCH 1024/1170] 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 1025/1170] 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 1026/1170] 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 1027/1170] 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 1028/1170] 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 1029/1170] 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 de41ffa18f5c2d3533fab3222aa401c193a0baac Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 7 Aug 2024 12:06:14 +0200 Subject: [PATCH 1030/1170] fix docks for groupConcat function documentation had wrong usage of a function. Examples of usage - https://github.com/ClickHouse/ClickHouse/blob/763952bf36d0b55f1b33ff11c693267574aa9666/tests/queries/0_stateless/03156_group_concat.sql --- .../aggregate-functions/reference/groupconcat.md | 6 +++--- tests/instructions/easy_tasks_sorted_ru.md | 4 ---- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md index 072252de8c9..bfa6160bbcc 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -10,7 +10,7 @@ Calculates a concatenated string from a group of strings, optionally separated b **Syntax** ``` sql -groupConcat(expression [, delimiter] [, limit]); +groupConcat[(delimiter [, limit])](expression); ``` **Arguments** @@ -20,7 +20,7 @@ groupConcat(expression [, delimiter] [, limit]); - `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. :::note -If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. +If delimiter is specified without limit, it must be the first parameter. If both delimiter and limit are specified, delimiter must precede limit. ::: **Returned value** @@ -61,7 +61,7 @@ This concatenates all names into one continuous string without any separator. Query: ``` sql -SELECT groupConcat(Name, ', ', 2) FROM Employees; +SELECT groupConcat(', ')(Name) FROM Employees; ``` Result: diff --git a/tests/instructions/easy_tasks_sorted_ru.md b/tests/instructions/easy_tasks_sorted_ru.md index fbd86ebf08f..e0607126ecc 100644 --- a/tests/instructions/easy_tasks_sorted_ru.md +++ b/tests/instructions/easy_tasks_sorted_ru.md @@ -97,10 +97,6 @@ Upd: сделали по-другому: теперь всё безопасно. Возвращает инкрементальное число для повторно встречающихся значений key. -## Агрегатная функция groupConcat. - -`groupConcat(x, ',')` - собрать из переданных значений x строку, разделённую запятыми. - ## Функции DATE_ADD, DATE_SUB как синонимы для совместимости с SQL. https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-add From a9c284dd8efb439ff06cf0f95e2a9920a26fdf5d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 10:07:27 +0000 Subject: [PATCH 1031/1170] 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 a9735f470c195dadbfffe02545b0979bfa9bd778 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 7 Aug 2024 12:09:26 +0200 Subject: [PATCH 1032/1170] squash! fix docks for groupConcat function documentation had wrong usage of a function. Examples of usage - https://github.com/ClickHouse/ClickHouse/blob/763952bf36d0b55f1b33ff11c693267574aa9666/tests/queries/0_stateless/03156_group_concat.sql --- .../sql-reference/aggregate-functions/reference/groupconcat.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md index bfa6160bbcc..6a24aa244bf 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -78,7 +78,7 @@ This output shows the names separated by a comma followed by a space. Query: ``` sql -SELECT groupConcat(Name, ', ', 2) FROM Employees; +SELECT groupConcat(', ', 2)(Name) FROM Employees; ``` Result: 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 1033/1170] 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 1034/1170] 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 e8bf5129c03dd88712829e9e187145e248ba4f04 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 7 Aug 2024 12:44:31 +0200 Subject: [PATCH 1035/1170] fix docks for clickhouse-keeper-client starting 24.7 paths are not accepted as bare strings, only as string literals - https://github.com/ClickHouse/ClickHouse/pull/65494 --- .../utilities/clickhouse-keeper-client.md | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 6407c66783b..a66ecbc1372 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -28,39 +28,39 @@ A client application to interact with clickhouse-keeper by its native protocol. Connected to ZooKeeper at [::1]:9181 with session_id 137 / :) ls keeper foo bar -/ :) cd keeper +/ :) cd 'keeper' /keeper :) ls api_version -/keeper :) cd api_version +/keeper :) cd 'api_version' /keeper/api_version :) ls -/keeper/api_version :) cd xyz +/keeper/api_version :) cd 'xyz' Path /keeper/api_version/xyz does not exist /keeper/api_version :) cd ../../ / :) ls keeper foo bar -/ :) get keeper/api_version +/ :) get 'keeper/api_version' 2 ``` ## Commands {#clickhouse-keeper-client-commands} -- `ls [path]` -- Lists the nodes for the given path (default: cwd) -- `cd [path]` -- Changes the working path (default `.`) -- `exists ` -- Returns `1` if node exists, `0` otherwise -- `set [version]` -- Updates the node's value. Only updates if version matches (default: -1) -- `create [mode]` -- Creates new node with the set value -- `touch ` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists -- `get ` -- Returns the node's value -- `rm [version]` -- Removes the node only if version matches (default: -1) -- `rmr ` -- Recursively deletes path. Confirmation required +- `ls '[path]'` -- Lists the nodes for the given path (default: cwd) +- `cd '[path]'` -- Changes the working path (default `.`) +- `exists ''` -- Returns `1` if node exists, `0` otherwise +- `set '' [version]` -- Updates the node's value. Only updates if version matches (default: -1) +- `create '' [mode]` -- Creates new node with the set value +- `touch ''` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists +- `get ''` -- Returns the node's value +- `rm '' [version]` -- Removes the node only if version matches (default: -1) +- `rmr ''` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message -- `get_direct_children_number [path]` -- Get numbers of direct children nodes under a specific path -- `get_all_children_number [path]` -- Get all numbers of children nodes under a specific path -- `get_stat [path]` -- Returns the node's stat (default `.`) -- `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) +- `get_direct_children_number '[path]'` -- Get numbers of direct children nodes under a specific path +- `get_all_children_number '[path]'` -- Get all numbers of children nodes under a specific path +- `get_stat '[path]'` -- Returns the node's stat (default `.`) +- `find_super_nodes '[path]'` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive - `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) -- `sync ` -- Synchronizes node between processes and leader +- `sync ''` -- Synchronizes node between processes and leader - `reconfig "" [version]` -- Reconfigure Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration From 016d1fea6d72c25179633f72f8ca8338dc59338f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 7 Aug 2024 13:58:03 +0200 Subject: [PATCH 1036/1170] 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 1037/1170] 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 c39cdcffaff5917510b27f457e35a63dddeed0b5 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 7 Aug 2024 14:03:53 +0200 Subject: [PATCH 1038/1170] docs for lightweight_mutation_projection_mode option starting with 24.7 we can lightweight delete in tables with projections. Fixed docs stating it is not possible --- docs/en/sql-reference/statements/delete.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index a52b7204c30..8745a06c124 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -36,9 +36,10 @@ If you anticipate frequent deletes, consider using a [custom partitioning key](/ ## Limitations of lightweight `DELETE` -### Lightweight `DELETE`s do not work with projections +### Lightweight `DELETE`s with projections -Currently, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation and may require the projection to be rebuilt, negatively affecting `DELETE` performance. +By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation and may require the projection to be rebuilt, negatively affecting `DELETE` performance. +However, there is an option to change this behavior. By changing setting `lightweight_mutation_projection_mode = 'drop'` will work with projections. ## Performance considerations when using lightweight `DELETE` From 37a6bd25f6d6dd2bfe3913639a7650c313642fb5 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 7 Aug 2024 14:06:02 +0200 Subject: [PATCH 1039/1170] squash! docs for lightweight_mutation_projection_mode option starting with 24.7 we can lightweight delete in tables with projections. Fixed docs stating it is not possible --- docs/en/sql-reference/statements/delete.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 8745a06c124..88a9c933519 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -39,7 +39,7 @@ If you anticipate frequent deletes, consider using a [custom partitioning key](/ ### Lightweight `DELETE`s with projections By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation and may require the projection to be rebuilt, negatively affecting `DELETE` performance. -However, there is an option to change this behavior. By changing setting `lightweight_mutation_projection_mode = 'drop'` will work with projections. +However, there is an option to change this behavior. By changing setting `lightweight_mutation_projection_mode = 'drop'`, deletes will work with projections. ## Performance considerations when using lightweight `DELETE` From 4a537874cad725227c847850b2da42d0ed86ccaf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 14:35:05 +0200 Subject: [PATCH 1040/1170] 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 1041/1170] 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 1042/1170] 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 1043/1170] 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 1044/1170] 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 1045/1170] 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 1046/1170] 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 ecba21bfe78fabf76eb7207f40e3fe53ecb70eaf Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 7 Aug 2024 15:23:19 +0200 Subject: [PATCH 1047/1170] Fix documentation for memory overcommit --- docs/en/operations/settings/settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b11c836fc1..968481062e9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4629,8 +4629,8 @@ Default Value: 5. ## memory_overcommit_ratio_denominator {#memory_overcommit_ratio_denominator} -It represents soft memory limit in case when hard limit is reached on user level. -This value is used to compute overcommit ratio for the query. +It represents the soft memory limit when the hard limit is reached on the global level. +This value is used to compute the overcommit ratio for the query. Zero means skip the query. Read more about [memory overcommit](memory-overcommit.md). @@ -4646,8 +4646,8 @@ Default value: `5000000`. ## memory_overcommit_ratio_denominator_for_user {#memory_overcommit_ratio_denominator_for_user} -It represents soft memory limit in case when hard limit is reached on global level. -This value is used to compute overcommit ratio for the query. +It represents the soft memory limit when the hard limit is reached on the user level. +This value is used to compute the overcommit ratio for the query. Zero means skip the query. Read more about [memory overcommit](memory-overcommit.md). 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 1048/1170] 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 1049/1170] 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 9a812fc53502cbd3bb69964d8e8b1afb186e6b46 Mon Sep 17 00:00:00 2001 From: "Max K." Date: Wed, 7 Aug 2024 15:55:03 +0200 Subject: [PATCH 1050/1170] Revert "CI: Strict job timeout 1.5h for tests, 2h for builds" --- tests/ci/ci_config.py | 1 + tests/ci/ci_definitions.py | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index f578cd8b559..7a19eb6f827 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -510,6 +510,7 @@ class CI: JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], run_by_label=Tags.libFuzzer, + timeout=10800, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.STYLE_CHECKER, ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 69e7ed259d5..48847b0d7a6 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -331,7 +331,7 @@ class JobConfig: # will be triggered for the job if omitted in CI workflow yml run_command: str = "" # job timeout, seconds - timeout: Optional[int] = 5400 + timeout: Optional[int] = None # sets number of batches for a multi-batch job num_batches: int = 1 # label that enables job in CI, if set digest isn't used @@ -420,6 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, + timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", @@ -530,6 +531,7 @@ class CommonJobConfigs: docker=["clickhouse/sqllogic-test"], ), run_command="sqllogic_test.py", + timeout=10800, release_only=True, runner_type=Runners.FUNC_TESTER, ) @@ -541,6 +543,7 @@ class CommonJobConfigs: docker=["clickhouse/sqltest"], ), run_command="sqltest.py", + timeout=10800, release_only=True, runner_type=Runners.FUZZER_UNIT_TESTER, ) @@ -610,7 +613,6 @@ class CommonJobConfigs: docker=["clickhouse/binary-builder"], git_submodules=True, ), - timeout=7200, run_command="build_check.py $BUILD_NAME", runner_type=Runners.BUILDER, ) From 364e973ef7aaa561780c50cd795b9edbcae51a41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Aug 2024 16:23:47 +0200 Subject: [PATCH 1051/1170] 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 1052/1170] 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 1053/1170] 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 1054/1170] 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 1055/1170] 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 1056/1170] 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 1057/1170] 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 1058/1170] 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 1059/1170] 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 1060/1170] 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 1061/1170] 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 1062/1170] 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 1063/1170] 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 1064/1170] 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 1065/1170] 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 1066/1170] 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 1067/1170] 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 1068/1170] 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 1069/1170] 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 1070/1170] 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 1071/1170] 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 1072/1170] 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 1073/1170] 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 1074/1170] 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 1075/1170] 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 1076/1170] 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 1077/1170] 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 1078/1170] 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 1079/1170] 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 1080/1170] 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 1081/1170] 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 1082/1170] 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 1083/1170] 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 1084/1170] 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 1085/1170] 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 1086/1170] 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 1087/1170] 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 1088/1170] 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 1089/1170] 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 1090/1170] 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 1091/1170] 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 1092/1170] 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 1093/1170] 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 1094/1170] 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 1095/1170] 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 1096/1170] 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 1097/1170] 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 1098/1170] 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 1099/1170] 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 1100/1170] 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 1101/1170] 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 1102/1170] 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 1103/1170] 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 1104/1170] 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 1105/1170] 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 1106/1170] 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 1107/1170] 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 1108/1170] 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 1109/1170] 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 1110/1170] 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 1111/1170] 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 1112/1170] 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 1113/1170] 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 1114/1170] 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 1115/1170] 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 1116/1170] 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 1117/1170] 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 1118/1170] 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 1119/1170] 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 1120/1170] 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 1121/1170] 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 1122/1170] 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 1123/1170] 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 1124/1170] 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 1125/1170] 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 1126/1170] 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 1127/1170] 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 1128/1170] 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 1129/1170] 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 1130/1170] 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 1131/1170] 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 1132/1170] 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 1133/1170] 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 1134/1170] 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 1135/1170] [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 1136/1170] 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 1137/1170] 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 1138/1170] 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 1139/1170] 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 1140/1170] 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 1141/1170] 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 1142/1170] 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 1143/1170] 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 1144/1170] 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 1145/1170] 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 1146/1170] 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 1147/1170] 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 1148/1170] 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 1149/1170] 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 1150/1170] 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 1151/1170] 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 1152/1170] 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 1153/1170] 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 1154/1170] 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 1155/1170] 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 1156/1170] 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 1157/1170] 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 1158/1170] Push CI From e582118544f3c49c3c6600ac8fa252151714d25f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 10 Aug 2024 13:09:40 +0200 Subject: [PATCH 1159/1170] 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 1160/1170] 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 1161/1170] 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 1162/1170] 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 1163/1170] 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 1164/1170] 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 1165/1170] 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 1166/1170] 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 1167/1170] 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 1168/1170] 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 1169/1170] 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 1170/1170] 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 $?