diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 4e38f28d5f0..f8ecdf1aa21 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -104,12 +104,16 @@ EOL function stop() { + local pid + # Preserve the pid, since the server can hung after the PID will be deleted. + pid="$(cat /var/run/clickhouse-server/clickhouse-server.pid)" + clickhouse stop --do-not-kill && return # We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces. kill -TERM "$(pidof gdb)" ||: sleep 5 echo "thread apply all backtrace (on stop)" >> /test_output/gdb.log - gdb -batch -ex 'thread apply all backtrace' -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log + gdb -batch -ex 'thread apply all backtrace' -p "$pid" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log clickhouse stop --force } @@ -441,6 +445,13 @@ else [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt fi +dmesg -T > /test_output/dmesg.log + +# OOM in dmesg -- those are real +grep -q -F -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE' /test_output/dmesg.log \ + && echo -e 'OOM in dmesg\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'No OOM in dmesg\tOK' >> /test_output/test_results.tsv + tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: mv /var/log/clickhouse-server/stderr.log /test_output/ @@ -462,5 +473,3 @@ for core in core.*; do pigz $core mv $core.gz /test_output/ done - -dmesg -T > /test_output/dmesg.log diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 82617e9425d..cd82d404960 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -13,6 +13,8 @@ Syntax: `URL(URL [,Format] [,CompressionMethod])` - The `Format` must be one that ClickHouse can use in `SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see [Formats](../../../interfaces/formats.md#formats). + If this argument is not specified, ClickHouse detectes the format automatically from the suffix of the `URL` parameter. If the suffix of `URL` parameter does not match any supported formats, it fails to create table. For example, for engine expression `URL('http://localhost/test.json')`, `JSON` format is applied. + - `CompressionMethod` indicates that whether the HTTP body should be compressed. If the compression is enabled, the HTTP packets sent by the URL engine contain 'Content-Encoding' header to indicate which compression method is used. To enable compression, please first make sure the remote HTTP endpoint indicated by the `URL` parameter supports corresponding compression algorithm. @@ -27,6 +29,11 @@ The supported `CompressionMethod` should be one of following: - bz2 - snappy - none +- auto + +If `CompressionMethod` is not specified, it defaults to `auto`. This means ClickHouse detects compression method from the suffix of `URL` parameter automatically. If the suffix matches any of compression method listed above, corresponding compression is applied or there won't be any compression enabled. + +For example, for engine expression `URL('http://localhost/test.gzip')`, `gzip` compression method is applied, but for `URL('http://localhost/test.fr')`, no compression is enabled because the suffix `fr` does not match any compression methods above. ## Usage {#using-the-engine-in-the-clickhouse-server} diff --git a/docs/ru/about-us/_category_.yml b/docs/ru/about-us/_category_.yml new file mode 100644 index 00000000000..62abf8e9e43 --- /dev/null +++ b/docs/ru/about-us/_category_.yml @@ -0,0 +1,8 @@ +position: 80 +label: 'About Us' +collapsible: true +collapsed: true +link: + type: generated-index + title: About Us + slug: /ru/about-us/ diff --git a/docs/ru/about-us/adopters.mdx b/docs/ru/about-us/adopters.mdx new file mode 100644 index 00000000000..7c0b9e030b7 --- /dev/null +++ b/docs/ru/about-us/adopters.mdx @@ -0,0 +1,8 @@ +--- +sidebar_label: Adopters +title: ClickHouse Adopters +--- + +import Adopters from '@site/docs/en/about-us/adopters.md'; + + diff --git a/docs/ru/about-us/support.mdx b/docs/ru/about-us/support.mdx new file mode 100644 index 00000000000..85e1a5675dd --- /dev/null +++ b/docs/ru/about-us/support.mdx @@ -0,0 +1,8 @@ +--- +sidebar_label: Commercial Support +title: ClickHouse Commercial Support Service +--- + +import Support from '@site/docs/en/about-us/support.md'; + + diff --git a/docs/ru/faq/operations/production.md b/docs/ru/faq/operations/production.md index 0dbe504b676..1984a464edb 100644 --- a/docs/ru/faq/operations/production.md +++ b/docs/ru/faq/operations/production.md @@ -67,4 +67,4 @@ sidebar_position: 10 Часто компании, которые изначально ориентировались на релизы `lts`, позднее переходят на `stable`, поскольку хотят быстрее получать доступ к новым возможностям. :::danger "Важно" - Мы всегда стремимся поддерживать совместимость релизов, но иногда это правило нарушается, и какие-то отдельные возможности в новых релизах становятся недоступны. Перед обновлением ClickHouse обязательно изучите [журнал изменений](../../whats-new/changelog/index.md), чтобы убедиться, что в нем нет объявлений о нарушении обратной совместимости. + Мы всегда стремимся поддерживать совместимость релизов, но иногда это правило нарушается, и какие-то отдельные возможности в новых релизах становятся недоступны. Перед обновлением ClickHouse обязательно изучите [журнал изменений](../../whats-new/changelog/index.mdx), чтобы убедиться, что в нем нет объявлений о нарушении обратной совместимости. diff --git a/docs/ru/whats-new/_category_.yml b/docs/ru/whats-new/_category_.yml new file mode 100644 index 00000000000..570ed7642a2 --- /dev/null +++ b/docs/ru/whats-new/_category_.yml @@ -0,0 +1,8 @@ +position: 70 +label: "What's New" +collapsible: true +collapsed: true +link: + type: generated-index + title: "What's New" + slug: /ru/whats-new/ diff --git a/docs/ru/whats-new/changelog/2017.mdx b/docs/ru/whats-new/changelog/2017.mdx new file mode 100644 index 00000000000..e91bdf0e908 --- /dev/null +++ b/docs/ru/whats-new/changelog/2017.mdx @@ -0,0 +1,9 @@ +--- +sidebar_position: 6 +sidebar_label: 2017 +title: 2017 Changelog +--- + +import Changelog from '@site/docs/en/whats-new/changelog/2017.md'; + + diff --git a/docs/ru/whats-new/changelog/2018.mdx b/docs/ru/whats-new/changelog/2018.mdx new file mode 100644 index 00000000000..40270b0d314 --- /dev/null +++ b/docs/ru/whats-new/changelog/2018.mdx @@ -0,0 +1,9 @@ +--- +sidebar_position: 5 +sidebar_label: 2018 +title: 2018 Changelog +--- + +import Changelog from '@site/docs/en/whats-new/changelog/2018.md'; + + diff --git a/docs/ru/whats-new/changelog/2019.mdx b/docs/ru/whats-new/changelog/2019.mdx new file mode 100644 index 00000000000..bad312817f4 --- /dev/null +++ b/docs/ru/whats-new/changelog/2019.mdx @@ -0,0 +1,9 @@ +--- +sidebar_position: 4 +sidebar_label: 2019 +title: 2019 Changelog +--- + +import Changelog from '@site/docs/en/whats-new/changelog/2019.md'; + + diff --git a/docs/ru/whats-new/changelog/2020.mdx b/docs/ru/whats-new/changelog/2020.mdx new file mode 100644 index 00000000000..6488d1b6ce5 --- /dev/null +++ b/docs/ru/whats-new/changelog/2020.mdx @@ -0,0 +1,9 @@ +--- +sidebar_position: 3 +sidebar_label: 2020 +title: 2020 Changelog +--- + +import Changelog from '@site/docs/en/whats-new/changelog/2020.md'; + + diff --git a/docs/ru/whats-new/changelog/2021.mdx b/docs/ru/whats-new/changelog/2021.mdx new file mode 100644 index 00000000000..4cc1390a10e --- /dev/null +++ b/docs/ru/whats-new/changelog/2021.mdx @@ -0,0 +1,9 @@ +--- +sidebar_position: 2 +sidebar_label: 2021 +title: 2021 Changelog +--- + +import Changelog from '@site/docs/en/whats-new/changelog/2021.md'; + + diff --git a/docs/ru/whats-new/changelog/_category_.yml b/docs/ru/whats-new/changelog/_category_.yml new file mode 100644 index 00000000000..7a496eace66 --- /dev/null +++ b/docs/ru/whats-new/changelog/_category_.yml @@ -0,0 +1,6 @@ +label: 'Changelog' +collapsible: true +collapsed: true +link: + type: doc + id: ru/whats-new/changelog/index diff --git a/docs/ru/whats-new/changelog/index.mdx b/docs/ru/whats-new/changelog/index.mdx new file mode 100644 index 00000000000..311be3be313 --- /dev/null +++ b/docs/ru/whats-new/changelog/index.mdx @@ -0,0 +1,10 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +title: 2022 Changelog +slug: /ru/whats-new/changelog/index +--- + +import Changelog from '@site/docs/en/whats-new/changelog/index.md'; + + diff --git a/docs/ru/whats-new/index.md b/docs/ru/whats-new/index.md deleted file mode 100644 index 0d87743d4f0..00000000000 --- a/docs/ru/whats-new/index.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -slug: /ru/whats-new/ -sidebar_label: "Что нового?" -sidebar_position: 82 ---- - -# Что нового в ClickHouse? - -Планы развития вкратце изложены [здесь](https://github.com/ClickHouse/ClickHouse/issues/32513), а новости по предыдущим релизам подробно описаны в [журнале изменений](./changelog/index.md). diff --git a/docs/ru/whats-new/roadmap.mdx b/docs/ru/whats-new/roadmap.mdx new file mode 100644 index 00000000000..6b7565cd28c --- /dev/null +++ b/docs/ru/whats-new/roadmap.mdx @@ -0,0 +1,8 @@ +--- +sidebar_label: Roadmap +title: Roadmap +--- + +import Roadmap from '@site/docs/en/whats-new/roadmap.md'; + + diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fe446796c16..89e4f871265 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -122,18 +122,18 @@ M(ExternalProcessingCompressedBytesTotal, "Number of compressed bytes written by external processing (sorting/aggragating/joining)") \ M(ExternalProcessingUncompressedBytesTotal, "Number of uncompressed bytes written by external processing (sorting/aggragating/joining)") \ M(ExternalProcessingFilesTotal, "Number of files used by external processing (sorting/aggragating/joining)") \ - M(ExternalSortWritePart, "") \ - M(ExternalSortMerge, "") \ + M(ExternalSortWritePart, "Number of temporary files for external sorting") \ + M(ExternalSortMerge, "Number of merging streams for temporary on-disk data written by external sorting") \ M(ExternalSortCompressedBytes, "Number of compressed bytes written by external sort") \ M(ExternalSortUncompressedBytes, "Number of uncompressed bytes written by external sort") \ - M(ExternalAggregationWritePart, "") \ - M(ExternalAggregationMerge, "") \ + M(ExternalAggregationWritePart, "Number of temporary files for external aggregation") \ + M(ExternalAggregationMerge, "Number of merging streams for temporary on-disk data written by external aggregation") \ M(ExternalAggregationCompressedBytes, "Number of compressed bytes written by external aggregation") \ M(ExternalAggregationUncompressedBytes, "Number of uncompressed bytes written by external aggregation") \ - M(ExternalJoinWritePart, "") \ - M(ExternalJoinMerge, "") \ - M(ExternalJoinCompressedBytes, "Number of compressed bytes written by external join") \ - M(ExternalJoinUncompressedBytes, "Number of uncompressed bytes written by external join") \ + M(ExternalJoinWritePart, "Number of temporary files for external JOIN") \ + M(ExternalJoinMerge, "Number of merging streams for temporary on-disk data written by external JOIN") \ + M(ExternalJoinCompressedBytes, "Number of compressed bytes written by external JOIN") \ + M(ExternalJoinUncompressedBytes, "Number of uncompressed bytes written by external JOIN") \ \ M(SlowRead, "Number of reads from a file that were slow. This indicate system overload. Thresholds are controlled by read_backoff_* settings.") \ M(ReadBackoff, "Number of times the number of query processing threads was lowered due to slow reads.") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 96769f4e311..2a4e5a84ab0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -64,8 +64,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 50, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 100, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ - M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also send on the corresponding connection end on the server.", 0) \ - M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the 'receive_timeout' for the socket will be also send on the corresponding connection end on the server.", 0) \ + M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ + M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the 'receive_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ M(Seconds, drain_timeout, 3, "Timeout for draining remote connections, -1 means synchronous drain without ignoring errors", 0) \ M(Seconds, tcp_keep_alive_timeout, 290 /* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ M(Milliseconds, hedged_connection_timeout_ms, 100, "Connection timeout for establishing connection with replica for Hedged requests", 0) \ @@ -506,7 +506,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ - M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \ + M(Bool, system_events_show_zero_values, false, "When querying system.events or system.metrics tables, include all metrics, even with zero values.", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal', 'datetime64', 'date2Date32' or 'date2String'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \ M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp index 761e49fe7bd..6f8b5eb9de2 100644 --- a/src/Functions/CastOverloadResolver.cpp +++ b/src/Functions/CastOverloadResolver.cpp @@ -8,8 +8,7 @@ namespace DB REGISTER_FUNCTION(CastOverloadResolvers) { factory.registerFunction>(FunctionFactory::CaseInsensitive); - factory.registerFunction>(); - factory.registerFunction>(); + /// Note: "internal" (not affected by null preserving setting) versions of accurate cast functions are unneeded. factory.registerFunction>(FunctionFactory::CaseInsensitive); factory.registerFunction>(); diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index e62020e10fe..6aa3d97ff0a 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -9,14 +9,13 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -/* - * CastInternal does not preserve nullability of the data type, - * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). - * - * Cast preserves nullability according to setting `cast_keep_nullable`, - * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. -**/ -template +/** CastInternal does not preserve nullability of the data type, + * i.e. CastInternal(toNullable(toInt8(1)) as Int32) will be Int32(1). + * + * Cast preserves nullability according to setting `cast_keep_nullable`, + * i.e. Cast(toNullable(toInt8(1)) as Int32) will be Nullable(Int32(1)) if `cast_keep_nullable` == 1. + */ +template class CastOverloadResolverImpl : public IFunctionOverloadResolver { public: diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 449335407aa..83db5eeaeed 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -94,7 +94,7 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::None; throw Exception( - "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'lz4', 'bz2', 'snappy' are supported as compression methods", + "Unknown compression method '" + hint + "'. Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'lz4', 'bz2', 'snappy' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 1ad3d0057cd..67fb256b1c9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include "config_core.h" @@ -32,6 +33,7 @@ # include #endif + namespace CurrentMetrics { extern const Metric TablesToDropQueueSize; @@ -255,6 +257,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( ContextPtr context_, std::optional * exception) const { + checkStackSize(); + if (!table_id) { if (exception) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c14584a382b..3fc00a79bbe 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -102,6 +102,19 @@ std::unique_lock StorageBuffer::Buffer::lockImpl(bool read) const } +StoragePtr StorageBuffer::getDestinationTable() const +{ + if (!destination_id) + return {}; + + auto destination = DatabaseCatalog::instance().tryGetTable(destination_id, getContext()); + if (destination.get() == this) + throw Exception("Destination table is myself. Will lead to infinite loop.", ErrorCodes::INFINITE_LOOP); + + return destination; +} + + StorageBuffer::StorageBuffer( const StorageID & table_id_, const ColumnsDescription & columns_, @@ -134,6 +147,7 @@ StorageBuffer::StorageBuffer( } else storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); @@ -193,13 +207,8 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage( const StorageSnapshotPtr &, SelectQueryInfo & query_info) const { - if (destination_id) + if (auto destination = getDestinationTable()) { - auto destination = DatabaseCatalog::instance().getTable(destination_id, local_context); - - if (destination.get() == this) - throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - /// TODO: Find a way to support projections for StorageBuffer query_info.ignore_projections = true; const auto & destination_metadata = destination->getInMemoryMetadataPtr(); @@ -221,13 +230,8 @@ void StorageBuffer::read( { const auto & metadata_snapshot = storage_snapshot->metadata; - if (destination_id) + if (auto destination = getDestinationTable()) { - auto destination = DatabaseCatalog::instance().getTable(destination_id, local_context); - - if (destination.get() == this) - throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - auto destination_lock = destination->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); @@ -521,8 +525,8 @@ public: auto block = getHeader().cloneWithColumns(chunk.getColumns()); - StoragePtr destination; - if (storage.destination_id) + StoragePtr destination = storage.getDestinationTable(); + if (destination) { destination = DatabaseCatalog::instance().tryGetTable(storage.destination_id, storage.getContext()); if (destination.get() == &storage) @@ -537,7 +541,7 @@ public: /// If the block already exceeds the maximum limit, then we skip the buffer. if (rows > storage.max_thresholds.rows || bytes > storage.max_thresholds.bytes) { - if (storage.destination_id) + if (destination) { LOG_DEBUG(storage.log, "Writing block with {} rows, {} bytes directly.", rows, bytes); storage.writeBlockToDestination(block, destination); @@ -628,15 +632,9 @@ SinkToStoragePtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMet bool StorageBuffer::mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const { - if (!destination_id) - return false; - - auto destination = DatabaseCatalog::instance().getTable(destination_id, query_context); - - if (destination.get() == this) - throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - - return destination->mayBenefitFromIndexForIn(left_in_operand, query_context, destination->getInMemoryMetadataPtr()); + if (auto destination = getDestinationTable()) + return destination->mayBenefitFromIndexForIn(left_in_operand, query_context, destination->getInMemoryMetadataPtr()); + return false; } @@ -703,11 +701,8 @@ bool StorageBuffer::optimize( bool StorageBuffer::supportsPrewhere() const { - if (!destination_id) - return false; - auto dest = DatabaseCatalog::instance().tryGetTable(destination_id, getContext()); - if (dest && dest.get() != this) - return dest->supportsPrewhere(); + if (auto destination = getDestinationTable()) + return destination->supportsPrewhere(); return false; } @@ -834,7 +829,7 @@ bool StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc Stopwatch watch; try { - writeBlockToDestination(block_to_write, DatabaseCatalog::instance().tryGetTable(destination_id, getContext())); + writeBlockToDestination(block_to_write, getDestinationTable()); } catch (...) { @@ -1010,14 +1005,10 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context std::optional StorageBuffer::totalRows(const Settings & settings) const { std::optional underlying_rows; - auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id, getContext()); + if (auto destination = getDestinationTable()) + underlying_rows = destination->totalRows(settings); - if (underlying) - underlying_rows = underlying->totalRows(settings); - if (!underlying_rows) - return underlying_rows; - - return total_writes.rows + *underlying_rows; + return total_writes.rows + underlying_rows.value_or(0); } std::optional StorageBuffer::totalBytes(const Settings & /*settings*/) const diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 200b3fc1838..580742c0c84 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -169,6 +169,8 @@ private: void backgroundFlush(); void reschedule(); + StoragePtr getDestinationTable() const; + BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder flush_handle; }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 2d7eab3111b..c3da392b9cd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -71,7 +71,7 @@ IStorageURLBase::IStorageURLBase( ASTPtr partition_by_) : IStorage(table_id_) , uri(uri_) - , compression_method(compression_method_) + , compression_method(chooseCompressionMethod(Poco::URI(uri_).getPath(), compression_method_)) , format_name(format_name_) , format_settings(format_settings_) , headers(headers_) @@ -164,7 +164,7 @@ namespace const ColumnsDescription & columns, UInt64 max_block_size, const ConnectionTimeouts & timeouts, - const String & compression_method, + CompressionMethod compression_method, size_t download_threads, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}, @@ -245,7 +245,7 @@ namespace const String & http_method, std::function callback, const ConnectionTimeouts & timeouts, - const String & compression_method, + CompressionMethod compression_method, Poco::Net::HTTPBasicCredentials & credentials, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers, bool glob_url, @@ -354,7 +354,7 @@ namespace std::move(read_buffer_factory), threadPoolCallbackRunner(IOThreadPool::get()), download_threads), - chooseCompressionMethod(request_uri.getPath(), compression_method), + compression_method, settings.zstd_window_log_max); } } @@ -386,7 +386,7 @@ namespace delay_initialization, /* use_external_buffer */ false, /* skip_url_not_found_error */ skip_url_not_found_error), - chooseCompressionMethod(request_uri.getPath(), compression_method), + compression_method, settings.zstd_window_log_max); } catch (...) @@ -566,7 +566,7 @@ std::function IStorageURLBase::getReadPOSTDataCallback( ColumnsDescription IStorageURLBase::getTableStructureFromData( const String & format, const String & uri, - const String & compression_method, + CompressionMethod compression_method, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers, const std::optional & format_settings, ContextPtr context) @@ -791,7 +791,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad metadata_snapshot->getSampleBlock(), context, ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri, compression_method), + compression_method, http_method); } else @@ -803,7 +803,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad metadata_snapshot->getSampleBlock(), context, ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri, compression_method), + compression_method, http_method); } } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fb0d8e8fa43..63c803f2d26 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -44,7 +44,7 @@ public: static ColumnsDescription getTableStructureFromData( const String & format, const String & uri, - const String & compression_method, + CompressionMethod compression_method, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers, const std::optional & format_settings, ContextPtr context); @@ -67,7 +67,7 @@ protected: ASTPtr partition_by = nullptr); String uri; - String compression_method; + CompressionMethod compression_method; String format_name; // For URL engine, we use format settings from server context + `SETTINGS` // clause of the `CREATE` query. In this case, format_settings is set. diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index cc228df0053..aacbb5fa302 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -137,7 +137,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet metadata_snapshot->getSampleBlock(), local_context, ConnectionTimeouts::getHTTPTimeouts(local_context), - chooseCompressionMethod(uri, compression_method)); + compression_method); } bool StorageXDBC::supportsSubsetOfColumns() const diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 898a11ac0dc..bbae0990062 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -113,7 +113,12 @@ ReadWriteBufferFromHTTP::HTTPHeaderEntries TableFunctionURL::getHeaders() const ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context) const { if (structure == "auto") - return StorageURL::getTableStructureFromData(format, filename, compression_method, getHeaders(), std::nullopt, context); + return StorageURL::getTableStructureFromData(format, + filename, + chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method), + getHeaders(), + std::nullopt, + context); return parseColumnsListFromString(structure, context); } diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 40befc78de2..78cf9fad001 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -141,11 +141,19 @@ if __name__ == "__main__": if not os.path.exists(result_path): os.makedirs(result_path) - docker_env += ( - " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL" - " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER" - " -e CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD" - ) + database_url = get_parameter_from_ssm("clickhouse-test-stat-url") + database_username = get_parameter_from_ssm("clickhouse-test-stat-login") + database_password = get_parameter_from_ssm("clickhouse-test-stat-password") + + env_extra = { + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL": f"{database_url}:9440", + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER": database_username, + "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD": database_password, + "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME": check_name_with_group, + "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX": check_name_prefix, + } + + docker_env += "".join([f" -e {name}" for name in env_extra]) run_command = get_run_command( result_path, @@ -158,23 +166,10 @@ if __name__ == "__main__": ) logging.info("Going to run command %s", run_command) - popen_env = os.environ.copy() - - database_url = get_parameter_from_ssm("clickhouse-test-stat-url") - database_username = get_parameter_from_ssm("clickhouse-test-stat-login") - database_password = get_parameter_from_ssm("clickhouse-test-stat-password") - - popen_env.update( - { - "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL": f"{database_url}:9440", - "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER": database_username, - "CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD": database_password, - "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME": check_name_with_group, - "CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX": check_name_prefix, - } - ) - run_log_path = os.path.join(temp_path, "runlog.log") + + popen_env = os.environ.copy() + popen_env.update(env_extra) with TeePopen(run_command, run_log_path, env=popen_env) as process: retcode = process.wait() if retcode == 0: diff --git a/tests/queries/0_stateless/01030_storage_url_syntax.sql b/tests/queries/0_stateless/01030_storage_url_syntax.sql index e855383f0ac..9b31558eece 100644 --- a/tests/queries/0_stateless/01030_storage_url_syntax.sql +++ b/tests/queries/0_stateless/01030_storage_url_syntax.sql @@ -6,3 +6,61 @@ create table test_table_url_syntax (id UInt32) ENGINE = URL('','','','') ; -- { serverError 42 } drop table if exists test_table_url_syntax ; + +drop table if exists test_table_url +; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint') +; -- { serverError 36 } + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint.json'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'ErrorFormat') +; -- { serverError 73 } + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'gzip'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'gz'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'deflate'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'brotli'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'lzma'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'zstd'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'lz4'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'bz2'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'snappy'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'none'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'auto'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint.gz', 'JSONEachRow'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint.fr', 'JSONEachRow'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow'); +drop table test_table_url; + +create table test_table_url(id UInt32) ENGINE = URL('http://localhost/endpoint', 'JSONEachRow', 'zip') +; -- { serverError 48 } + diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 82ce4573e4d..a90b3eac6dc 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also send on the corresponding connection end on the server. \N \N 0 Seconds +send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds storage_policy default 0 Name of storage disk policy String 1 1 diff --git a/tests/queries/0_stateless/02391_recursive_buffer.reference b/tests/queries/0_stateless/02391_recursive_buffer.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02391_recursive_buffer.sql b/tests/queries/0_stateless/02391_recursive_buffer.sql new file mode 100644 index 00000000000..c0954ed834b --- /dev/null +++ b/tests/queries/0_stateless/02391_recursive_buffer.sql @@ -0,0 +1,18 @@ +-- Tags: no-parallel +-- because of system.tables poisoning + +DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32) Engine = Buffer(currentDatabase(), test, 16, 10, 100, 10000, 1000000, 10000000, 100000000); +SELECT * FROM test; -- { serverError 269 } +SELECT * FROM system.tables WHERE table = 'test' AND database = currentDatabase() FORMAT Null; -- { serverError 269 } +DROP TABLE test; + +DROP TABLE IF EXISTS test1; +DROP TABLE IF EXISTS test2; +CREATE TABLE test1 (key UInt32) Engine = Buffer(currentDatabase(), test2, 16, 10, 100, 10000, 1000000, 10000000, 100000000); +CREATE TABLE test2 (key UInt32) Engine = Buffer(currentDatabase(), test1, 16, 10, 100, 10000, 1000000, 10000000, 100000000); +SELECT * FROM test1; -- { serverError 306 } +SELECT * FROM test2; -- { serverError 306 } +SELECT * FROM system.tables WHERE table IN ('test1', 'test2') AND database = currentDatabase(); -- { serverError 306 } +DROP TABLE test1; +DROP TABLE test2; diff --git a/tests/queries/0_stateless/02393_every_metric_must_have_documentation.reference b/tests/queries/0_stateless/02393_every_metric_must_have_documentation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02393_every_metric_must_have_documentation.sql b/tests/queries/0_stateless/02393_every_metric_must_have_documentation.sql new file mode 100644 index 00000000000..cf98b6b2cb2 --- /dev/null +++ b/tests/queries/0_stateless/02393_every_metric_must_have_documentation.sql @@ -0,0 +1,2 @@ +SET system_events_show_zero_values = true; +SELECT metric FROM system.metrics WHERE length(description) < 10; diff --git a/tests/queries/0_stateless/02410_to_decimal_or_default.reference b/tests/queries/0_stateless/02410_to_decimal_or_default.reference new file mode 100644 index 00000000000..10ad131d596 --- /dev/null +++ b/tests/queries/0_stateless/02410_to_decimal_or_default.reference @@ -0,0 +1,8 @@ +111 Decimal(9, 3) +222 Decimal(18, 3) +333 Decimal(38, 3) +444 Decimal(76, 3) +123.456 Decimal(9, 3) +123.456 Decimal(18, 3) +123.456 Decimal(38, 3) +123.456 Decimal(76, 3) diff --git a/tests/queries/0_stateless/02410_to_decimal_or_default.sql b/tests/queries/0_stateless/02410_to_decimal_or_default.sql new file mode 100644 index 00000000000..8db464038b2 --- /dev/null +++ b/tests/queries/0_stateless/02410_to_decimal_or_default.sql @@ -0,0 +1,9 @@ +SELECT toDecimal32OrDefault(111, 3, 123.456::Decimal32(3)) AS x, toTypeName(x); +SELECT toDecimal64OrDefault(222, 3, 123.456::Decimal64(3)) AS x, toTypeName(x); +SELECT toDecimal128OrDefault(333, 3, 123.456::Decimal128(3)) AS x, toTypeName(x); +SELECT toDecimal256OrDefault(444, 3, 123.456::Decimal256(3)) AS x, toTypeName(x); + +SELECT toDecimal32OrDefault('Hello', 3, 123.456::Decimal32(3)) AS x, toTypeName(x); +SELECT toDecimal64OrDefault('Hello', 3, 123.456::Decimal64(3)) AS x, toTypeName(x); +SELECT toDecimal128OrDefault('Hello', 3, 123.456::Decimal128(3)) AS x, toTypeName(x); +SELECT toDecimal256OrDefault('Hello', 3, 123.456::Decimal256(3)) AS x, toTypeName(x); diff --git a/tests/queries/0_stateless/02413_model_evaluate_smoke.reference b/tests/queries/0_stateless/02413_model_evaluate_smoke.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02413_model_evaluate_smoke.sql b/tests/queries/0_stateless/02413_model_evaluate_smoke.sql new file mode 100644 index 00000000000..3b20067abfe --- /dev/null +++ b/tests/queries/0_stateless/02413_model_evaluate_smoke.sql @@ -0,0 +1,2 @@ +-- This model does not exist: +SELECT modelEvaluate('hello', 1, 2, 3); -- { serverError 36 }