From bda40b5a98974a9d78ca055855215b01bba546dc Mon Sep 17 00:00:00 2001 From: Filipp Bakanov Date: Thu, 14 Sep 2023 17:36:21 +0000 Subject: [PATCH 01/41] Fixed crash while using override + null field with MaterializedMySQL, #54647 --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 3578362b8dd..a0edbfd6ce2 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -577,8 +577,16 @@ static void writeFieldsToColumn( } null_map_column->insertValue(0); + } else { + // Column is not null but field is null. It's possible due to overrides + if (field.isNull()) + { + column_to.insertDefault(); + return false; + } } + return true; }; @@ -653,7 +661,7 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { - const String & data = value.get(); + const String & data = value.safeGet(); casted_string_column->insertData(data.data(), data.size()); } } From a46f2826ecd933dc5aa9f6243f24c1b35a5a04ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Dec 2023 06:49:36 +0300 Subject: [PATCH 02/41] Update MaterializedMySQLSyncThread.cpp --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a0edbfd6ce2..772ce09f0eb 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -577,7 +577,9 @@ static void writeFieldsToColumn( } null_map_column->insertValue(0); - } else { + } + else + { // Column is not null but field is null. It's possible due to overrides if (field.isNull()) { From e74ab64b0c1b6bfd3d3df4a6ce9c95eee7b6a716 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 23 Jun 2024 05:02:58 +0000 Subject: [PATCH 03/41] filter for columns in projection --- src/Storages/MergeTree/MergeTask.cpp | 7 +++++- .../03174_projection_deduplicate.reference | 1 + .../03174_projection_deduplicate.sql | 24 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03174_projection_deduplicate.reference create mode 100644 tests/queries/0_stateless/03174_projection_deduplicate.sql diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 56bd1181fef..9fe16ad8650 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -818,6 +818,11 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; projection_future_part->part_info = {"all", 0, 0, 0}; + Names deduplicate_by_columns; + for (const auto & column : global_ctx->deduplicate_by_columns) + if (projection.metadata->getColumns().has(column)) + deduplicate_by_columns.emplace_back(column); + MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; if (projection.type == ProjectionDescription::Type::Aggregate) @@ -832,7 +837,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->context, global_ctx->space_reservation, global_ctx->deduplicate, - global_ctx->deduplicate_by_columns, + deduplicate_by_columns, global_ctx->cleanup, projection_merging_params, global_ctx->need_prefix, diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference new file mode 100644 index 00000000000..d2ee9d25154 --- /dev/null +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -0,0 +1 @@ +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql new file mode 100644 index 00000000000..529c3260baa --- /dev/null +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -0,0 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/65548 +DROP TABLE IF EXISTS test_projection_deduplicate; + +CREATE TABLE test_projection_deduplicate +( + `id` Int32, + `string` String, + PROJECTION test_projection + ( + SELECT id + GROUP BY id + ) +) +ENGINE = MergeTree +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; + +SELECT * FROM test_projection_deduplicate; + +DROP TABLE test_projection_deduplicate; From bbafb7adb22d6edaf304ea68f6b50efa71744d84 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jun 2024 01:58:28 +0000 Subject: [PATCH 04/41] Fix 02834_apache_arrow_abort flakiness with MSAN --- tests/queries/0_stateless/02834_apache_arrow_abort.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02834_apache_arrow_abort.sql b/tests/queries/0_stateless/02834_apache_arrow_abort.sql index 47db46f1e43..bd29e95db9a 100644 --- a/tests/queries/0_stateless/02834_apache_arrow_abort.sql +++ b/tests/queries/0_stateless/02834_apache_arrow_abort.sql @@ -1,4 +1,4 @@ -- Tags: no-fasttest -- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library. -INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/hits.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/hits.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION } +INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION } From 9c25a1f69b0680136dccbc29eab82fb4f1d3b652 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jun 2024 02:12:08 +0000 Subject: [PATCH 05/41] Exclude some more HTTP codes in ReadWriteBufferFromHTTP::getFileInfo() --- src/IO/ReadWriteBufferFromHTTP.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 303ffb744b5..f17c19f3cc2 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -713,8 +713,16 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() /// fall back to slow whole-file reads when HEAD is actually supported; that sounds /// like a nightmare to debug.) if (e.getHTTPStatus() >= 400 && e.getHTTPStatus() <= 499 && - e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS) + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS && + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_REQUEST_TIMEOUT && + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_MISDIRECTED_REQUEST) + { + LOG_DEBUG(log, + "HEAD request to '{}'{} failed with HTTP status {}", + initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()), + e.getHTTPStatus()); return HTTPFileInfo{}; + } throw; } From 1402a00c8c741ceb2daa798e533ab95600c0e7a2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jun 2024 23:15:15 +0000 Subject: [PATCH 06/41] optimize deduplicate for tables with projections is not supported --- src/Storages/MergeTree/MergeTask.cpp | 7 +------ src/Storages/StorageMergeTree.cpp | 6 ++++++ src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++++ .../0_stateless/03174_projection_deduplicate.reference | 1 + tests/queries/0_stateless/03174_projection_deduplicate.sql | 2 +- 5 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9fe16ad8650..56bd1181fef 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -818,11 +818,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; projection_future_part->part_info = {"all", 0, 0, 0}; - Names deduplicate_by_columns; - for (const auto & column : global_ctx->deduplicate_by_columns) - if (projection.metadata->getColumns().has(column)) - deduplicate_by_columns.emplace_back(column); - MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; if (projection.type == ProjectionDescription::Type::Aggregate) @@ -837,7 +832,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->context, global_ctx->space_reservation, global_ctx->deduplicate, - deduplicate_by_columns, + global_ctx->deduplicate_by_columns, global_ctx->cleanup, projection_merging_params, global_ctx->need_prefix, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..0dd0f051244 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1549,6 +1549,12 @@ bool StorageMergeTree::optimize( { assertNotReadonly(); + if (deduplicate && getInMemoryMetadataPtr()->hasProjections()) + 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", + getStorageID().getTableName()); + if (deduplicate) { if (deduplicate_by_columns.empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..403313a7177 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5746,6 +5746,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()) + 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", + getStorageID().getTableName()); + if (cleanup) { if (!getSettings()->allow_experimental_replacing_merge_with_cleanup) diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference index d2ee9d25154..586b63a392e 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.reference +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -1 +1,2 @@ 1 one +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql index 529c3260baa..ed6da5d73fc 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; +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } SELECT * FROM test_projection_deduplicate; From 9341bbe19733fc2da455c6b0c51d0ee9bb13bf49 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 3 Jul 2024 16:36:08 +0800 Subject: [PATCH 07/41] ORC read use writer time zone --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/NativeORCBlockInputFormat.cpp | 5 +++++ .../03198_orc_read_time_zone.reference | 1 + .../0_stateless/03198_orc_read_time_zone.sh | 11 +++++++++++ .../data_orc/read_time_zone.snappy.orc | Bin 0 -> 427 bytes 7 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03198_orc_read_time_zone.reference create mode 100644 tests/queries/0_stateless/03198_orc_read_time_zone.sh create mode 100644 tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..47dde4b5d14 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1041,6 +1041,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(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/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 79c2e6b4890..6de106893f3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -243,6 +243,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.orc.output_row_index_stride = settings.output_format_orc_row_index_stride; format_settings.orc.use_fast_decoder = settings.input_format_orc_use_fast_decoder; format_settings.orc.filter_push_down = settings.input_format_orc_filter_push_down; + format_settings.orc.read_use_writer_time_zone = settings.input_format_orc_read_use_writer_time_zone; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8ac783a1d86..446dc17a187 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -403,6 +403,7 @@ struct FormatSettings bool use_fast_decoder = true; bool filter_push_down = true; UInt64 output_row_index_stride = 10'000; + bool read_use_writer_time_zone = false; } orc{}; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index dcd5a531b05..c10969b02b7 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -900,6 +900,11 @@ bool NativeORCBlockInputFormat::prepareStripeReader() orc::RowReaderOptions row_reader_options; row_reader_options.includeTypes(include_indices); + if (format_settings.orc.read_use_writer_time_zone) + { + String writer_time_zone = current_stripe_info->getWriterTimezone(); + row_reader_options.setTimezoneName(writer_time_zone); + } 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.reference b/tests/queries/0_stateless/03198_orc_read_time_zone.reference new file mode 100644 index 00000000000..809dba44400 --- /dev/null +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.reference @@ -0,0 +1 @@ +1 2024-06-30 20:00:00.000 diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh new file mode 100644 index 00000000000..fec4e2bcc72 --- /dev/null +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +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/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc b/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc new file mode 100644 index 0000000000000000000000000000000000000000..ab1b785dbbfc2381e23edb45244a18b07eeb370e GIT binary patch literal 427 zcmZ{g&q~8U5XNUW&AM4zjS+PT9%@tw3Tw28C8uf;s)8*Qq}MeKZLoi06AHZrPaYI} z4e{#9ckwNJ0N3^+qKDy|?;D2sG4*D334jqRM^OWIRlCeI0I>2DS%hK}bKHPal@}RN z`S)m*qH~jH4W4`x z`|-{O Date: Thu, 29 Feb 2024 13:00:38 +0100 Subject: [PATCH 08/41] Move function exceptionCodeToHTTPStatus() from HTTPHandler.cpp to a separate header in order to reuse it. --- src/Server/HTTP/exceptionCodeToHTTPStatus.cpp | 158 ++++++++++++++++++ src/Server/HTTP/exceptionCodeToHTTPStatus.h | 11 ++ src/Server/HTTPHandler.cpp | 139 +-------------- 3 files changed, 170 insertions(+), 138 deletions(-) create mode 100644 src/Server/HTTP/exceptionCodeToHTTPStatus.cpp create mode 100644 src/Server/HTTP/exceptionCodeToHTTPStatus.h diff --git a/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp b/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp new file mode 100644 index 00000000000..6de57217aac --- /dev/null +++ b/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp @@ -0,0 +1,158 @@ +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int CANNOT_PARSE_IPV4; + extern const int CANNOT_PARSE_IPV6; + extern const int CANNOT_PARSE_UUID; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_SCHEDULE_TASK; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_COMPILE_REGEXP; + extern const int DUPLICATE_COLUMN; + extern const int ILLEGAL_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int TOO_DEEP_AST; + extern const int TOO_BIG_AST; + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + + extern const int SYNTAX_ERROR; + + extern const int INCORRECT_DATA; + extern const int TYPE_MISMATCH; + + extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_SETTING; + extern const int UNKNOWN_DIRECTION_OF_SORTING; + extern const int UNKNOWN_AGGREGATE_FUNCTION; + extern const int UNKNOWN_FORMAT; + extern const int UNKNOWN_DATABASE_ENGINE; + extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int UNKNOWN_ROLE; + + extern const int QUERY_IS_TOO_LARGE; + + extern const int NOT_IMPLEMENTED; + extern const int SOCKET_TIMEOUT; + + extern const int UNKNOWN_USER; + extern const int WRONG_PASSWORD; + extern const int REQUIRED_PASSWORD; + extern const int AUTHENTICATION_FAILED; + extern const int SET_NON_GRANTED_ROLE; + + extern const int HTTP_LENGTH_REQUIRED; + + extern const int TIMEOUT_EXCEEDED; +} + + +Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) +{ + using namespace Poco::Net; + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + return HTTPResponse::HTTP_UNAUTHORIZED; + } + else if (exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::AUTHENTICATION_FAILED || + exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) + { + return HTTPResponse::HTTP_FORBIDDEN; + } + else if (exception_code == ErrorCodes::BAD_ARGUMENTS || + exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || + exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || + exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || + exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::CANNOT_PARSE_UUID || + exception_code == ErrorCodes::DUPLICATE_COLUMN || + exception_code == ErrorCodes::ILLEGAL_COLUMN || + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::THERE_IS_NO_COLUMN || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + exception_code == ErrorCodes::SYNTAX_ERROR || + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH || + exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) + { + return HTTPResponse::HTTP_BAD_REQUEST; + } + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || + exception_code == ErrorCodes::UNKNOWN_ROLE) + { + return HTTPResponse::HTTP_NOT_FOUND; + } + else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) + { + return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; + } + else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) + { + return HTTPResponse::HTTP_NOT_IMPLEMENTED; + } + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) + { + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + } + else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + { + return HTTPResponse::HTTP_LENGTH_REQUIRED; + } + else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED) + { + return HTTPResponse::HTTP_REQUEST_TIMEOUT; + } + else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK) + { + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + } + + return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; +} + +} diff --git a/src/Server/HTTP/exceptionCodeToHTTPStatus.h b/src/Server/HTTP/exceptionCodeToHTTPStatus.h new file mode 100644 index 00000000000..aadec7aac5a --- /dev/null +++ b/src/Server/HTTP/exceptionCodeToHTTPStatus.h @@ -0,0 +1,11 @@ +#pragma once +#include + + +namespace DB +{ + +/// Converts Exception code to HTTP status code. +Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index a00f6fb5412..6f51a269e0e 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -73,56 +74,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; - extern const int CANNOT_PARSE_IPV4; - extern const int CANNOT_PARSE_IPV6; - extern const int CANNOT_PARSE_UUID; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_SCHEDULE_TASK; - extern const int DUPLICATE_COLUMN; - extern const int ILLEGAL_COLUMN; - extern const int THERE_IS_NO_COLUMN; - extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int SYNTAX_ERROR; - - extern const int INCORRECT_DATA; - extern const int TYPE_MISMATCH; - - extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_FUNCTION; - extern const int UNKNOWN_IDENTIFIER; - extern const int UNKNOWN_TYPE; - extern const int UNKNOWN_STORAGE; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_SETTING; - extern const int UNKNOWN_DIRECTION_OF_SORTING; - extern const int UNKNOWN_AGGREGATE_FUNCTION; - extern const int UNKNOWN_FORMAT; - extern const int UNKNOWN_DATABASE_ENGINE; - extern const int UNKNOWN_TYPE_OF_QUERY; - extern const int UNKNOWN_ROLE; extern const int NO_ELEMENTS_IN_CONFIG; - extern const int QUERY_IS_TOO_LARGE; - - extern const int NOT_IMPLEMENTED; - extern const int SOCKET_TIMEOUT; - - extern const int UNKNOWN_USER; - extern const int WRONG_PASSWORD; extern const int REQUIRED_PASSWORD; extern const int AUTHENTICATION_FAILED; extern const int SET_NON_GRANTED_ROLE; @@ -130,8 +84,6 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; extern const int SUPPORT_IS_DISABLED; - - extern const int TIMEOUT_EXCEEDED; } namespace @@ -193,95 +145,6 @@ static String base64Encode(const String & decoded) return ostr.str(); } -static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) -{ - using namespace Poco::Net; - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - { - return HTTPResponse::HTTP_UNAUTHORIZED; - } - else if (exception_code == ErrorCodes::UNKNOWN_USER || - exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::AUTHENTICATION_FAILED || - exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) - { - return HTTPResponse::HTTP_FORBIDDEN; - } - else if (exception_code == ErrorCodes::BAD_ARGUMENTS || - exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || - exception_code == ErrorCodes::CANNOT_PARSE_TEXT || - exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || - exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_DATE || - exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || - exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || - exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || - exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || - exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || - exception_code == ErrorCodes::CANNOT_PARSE_UUID || - exception_code == ErrorCodes::DUPLICATE_COLUMN || - exception_code == ErrorCodes::ILLEGAL_COLUMN || - exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || - exception_code == ErrorCodes::THERE_IS_NO_COLUMN || - exception_code == ErrorCodes::TOO_DEEP_AST || - exception_code == ErrorCodes::TOO_BIG_AST || - exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || - exception_code == ErrorCodes::SYNTAX_ERROR || - exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH || - exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) - { - return HTTPResponse::HTTP_BAD_REQUEST; - } - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || - exception_code == ErrorCodes::UNKNOWN_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || - exception_code == ErrorCodes::UNKNOWN_TYPE || - exception_code == ErrorCodes::UNKNOWN_STORAGE || - exception_code == ErrorCodes::UNKNOWN_DATABASE || - exception_code == ErrorCodes::UNKNOWN_SETTING || - exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || - exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_FORMAT || - exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || - exception_code == ErrorCodes::UNKNOWN_ROLE) - { - return HTTPResponse::HTTP_NOT_FOUND; - } - else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) - { - return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; - } - else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) - { - return HTTPResponse::HTTP_NOT_IMPLEMENTED; - } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || - exception_code == ErrorCodes::CANNOT_OPEN_FILE) - { - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - } - else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - { - return HTTPResponse::HTTP_LENGTH_REQUIRED; - } - else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED) - { - return HTTPResponse::HTTP_REQUEST_TIMEOUT; - } - else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK) - { - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - } - - return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; -} - - static std::chrono::steady_clock::duration parseSessionTimeout( const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) From bf56ad69dde18f90b8d96e00d39e9576d4188c9e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 May 2024 20:58:28 +0200 Subject: [PATCH 09/41] Make Context::setCurrentRoles() check that new current roles are granted in order to make HTTPHandler's implementation easier. --- .../include/Poco/Net/NameValueCollection.h | 2 +- base/poco/Net/src/NameValueCollection.cpp | 4 +- .../Access/InterpreterSetRoleQuery.cpp | 23 +------ src/Interpreters/AsynchronousInsertQueue.cpp | 5 +- src/Interpreters/Context.cpp | 65 +++++++++++++++---- src/Interpreters/Context.h | 10 ++- src/Server/HTTPHandler.cpp | 19 +----- 7 files changed, 70 insertions(+), 58 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/NameValueCollection.h b/base/poco/Net/include/Poco/Net/NameValueCollection.h index be499838d0e..b66819840b1 100644 --- a/base/poco/Net/include/Poco/Net/NameValueCollection.h +++ b/base/poco/Net/include/Poco/Net/NameValueCollection.h @@ -79,7 +79,7 @@ namespace Net /// Returns the value of the first name-value pair with the given name. /// If no value with the given name has been found, the defaultValue is returned. - const std::vector> getAll(const std::string & name) const; + const std::vector getAll(const std::string & name) const; /// Returns all values of all name-value pairs with the given name. /// /// Returns an empty vector if there are no name-value pairs with the given name. diff --git a/base/poco/Net/src/NameValueCollection.cpp b/base/poco/Net/src/NameValueCollection.cpp index 783ed48cc30..0cb31924579 100644 --- a/base/poco/Net/src/NameValueCollection.cpp +++ b/base/poco/Net/src/NameValueCollection.cpp @@ -102,9 +102,9 @@ const std::string& NameValueCollection::get(const std::string& name, const std:: return defaultValue; } -const std::vector> NameValueCollection::getAll(const std::string& name) const +const std::vector NameValueCollection::getAll(const std::string& name) const { - std::vector> values; + std::vector values; for (ConstIterator it = _map.find(name); it != _map.end(); it++) if (it->first == name) values.push_back(it->second); diff --git a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 24467923542..99a7a73d46c 100644 --- a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -29,33 +29,12 @@ BlockIO InterpreterSetRoleQuery::execute() void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { - auto & access_control = getContext()->getAccessControl(); auto session_context = getContext()->getSessionContext(); - auto user = session_context->getUser(); if (query.kind == ASTSetRoleQuery::Kind::SET_ROLE_DEFAULT) - { session_context->setCurrentRolesDefault(); - } else - { - RolesOrUsersSet roles_from_query{*query.roles, access_control}; - std::vector new_current_roles; - if (roles_from_query.all) - { - new_current_roles = user->granted_roles.findGranted(roles_from_query); - } - else - { - for (const auto & id : roles_from_query.getMatchingIDs()) - { - if (!user->granted_roles.isGranted(id)) - throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role should be granted to set current"); - new_current_roles.emplace_back(id); - } - } - session_context->setCurrentRoles(new_current_roles); - } + session_context->setCurrentRoles(RolesOrUsersSet{*query.roles, session_context->getAccessControl()}); } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index d72f3d81549..56055e7044c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -726,7 +726,10 @@ try /// Access rights must be checked for the user who executed the initial INSERT query. if (key.user_id) - insert_context->setUser(*key.user_id, key.current_roles); + { + insert_context->setUser(*key.user_id); + insert_context->setCurrentRoles(key.current_roles); + } insert_context->setSettings(key.settings); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b946c2cb21e..856dac28ed1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -190,6 +191,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; extern const int CLUSTER_DOESNT_EXIST; + extern const int SET_NON_GRANTED_ROLE; } #define SHUTDOWN(log, desc, ptr, method) do \ @@ -1303,7 +1305,7 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_, const std::optional> & current_roles_) +void Context::setUser(const UUID & user_id_) { /// Prepare lists of user's profiles, constraints, settings, roles. /// NOTE: AccessControl::read() and other AccessControl's functions may require some IO work, @@ -1312,8 +1314,8 @@ void Context::setUser(const UUID & user_id_, const std::optional(user_id_); - auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); - auto enabled_roles = access_control.getEnabledRolesInfo(new_current_roles, {}); + auto default_roles = user->granted_roles.findGranted(user->default_roles); + auto enabled_roles = access_control.getEnabledRolesInfo(default_roles, {}); auto enabled_profiles = access_control.getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); const auto & database = user->default_database; @@ -1327,7 +1329,7 @@ void Context::setUser(const UUID & user_id_, const std::optional Context::getUserID() const return user_id; } -void Context::setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard &) +void Context::setCurrentRolesWithLock(const std::vector & new_current_roles, const std::lock_guard &) { - if (current_roles_.empty()) + if (new_current_roles.empty()) current_roles = nullptr; else - current_roles = std::make_shared>(current_roles_); + current_roles = std::make_shared>(new_current_roles); need_recalculate_access = true; } -void Context::setCurrentRoles(const std::vector & current_roles_) +void Context::setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user) { - std::lock_guard lock(mutex); - setCurrentRolesWithLock(current_roles_, lock); + if (skip_if_not_granted) + { + auto filtered_role_ids = user->granted_roles.findGranted(new_current_roles); + std::lock_guard lock{mutex}; + setCurrentRolesWithLock(filtered_role_ids, lock); + return; + } + if (throw_if_not_granted) + { + for (const auto & role_id : new_current_roles) + { + if (!user->granted_roles.isGranted(role_id)) + { + auto role_name = getAccessControl().tryReadName(role_id); + throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", role_name.value_or(toString(role_id))); + } + } + } + std::lock_guard lock2{mutex}; + setCurrentRolesWithLock(new_current_roles, lock2); +} + +void Context::setCurrentRoles(const std::vector & new_current_roles, bool check_grants) +{ + setCurrentRolesImpl(new_current_roles, /* throw_if_not_granted= */ check_grants, /* skip_if_not_granted= */ !check_grants, getUser()); +} + +void Context::setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants) +{ + if (new_current_roles.all) + { + auto user = getUser(); + setCurrentRolesImpl(user->granted_roles.findGranted(new_current_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user); + } + else + { + setCurrentRoles(new_current_roles.getMatchingIDs(), check_grants); + } +} + +void Context::setCurrentRoles(const Strings & new_current_roles, bool check_grants) +{ + setCurrentRoles(getAccessControl().getIDs(new_current_roles), check_grants); } void Context::setCurrentRolesDefault() { auto user = getUser(); - setCurrentRoles(user->granted_roles.findGranted(user->default_roles)); + setCurrentRolesImpl(user->granted_roles.findGranted(user->default_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user); } std::vector Context::getCurrentRoles() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f9b91a45978..b6de166e27a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -63,6 +63,7 @@ class AccessFlags; struct AccessRightsElement; class AccessRightsElements; enum class RowPolicyFilterType : uint8_t; +struct RolesOrUsersSet; class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; @@ -602,13 +603,15 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_, const std::optional> & current_roles_ = {}); + void setUser(const UUID & user_id_); UserPtr getUser() const; std::optional getUserID() const; String getUserName() const; - void setCurrentRoles(const std::vector & current_roles_); + void setCurrentRoles(const Strings & new_current_roles, bool check_grants = true); + void setCurrentRoles(const std::vector & new_current_roles, bool check_grants = true); + void setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants = true); void setCurrentRolesDefault(); std::vector getCurrentRoles() const; std::vector getEnabledRoles() const; @@ -1335,7 +1338,7 @@ private: void setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock); - void setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard & lock); + void setCurrentRolesWithLock(const std::vector & new_current_roles, const std::lock_guard & lock); void setSettingWithLock(std::string_view name, const String & value, const std::lock_guard & lock); @@ -1368,6 +1371,7 @@ private: void initGlobal(); void setUserID(const UUID & user_id_); + void setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user); template void checkAccessImpl(const Args &... args) const; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 6f51a269e0e..5d3e6aa1b05 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -2,10 +2,7 @@ #include #include -#include #include -#include -#include #include #include #include @@ -79,7 +76,6 @@ namespace ErrorCodes extern const int REQUIRED_PASSWORD; extern const int AUTHENTICATION_FAILED; - extern const int SET_NON_GRANTED_ROLE; extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; @@ -615,20 +611,7 @@ void HTTPHandler::processQuery( auto roles = params.getAll("role"); if (!roles.empty()) - { - const auto & access_control = context->getAccessControl(); - const auto & user = context->getUser(); - std::vector roles_ids(roles.size()); - for (size_t i = 0; i < roles.size(); i++) - { - auto role_id = access_control.getID(roles[i]); - if (user->granted_roles.isGranted(role_id)) - roles_ids[i] = role_id; - else - throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", roles[i].get()); - } - context->setCurrentRoles(roles_ids); - } + context->setCurrentRoles(roles); /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not From 25dc96aaca0abd5498214bc3618f3ea9ea75e4d4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 9 May 2024 22:34:13 +0200 Subject: [PATCH 10/41] Simplify initialization of settings in HTTPHandler. --- .../setReadOnlyIfHTTPMethodIdempotent.cpp | 24 +++++ .../HTTP/setReadOnlyIfHTTPMethodIdempotent.h | 12 +++ src/Server/HTTPHandler.cpp | 99 +++++++------------ 3 files changed, 70 insertions(+), 65 deletions(-) create mode 100644 src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp create mode 100644 src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h diff --git a/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp new file mode 100644 index 00000000000..d42bd77e339 --- /dev/null +++ b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp @@ -0,0 +1,24 @@ +#include + +#include +#include + + +namespace DB +{ + +void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method) +{ + /// Anything else beside HTTP POST should be readonly queries. + if (http_method != HTTPServerRequest::HTTP_POST) + { + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + if (context->getSettingsRef().readonly == 0) + context->setSetting("readonly", 2); + } +} + +} diff --git a/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h new file mode 100644 index 00000000000..c46f2032d82 --- /dev/null +++ b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h @@ -0,0 +1,12 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Sets readonly = 2 if the current HTTP method is not HTTP POST and if readonly is not set already. +void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 5d3e6aa1b05..1cf710e8b8e 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include @@ -586,10 +587,22 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"}; + auto roles = params.getAll("role"); + if (!roles.empty()) + context->setCurrentRoles(roles); - Names reserved_param_suffixes; + std::string database = request.get("X-ClickHouse-Database", params.get("database", "")); + if (!database.empty()) + context->setCurrentDatabase(database); + + std::string default_format = request.get("X-ClickHouse-Format", params.get("default_format", "")); + if (!default_format.empty()) + context->setDefaultFormat(default_format); + + /// Anything else beside HTTP POST should be readonly queries. + setReadOnlyIfHTTPMethodIdempotent(context, request.getMethod()); + + bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); auto param_could_be_skipped = [&] (const String & name) { @@ -597,74 +610,36 @@ void HTTPHandler::processQuery( 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{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session", + "database", "default_format"}; + if (reserved_param_names.contains(name)) return true; - for (const String & suffix : reserved_param_suffixes) + /// For external data we also want settings. + if (has_external_data) { - if (endsWith(name, suffix)) - return true; + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + static const Names reserved_param_suffixes = {"_format", "_types", "_structure"}; + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } } return false; }; - auto roles = params.getAll("role"); - if (!roles.empty()) - context->setCurrentRoles(roles); - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - const auto & settings = context->getSettingsRef(); - - /// Anything else beside HTTP POST should be readonly queries. - if (request.getMethod() != HTTPServerRequest::HTTP_POST) - { - if (settings.readonly == 0) - context->setSetting("readonly", 2); - } - - bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); - - if (has_external_data) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - - std::string database = request.get("X-ClickHouse-Database", ""); - std::string default_format = request.get("X-ClickHouse-Format", ""); - SettingsChanges settings_changes; for (const auto & [key, value] : params) { - if (key == "database") - { - if (database.empty()) - database = value; - } - else if (key == "default_format") - { - if (default_format.empty()) - default_format = value; - } - else if (param_could_be_skipped(key)) - { - } - else + if (!param_could_be_skipped(key)) { /// Other than query parameters are treated as settings. if (!customizeQueryParam(context, key, value)) @@ -672,15 +647,9 @@ void HTTPHandler::processQuery( } } - if (!database.empty()) - context->setCurrentDatabase(database); - - if (!default_format.empty()) - context->setDefaultFormat(default_format); - - /// For external data we also want settings context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); context->applySettingsChanges(settings_changes); + const auto & settings = context->getSettingsRef(); /// 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", ""))); From 49b982747a3a772eb6f2ec417cebfa72c6c1edae Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 9 May 2024 23:19:01 +0200 Subject: [PATCH 11/41] Move function authenticateUserByHTTP() to a separate header in order to reuse it. --- src/Server/HTTP/authenticateUserByHTTP.cpp | 269 +++++++++++++++++++++ src/Server/HTTP/authenticateUserByHTTP.h | 31 +++ src/Server/HTTPHandler.cpp | 242 +----------------- 3 files changed, 305 insertions(+), 237 deletions(-) create mode 100644 src/Server/HTTP/authenticateUserByHTTP.cpp create mode 100644 src/Server/HTTP/authenticateUserByHTTP.h diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp new file mode 100644 index 00000000000..3bcf6552e54 --- /dev/null +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -0,0 +1,269 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#if USE_SSL +#include +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int AUTHENTICATION_FAILED; + extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; +} + + +namespace +{ + String base64Decode(const String & encoded) + { + String decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; + } + + String base64Encode(const String & decoded) + { + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); + } +} + + +bool authenticateUserByHTTP( + const HTTPServerRequest & request, + const HTMLForm & params, + HTTPServerResponse & response, + Session & session, + std::unique_ptr & request_credentials, + ContextPtr global_context, + LoggerPtr log) +{ + /// Get the credentials created by the previous call of authenticateUserByHTTP() while handling the previous HTTP request. + auto current_credentials = std::move(request_credentials); + + /// The user and password can be passed by headers (similar to X-Auth-*), + /// which is used by load balancers to pass authentication information. + std::string user = request.get("X-ClickHouse-User", ""); + std::string password = request.get("X-ClickHouse-Key", ""); + std::string quota_key = request.get("X-ClickHouse-Quota", ""); + + /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name + /// extracted from the SSL certificate used for this connection instead of checking password. + bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); + bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; + + /// User name and password can be passed using HTTP Basic auth or query parameters + /// (both methods are insecure). + bool has_http_credentials = request.hasCredentials(); + bool has_credentials_in_query_params = params.has("user") || params.has("password"); + + std::string spnego_challenge; + SSLCertificateSubjects certificate_subjects; + + if (has_auth_headers) + { + /// It is prohibited to mix different authorization schemes. + if (has_http_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + if (has_credentials_in_query_params) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); + + if (has_ssl_certificate_auth) + { +#if USE_SSL + if (!password.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via password simultaneously"); + + if (request.havePeerCertificate()) + certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); + + if (certificate_subjects.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); +#endif + } + } + else if (has_http_credentials) + { + /// It is prohibited to mix different authorization schemes. + if (has_credentials_in_query_params) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use Authorization HTTP header and authentication via parameters simultaneously"); + + std::string scheme; + std::string auth_info; + request.getCredentials(scheme, auth_info); + + if (Poco::icompare(scheme, "Basic") == 0) + { + Poco::Net::HTTPBasicCredentials credentials(auth_info); + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else if (Poco::icompare(scheme, "Negotiate") == 0) + { + spnego_challenge = auth_info; + + if (spnego_challenge.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty"); + } + else + { + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); + } + } + else + { + /// If the user name is not set we assume it's the 'default' user. + user = params.get("user", "default"); + password = params.get("password", ""); + } + + if (!certificate_subjects.empty()) + { + if (!current_credentials) + current_credentials = std::make_unique(user, std::move(certificate_subjects)); + + auto * certificate_credentials = dynamic_cast(current_credentials.get()); + if (!certificate_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme"); + } + else if (!spnego_challenge.empty()) + { + if (!current_credentials) + current_credentials = global_context->makeGSSAcceptorContext(); + + auto * gss_acceptor_context = dynamic_cast(current_credentials.get()); + if (!gss_acceptor_context) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" + const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log)); +#pragma clang diagnostic pop + + if (!spnego_response.empty()) + response.set("WWW-Authenticate", "Negotiate " + spnego_response); + + if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady()) + { + if (spnego_response.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure"); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + } + else // I.e., now using user name and password strings ("Basic"). + { + if (!current_credentials) + current_credentials = std::make_unique(); + + auto * basic_credentials = dynamic_cast(current_credentials.get()); + if (!basic_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); + + basic_credentials->setUserName(user); + basic_credentials->setPassword(password); + } + + if (params.has("quota_key")) + { + if (!quota_key.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and as parameter simultaneously"); + + quota_key = params.get("quota_key"); + } + + /// Set client info. It will be used for quota accounting parameters in 'setUser' method. + + session.setHTTPClientInfo(request); + session.setQuotaClientKey(quota_key); + + /// Extract the last entry from comma separated list of forwarded_for addresses. + /// Only the last proxy can be trusted (if any). + String forwarded_address = session.getClientInfo().getLastForwardedFor(); + try + { + if (!forwarded_address.empty() && global_context->getConfigRef().getBool("auth_use_forwarded_address", false)) + session.authenticate(*current_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port())); + else + session.authenticate(*current_credentials, request.clientAddress()); + } + catch (const Authentication::Require & required_credentials) + { + current_credentials = std::make_unique(); + + if (required_credentials.getRealm().empty()) + response.set("WWW-Authenticate", "Basic"); + else + response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\""); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + catch (const Authentication::Require & required_credentials) + { + current_credentials = global_context->makeGSSAcceptorContext(); + + if (required_credentials.getRealm().empty()) + response.set("WWW-Authenticate", "Negotiate"); + else + response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\""); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + + return true; +} + +} diff --git a/src/Server/HTTP/authenticateUserByHTTP.h b/src/Server/HTTP/authenticateUserByHTTP.h new file mode 100644 index 00000000000..3b5a04cae68 --- /dev/null +++ b/src/Server/HTTP/authenticateUserByHTTP.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class HTTPServerRequest; +class HTMLForm; +class HTTPServerResponse; +class Session; +class Credentials; + +/// Authenticates a user via HTTP protocol and initializes a session. +/// Usually retrieves the name and the password for that user from either the request's headers or from the query parameters. +/// Returns true when the user successfully authenticated, +/// the session instance will be configured accordingly, and the request_credentials instance will be dropped. +/// Returns false when the user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header, +/// in this case the `request_credentials` instance must be preserved until the next request or until any exception. +/// Throws an exception if authentication failed. +bool authenticateUserByHTTP( + const HTTPServerRequest & request, + const HTMLForm & params, + HTTPServerResponse & response, + Session & session, + std::unique_ptr & request_credentials, + ContextPtr global_context, + LoggerPtr log); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 1cf710e8b8e..619a18dab49 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,8 +1,6 @@ #include -#include #include -#include #include #include #include @@ -34,22 +32,14 @@ #include #include #include +#include #include #include #include -#include -#include "config.h" - -#include -#include -#include #include -#include -#include -#include -#include -#include + +#include "config.h" #include #include @@ -59,28 +49,21 @@ #include #include -#if USE_SSL -#include -#endif - namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_COMPILE_REGEXP; extern const int NO_ELEMENTS_IN_CONFIG; extern const int REQUIRED_PASSWORD; - extern const int AUTHENTICATION_FAILED; extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; - extern const int SUPPORT_IS_DISABLED; } namespace @@ -122,26 +105,6 @@ void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::Laye } } -static String base64Decode(const String & encoded) -{ - String decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; -} - -static String base64Encode(const String & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - static std::chrono::steady_clock::duration parseSessionTimeout( const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) @@ -218,204 +181,9 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTP HTTPHandler::~HTTPHandler() = default; -bool HTTPHandler::authenticateUser( - HTTPServerRequest & request, - HTMLForm & params, - HTTPServerResponse & response) +bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response) { - using namespace Poco::Net; - - /// The user and password can be passed by headers (similar to X-Auth-*), - /// which is used by load balancers to pass authentication information. - std::string user = request.get("X-ClickHouse-User", ""); - std::string password = request.get("X-ClickHouse-Key", ""); - std::string quota_key = request.get("X-ClickHouse-Quota", ""); - - /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name - /// extracted from the SSL certificate used for this connection instead of checking password. - bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); - bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; - - /// User name and password can be passed using HTTP Basic auth or query parameters - /// (both methods are insecure). - bool has_http_credentials = request.hasCredentials(); - bool has_credentials_in_query_params = params.has("user") || params.has("password"); - - std::string spnego_challenge; - SSLCertificateSubjects certificate_subjects; - - if (has_auth_headers) - { - /// It is prohibited to mix different authorization schemes. - if (has_http_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); - if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); - - if (has_ssl_certificate_auth) - { -#if USE_SSL - if (!password.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); - - if (request.havePeerCertificate()) - certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); - - if (certificate_subjects.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); -#endif - } - } - else if (has_http_credentials) - { - /// It is prohibited to mix different authorization schemes. - if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); - - std::string scheme; - std::string auth_info; - request.getCredentials(scheme, auth_info); - - if (Poco::icompare(scheme, "Basic") == 0) - { - HTTPBasicCredentials credentials(auth_info); - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else if (Poco::icompare(scheme, "Negotiate") == 0) - { - spnego_challenge = auth_info; - - if (spnego_challenge.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty"); - } - else - { - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); - } - } - else - { - /// If the user name is not set we assume it's the 'default' user. - user = params.get("user", "default"); - password = params.get("password", ""); - } - - if (!certificate_subjects.empty()) - { - if (!request_credentials) - request_credentials = std::make_unique(user, std::move(certificate_subjects)); - - auto * certificate_credentials = dynamic_cast(request_credentials.get()); - if (!certificate_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme"); - } - else if (!spnego_challenge.empty()) - { - if (!request_credentials) - request_credentials = server.context()->makeGSSAcceptorContext(); - - auto * gss_acceptor_context = dynamic_cast(request_credentials.get()); - if (!gss_acceptor_context) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); - -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wunreachable-code" - const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log)); -#pragma clang diagnostic pop - - if (!spnego_response.empty()) - response.set("WWW-Authenticate", "Negotiate " + spnego_response); - - if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady()) - { - if (spnego_response.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure"); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - } - else // I.e., now using user name and password strings ("Basic"). - { - if (!request_credentials) - request_credentials = std::make_unique(); - - auto * basic_credentials = dynamic_cast(request_credentials.get()); - if (!basic_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); - - basic_credentials->setUserName(user); - basic_credentials->setPassword(password); - } - - if (params.has("quota_key")) - { - if (!quota_key.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid authentication: it is not allowed " - "to use quota key as HTTP header and as parameter simultaneously"); - - quota_key = params.get("quota_key"); - } - - /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - - session->setHTTPClientInfo(request); - session->setQuotaClientKey(quota_key); - - /// Extract the last entry from comma separated list of forwarded_for addresses. - /// Only the last proxy can be trusted (if any). - String forwarded_address = session->getClientInfo().getLastForwardedFor(); - try - { - if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) - session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port())); - else - session->authenticate(*request_credentials, request.clientAddress()); - } - catch (const Authentication::Require & required_credentials) - { - request_credentials = std::make_unique(); - - if (required_credentials.getRealm().empty()) - response.set("WWW-Authenticate", "Basic"); - else - response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\""); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - catch (const Authentication::Require & required_credentials) - { - request_credentials = server.context()->makeGSSAcceptorContext(); - - if (required_credentials.getRealm().empty()) - response.set("WWW-Authenticate", "Negotiate"); - else - response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\""); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - - request_credentials.reset(); - return true; + return authenticateUserByHTTP(request, params, response, *session, request_credentials, server.context(), log); } From ecfe6fddcfbe68477fb10fabc4b0252e08d198ef Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 May 2024 16:59:46 +0200 Subject: [PATCH 12/41] Move function HTTPHandler::formatExceptionForClient() to a separate header in order to reuse it. --- src/Server/HTTP/sendExceptionToHTTPClient.cpp | 80 +++++++++++++++++++ src/Server/HTTP/sendExceptionToHTTPClient.h | 27 +++++++ src/Server/HTTPHandler.cpp | 40 +--------- src/Server/HTTPHandler.h | 6 -- 4 files changed, 110 insertions(+), 43 deletions(-) create mode 100644 src/Server/HTTP/sendExceptionToHTTPClient.cpp create mode 100644 src/Server/HTTP/sendExceptionToHTTPClient.h diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp new file mode 100644 index 00000000000..78650758e35 --- /dev/null +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -0,0 +1,80 @@ +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int HTTP_LENGTH_REQUIRED; + extern const int REQUIRED_PASSWORD; +} + + +void sendExceptionToHTTPClient( + const String & exception_message, + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + WriteBufferFromHTTPServerResponse * out, + LoggerPtr log) +{ + setHTTPResponseStatusAndHeadersForException(exception_code, request, response, out, log); + + if (!out) + { + /// If nothing was sent yet. + WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT}; + + out_for_message.writeln(exception_message); + out_for_message.finalize(); + } + else + { + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + bool data_sent = (out->count() != out->offset()); + + if (!data_sent) + out->position() = out->buffer().begin(); + + out->writeln(exception_message); + out->finalize(); + } +} + + +void setHTTPResponseStatusAndHeadersForException( + int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log) +{ + if (out) + out->setExceptionCode(exception_code); + else + response.set("X-ClickHouse-Exception-Code", toString(exception_code)); + + /// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body + /// to avoid reading part of the current request body in the next request. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() + && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + { + try + { + if (!request.getStream().eof()) + request.getStream().ignoreAll(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot read remaining request body during exception handling"); + response.setKeepAlive(false); + } + } + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + response.requireAuthentication("ClickHouse server HTTP API"); + else + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); +} +} diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.h b/src/Server/HTTP/sendExceptionToHTTPClient.h new file mode 100644 index 00000000000..31fda88d900 --- /dev/null +++ b/src/Server/HTTP/sendExceptionToHTTPClient.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class HTTPServerRequest; +class HTTPServerResponse; +class WriteBufferFromHTTPServerResponse; + +/// Sends an exception to HTTP client. This function doesn't handle its own exceptions so it needs to be wrapped in try-catch. +/// Argument `out` may be either created from `response` or be nullptr (if it wasn't created before the exception). +void sendExceptionToHTTPClient( + const String & exception_message, + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + WriteBufferFromHTTPServerResponse * out, + LoggerPtr log); + +/// Sets "X-ClickHouse-Exception-Code" header and the correspondent HTTP status in the response for an exception. +/// This is a part of what sendExceptionToHTTPClient() does. +void setHTTPResponseStatusAndHeadersForException( + int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log); +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 619a18dab49..370af79e456 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include @@ -60,8 +60,6 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; } @@ -519,7 +517,7 @@ void HTTPHandler::processQuery( { bool with_stacktrace = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace); - formatExceptionForClient(status.code, request, response, used_output); + setHTTPResponseStatusAndHeadersForException(status.code, request, response, used_output.out_holder.get(), log); current_output_format.setException(status.message); current_output_format.finalize(); used_output.exception_is_written = true; @@ -553,7 +551,7 @@ void HTTPHandler::trySendExceptionToClient( const std::string & s, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) try { - formatExceptionForClient(exception_code, request, response, used_output); + setHTTPResponseStatusAndHeadersForException(exception_code, request, response, used_output.out_holder.get(), log); if (!used_output.out_holder && !used_output.exception_is_written) { @@ -615,38 +613,6 @@ catch (...) used_output.cancel(); } -void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) -{ - if (used_output.out_holder) - used_output.out_holder->setExceptionCode(exception_code); - else - response.set("X-ClickHouse-Exception-Code", toString(exception_code)); - - /// FIXME: make sure that no one else is reading from the same stream at the moment. - - /// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body - /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() - && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) - { - try - { - if (!request.getStream().eof()) - request.getStream().ignoreAll(); - } - catch (...) - { - tryLogCurrentException(log, "Cannot read remaining request body during exception handling"); - response.setKeepAlive(false); - } - } - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - response.requireAuthentication("ClickHouse server HTTP API"); - else - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); -} - void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index c78c45826f0..6580b317f6e 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -173,12 +173,6 @@ private: HTTPServerResponse & response, Output & used_output); - void formatExceptionForClient( - int exception_code, - HTTPServerRequest & request, - HTTPServerResponse & response, - Output & used_output); - static void pushDelayedResults(Output & used_output); }; From 5c0d88221fb6794c94ed586ef781d3c26842276f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 09:43:22 +0800 Subject: [PATCH 13/41] check style --- 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 fec4e2bcc72..adef85ac6f1 100644 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $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/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" +$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" $CLICKHOUSE_CLIENT -q "select * from test" $CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file From 45a73dc3b585597b4c9db99236ce4636726d6793 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 10:37:55 +0800 Subject: [PATCH 14/41] fast test --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..f34dc15b777 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list Date: Thu, 4 Jul 2024 11:01:12 +0800 Subject: [PATCH 15/41] change permission of test shell --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03198_orc_read_time_zone.sh diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh old mode 100644 new mode 100755 From baa38ecd61109df0a8f19264dc8bb9e5c6b93297 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 11:32:37 +0800 Subject: [PATCH 16/41] add no-fasttest tag --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 1 + 1 file changed, 1 insertion(+) 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 adef85ac6f1..ba868e29dce 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From a55f39bc195bb462d10d479d004920777ba0fccc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 15:55:08 +0000 Subject: [PATCH 17/41] add test for 58998 --- .../03200_subcolumns_join_use_nulls.reference | 1 + .../0_stateless/03200_subcolumns_join_use_nulls.sql | 13 +++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql diff --git a/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference @@ -0,0 +1 @@ +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 new file mode 100644 index 00000000000..2dd0a37657d --- /dev/null +++ b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t_subcolumns_join; + +CREATE TABLE t_subcolumns_join (id UInt64) ENGINE=MergeTree ORDER BY tuple(); + +INSERT INTO t_subcolumns_join SELECT number as number FROM numbers(10000); + +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; + +DROP TABLE t_subcolumns_join; From b98c123f3a8516788ba9b93168d3ea718ea7c565 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jul 2024 18:41:20 +0000 Subject: [PATCH 18/41] Remove logging, if this happens again we'll have to guess again --- src/IO/ReadWriteBufferFromHTTP.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index f17c19f3cc2..b753e66da48 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -717,10 +717,6 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_REQUEST_TIMEOUT && e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_MISDIRECTED_REQUEST) { - LOG_DEBUG(log, - "HEAD request to '{}'{} failed with HTTP status {}", - initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()), - e.getHTTPStatus()); return HTTPFileInfo{}; } From 3fc4d8988bd24dfff65636c4febd0c18a3e41bfb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 4 Jul 2024 21:51:57 +0000 Subject: [PATCH 19/41] better test --- .../0_stateless/03174_projection_deduplicate.reference | 1 + tests/queries/0_stateless/03174_projection_deduplicate.sql | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference index 586b63a392e..1796b2f1dee 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.reference +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -1,2 +1,3 @@ 1 one 1 one +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql index ed6da5d73fc..46222b69dc7 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.sql +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -21,4 +21,10 @@ OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError NOT_IMP SELECT * FROM test_projection_deduplicate; +ALTER TABLE test_projection_deduplicate DROP PROJECTION test_projection; + +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; + +SELECT * FROM test_projection_deduplicate; + DROP TABLE test_projection_deduplicate; From dcdad686ee5413d0c9530f2a679ae5f42379527f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 5 Jul 2024 10:25:00 +0800 Subject: [PATCH 20/41] fix stateless test --- .../queries/0_stateless/03198_orc_read_time_zone.sh | 4 ++-- ....snappy.orc => test_reader_time_zone.snappy.orc} | Bin 2 files changed, 2 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/data_orc/{read_time_zone.snappy.orc => test_reader_time_zone.snappy.orc} (100%) 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 ba868e29dce..27530c06237 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $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/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test" +$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 diff --git a/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc b/tests/queries/0_stateless/data_orc/test_reader_time_zone.snappy.orc similarity index 100% rename from tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc rename to tests/queries/0_stateless/data_orc/test_reader_time_zone.snappy.orc From 3ce470c57f4632030890cdabfd7f5eef3c5eba14 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 5 Jul 2024 14:42:43 +0200 Subject: [PATCH 21/41] Update AVRO submodule with the array block size fix --- contrib/avro | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/avro b/contrib/avro index d43acc84d3d..545e7002683 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit d43acc84d3d455b016f847d6666fbc3cd27f16a9 +Subproject commit 545e7002683cbc2198164d93088ac8e4955b4628 From dfef400e158b8065b2b7f32d33f5023445c848a4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Jul 2024 14:39:45 +0100 Subject: [PATCH 22/41] impl --- .../MergeTree/MergedBlockOutputStream.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 164658c914e..9d696b70d9f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -93,6 +93,7 @@ struct MergedBlockOutputStream::Finalizer::Impl void MergedBlockOutputStream::Finalizer::finish() { std::unique_ptr to_finish = std::move(impl); + impl.reset(); if (to_finish) to_finish->finish(); } @@ -130,7 +131,19 @@ MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} -MergedBlockOutputStream::Finalizer::~Finalizer() = default; +MergedBlockOutputStream::Finalizer::~Finalizer() +{ + try + { + if (impl) + finish(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, From ad70710f4bf284f7c41c49d7675eaff4094752ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 11:57:54 +0200 Subject: [PATCH 23/41] stateless/run.sh cleanup --- docker/test/stateless/run.sh | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5ece9743498..983e7f55d43 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,14 +11,13 @@ set -e -x -a # NOTE: that clickhouse-test will randomize session_timezone by itself as well # (it will choose between default server timezone and something specific). TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" -echo "Choosen random timezone $TZ" +echo "Chosen random timezone $TZ" ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone dpkg -i package_folder/clickhouse-common-static_*.deb dpkg -i package_folder/clickhouse-common-static-dbg_*.deb -# Accept failure in the next two commands until 24.4 is released (for compatibility and Bugfix validation run) -dpkg -i package_folder/clickhouse-odbc-bridge_*.deb || true -dpkg -i package_folder/clickhouse-library-bridge_*.deb || true +dpkg -i package_folder/clickhouse-odbc-bridge_*.deb +dpkg -i package_folder/clickhouse-library-bridge_*.deb dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb @@ -55,12 +54,6 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th rm /etc/clickhouse-server/users.d/s3_cache_new.xml rm /etc/clickhouse-server/config.d/zero_copy_destructive_operations.xml - #todo: remove these after 24.3 released. - sudo sed -i "s|azure<|azure_blob_storage<|" /etc/clickhouse-server/config.d/azure_storage_conf.xml - - #todo: remove these after 24.3 released. - sudo sed -i "s|local<|local_blob_storage<|" /etc/clickhouse-server/config.d/storage_conf.xml - function remove_keeper_config() { sudo sed -i "/<$1>$2<\/$1>/d" /etc/clickhouse-server/config.d/keeper_port.xml From 8a1dd3c7c805b3382cf1ed40965cbb3fabb04c2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 11:58:35 +0200 Subject: [PATCH 24/41] Fix download of debug files for BugFix checker --- tests/ci/download_release_packages.py | 10 +++++----- tests/ci/functional_test_check.py | 2 +- tests/ci/integration_test_check.py | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 8f3a2190ae8..887c94f619f 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -13,26 +13,26 @@ from get_previous_release_tag import ( PACKAGES_DIR = Path("previous_release_package_folder") -def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR) -> None: +def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False) -> None: dest_path.mkdir(parents=True, exist_ok=True) logging.info("Will download %s", release) for pkg, url in release.assets.items(): - if not pkg.endswith("_amd64.deb") or "-dbg_" in pkg: + if not pkg.endswith("_amd64.deb") or (not debug and "-dbg_" in pkg): continue pkg_name = dest_path / pkg download_build_with_progress(url, pkg_name) -def download_last_release(dest_path: Path) -> None: +def download_last_release(dest_path: Path, debug: bool = False) -> None: current_release = get_previous_release(None) if current_release is None: raise DownloadException("The current release is not found") - download_packages(current_release, dest_path=dest_path) + download_packages(current_release, dest_path=dest_path, debug=debug) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) release = get_release_by_tag(input()) - download_packages(release) + download_packages(release, debug=True) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index ee459ce35a0..d8e5a7fa27f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -253,7 +253,7 @@ def main(): packages_path.mkdir(parents=True, exist_ok=True) if validate_bugfix_check: - download_last_release(packages_path) + download_last_release(packages_path, debug=True) else: download_all_deb_packages(check_name, reports_path, packages_path) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 7c74f52b610..80ac1935d95 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -185,7 +185,7 @@ def main(): build_path.mkdir(parents=True, exist_ok=True) if validate_bugfix_check: - download_last_release(build_path) + download_last_release(build_path, debug=True) else: download_all_deb_packages(check_name, reports_path, build_path) From 3435e25b8a2ece06dca6a23dfd1d6f74cec0ee46 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 12:43:49 +0200 Subject: [PATCH 25/41] Linter --- tests/ci/download_release_packages.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 887c94f619f..e8260cf68f1 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -13,7 +13,9 @@ from get_previous_release_tag import ( PACKAGES_DIR = Path("previous_release_package_folder") -def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False) -> None: +def download_packages( + release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False +) -> None: dest_path.mkdir(parents=True, exist_ok=True) logging.info("Will download %s", release) From 918502202f06b45332b1ed62c11aab060f94baea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 14:07:39 +0200 Subject: [PATCH 26/41] Modify a random test file to enable BugFix checker --- tests/queries/0_stateless/00027_argMinMax.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00027_argMinMax.sql b/tests/queries/0_stateless/00027_argMinMax.sql index dbf7c9176d2..57f815add27 100644 --- a/tests/queries/0_stateless/00027_argMinMax.sql +++ b/tests/queries/0_stateless/00027_argMinMax.sql @@ -13,4 +13,4 @@ FROM SELECT arrayJoin([[10, 4, 3], [7, 5, 6], [8, 8, 2]]) AS num, arrayJoin([[1, 2, 4]]) AS id -) +); From 6e8edf008ff0090762faa875af14b9694f026a86 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Jul 2024 18:36:50 +0200 Subject: [PATCH 27/41] try fix --- contrib/libpq-cmake/CMakeLists.txt | 1 - src/Common/examples/CMakeLists.txt | 6 ++++-- src/IO/examples/CMakeLists.txt | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 6a0012c01bf..246e19593f6 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -54,7 +54,6 @@ set(SRCS "${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c" "${LIBPQ_SOURCE_DIR}/port/thread.c" "${LIBPQ_SOURCE_DIR}/port/path.c" - "${LIBPQ_SOURCE_DIR}/port/explicit_bzero.c" ) add_library(_libpq ${SRCS}) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 410576c2b4a..c133e9f5617 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -31,8 +31,10 @@ target_link_libraries (arena_with_free_lists PRIVATE dbms) clickhouse_add_executable (lru_hash_map_perf lru_hash_map_perf.cpp) target_link_libraries (lru_hash_map_perf PRIVATE dbms) -clickhouse_add_executable (thread_creation_latency thread_creation_latency.cpp) -target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) +if (OS_LINUX) + clickhouse_add_executable (thread_creation_latency thread_creation_latency.cpp) + target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) +endif() clickhouse_add_executable (array_cache array_cache.cpp) target_link_libraries (array_cache PRIVATE clickhouse_common_io) diff --git a/src/IO/examples/CMakeLists.txt b/src/IO/examples/CMakeLists.txt index 12b85c483a1..fc9d9c7dcd1 100644 --- a/src/IO/examples/CMakeLists.txt +++ b/src/IO/examples/CMakeLists.txt @@ -59,10 +59,10 @@ clickhouse_add_executable (parse_date_time_best_effort parse_date_time_best_effo target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io) clickhouse_add_executable (zlib_ng_bug zlib_ng_bug.cpp) -target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib) +target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib clickhouse_common_io) clickhouse_add_executable (dragonbox_test dragonbox_test.cpp) -target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars) +target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars clickhouse_common_io) clickhouse_add_executable (zstd_buffers zstd_buffers.cpp) target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) From d745a66c7e17196bc7dd70fced3b086059ad05d5 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 5 Jul 2024 18:49:04 +0200 Subject: [PATCH 28/41] Add a test for Avro negative block size arrays --- ...01_avro_negative_block_size_arrays.reference | 11 +++++++++++ .../03201_avro_negative_block_size_arrays.sh | 14 ++++++++++++++ .../data_avro/negative_block_size_arrays.avro | Bin 0 -> 37115 bytes 3 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference create mode 100755 tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh create mode 100644 tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro diff --git a/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference new file mode 100644 index 00000000000..912bff45da5 --- /dev/null +++ b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference @@ -0,0 +1,11 @@ +str_array Array(String) +1318 +5779 +1715 +6422 +5875 +1887 +3763 +4245 +4270 +758 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 new file mode 100755 index 00000000000..dcecd7b3bea --- /dev/null +++ b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_DIR=$CUR_DIR/data_avro + +# See https://github.com/ClickHouse/ClickHouse/issues/60438 +$CLICKHOUSE_LOCAL -q "DESC file('$DATA_DIR/negative_block_size_arrays.avro')" +$CLICKHOUSE_LOCAL -q "SELECT arraySum(arrayMap(x -> length(x), str_array)) AS res FROM file('$DATA_DIR/negative_block_size_arrays.avro')" diff --git a/tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro b/tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro new file mode 100644 index 0000000000000000000000000000000000000000..ec785a885dc84d85043d7ae60e174ae2b8756e32 GIT binary patch literal 37115 zcmX`TTgWwGdlr`US6T_Bl@KM^Aj6MHDQR%_GoeUlzu&LfHpb4(n(dm+n*Bxt60C@X zkOPrYJg_1~LMcUTpg9l^Y$`!Sf`}9;5+nzbqavkv5c;fN6yEP!^Un5pXP&w5>%Ok* zURKsV{)6BC!5>NYdHmTWuZuqYt3UX&KOEA&_~Fm}FdzG$&ATRl*WX!2xrTkMj%sUN%MdT7LL*|INQY*RsxkCY|T>`uBVP*?)J` zzgyj`MGuboA=tGUsvrL1FaGIY{K+2;EFWJ*{s)Jx>wfJ2_$$BN|EIqg|Lza|mE8P` zKm5P{#!vpkzxN0J%U}M(|NJ38*BAex77-0;$xHZV=$SAXYxPARy-vtAOO85?yfG11 z-~Mn8iNx1#zCPph+3(NB%(CD4YyaK#&W+0~SIfFzPb&U=S0Q!MZ+le2MagILX25(@ z|I3GAZv7aGuh&hA&iT}sA$)tz4RZu!-BBhCCI|CLB$X)3WgQBDe5KkvgvL$oy@zwk0*rr{!%YE3$`-y_)lonF|hXD)n4XIU*E_{BVhD2_G~ z$A9gSufhG93j#gtBuT;UI1x(6d4MI=o%?h-`d042Z-k+tCw;4aH4r#`U1x1zBGrX* zUxKaNRh+Ic2$pH$Ps*i42&}x`JNVy`rYlWa$b(6?8QHpS5mD0+UM*-Yb*%%UU$9m# zH%5Y;N4*}8#yK@43=yMlpsZq2I^Br&cOP7Y>tiG9aO3NVbYI~9^GudVr@&sU?p&o{ z(tMqF>vzr;8EyDFD&{vIBJhS^sQ=1^y#9z$ny9LD**Hb+MW4Va(&r`Z#1&+;B&STD z>Z%XvC77O}|Fk52r1d@oIeDQ88>GeoZH%L0uJuNIDImR4Q2MJ9Qn~19@Sw*UtJgYX z5c|(v>6Y{F80xNaCDqL($jA~tjbm$0J;KC_T7f$Ci<9-KM6Ng(vm`djt=tr6o6KK* z*dv}50#Q>?8zIkwY*1G7K9JsL_WT@V*7l8n#%C;Rnuf+V9d#`*#PEuCeax12_AaKQ zYl@CC`p5pa$KUyRwd9#lAiLU(LW;QkEu!i$C0}Iveopv(vNqyIUB787#bGTj?x0@^aqOJb8|5zb* zOwJf1EsBxn9DiycV9@Y<~?Ego>M ze&ao(J=!e#D7xs@Bby(Lb1QFAqtmvB>c94|p6C`nX~%j`XH|A~6R^C7WOKSHM2PQ0 zOz+`)eKSV`b0jG$fkS-iY#G7UeG-xIkFH$V}H-l^$lY3qM2cO{oZ57D!E{(!%Il>{4{BdGu`FB83qpD zSdK5Ka0#8Q;?>#_FOg{WYOV%t)<1Ymm%bs6tyyQ?vJzQZ0Ix z7I)W{qszp*XX@B~5{lb5fAjI&v}il((`4LznT}I?ZxoKKJ0-=`+rTBt`v!0Q5~noS zh|bIL4$|J)z)=!M7wwzUhmq{SE??*~2-C*ter#V-ZA0fFFztm^8;V#6e?Z)FkL^Ia zL#|+pFrRwRL`JrHjF|?@=rP~~VfZ2m)2sF6EfB?}if%TzaFw4j`Zo9G|zTH<_0T8ITnPmoip^ICH#{Omiir zw&LWiEYmdB(|6qd>j!^ri2+Wa>>RFt?%Sd1ZN+YNwe-$S5d28ec{I&t0Q(raBw^+X zmYR-)?%?hyj+&7wvcVko2qP}}7;gJfa7HUyptGP6$0S%=4l(QG0VPMn7QolZC8$i#_3=Y z?P=bZm#J9-=G;`|uLnH8L2Pm1LzGb2MIt&ZoAU<8=odb%%OGne^b|+ZU9&JUD|)t> zH*t8vKY6tK4Pc$=hwNDBbBuAG5ijD|S_C|u5xLZ%Fnb@@w0FHdC@brE!cX;$W600G zoaIdv;40ReNG|IcSff>aFi9gDoZrbJu1%8Oh{6NAyoDbpB~XC;0RsOw4+UynJTi5I zPm^vpmTB=szq@xX%)j^hzV<7HY}ZZ&85goGBUVh-iLD`DMqzXTb zUV*w1`{HP#@O@yxCxv0~sVi1gxf|`ixSsB=Zog~`v--dIxN9iT6ZlmgDjipO$2mEO zPYPPWAt`jw{;ZzE8lo%yLI-v61`YX{v^dp%$V!D|)sM$! z0%Jf-HrR)D5$Y}U;Q}DIBu}FnR*AaYnE&~qT0Fy*7De=3!9s_Lg@LE@9Ub)`a(ro6Y}{|Q0yA!cbA1|htPrseB zmvF1u@tM(sA_sSA5oh4AkR#Zq`UR7^aK*fG=F#F-tsq+n)BE7%MiUiwFemH!0H5(2 zqUraoG47$yj4q~W2PyH9EuDAbs&z({%GZbR*|+0HC#=vD<<}q9Tj8;YqKlDME35%) zSDbz$jWcV-m^7TRRKoBYQ~L6^^=#d@DcY%vY)`4id%5WLwiH#fBm5`+sZn^vpv-K! z0ZEzrrvOT5AT}6WI1PS=KS^lGzj0AerTA5TUb4Wizx0FMyiT-abDV9jiRw>B(fC-c zvRCk-lYsuF>)V}60uo-8-ZmY=UN$JY(=kV6xHYY1vP4z=zuXjg}1us3X4!P`|e!jgDyr~en$5-M1jXb|Kw_jTXwy{Ub zNxSOQu_jg~x|a;!^i3hHPgU+H4Ip?aWd6U$yRjK#_teS#&KVBrnPG`z?HvyH70Y!N z<{xLZ2^wsyl~_foE8JJL%uJI-K7zm8FC;d+qXBW5dev^^yw5y$CR9^XNU*Z^%(PBZ zrN=Bf8>J`v&{*2%secMIAb%!?MV?(#=x%d759h|n!DYr@aJh+14}`k>#hq!0B$+VLgdMgw-PNbQ)bwvZTCwt4>1tdYm18+1@_pvisg&~jI9Q84zTsn+ zJZf?TKC{)@ZJ@O@yApF9hBgt6vzXZuLsAsLjtf0FrY3jIA}UzT`MY+1D`NR-+NTQ^ z{Gyh-V9)x-FzIM8rbrsXPNHP6EifxH{UQ_fcOH{WJ{!3r0xLzHb-?9I+EL=pdw50$ zX*q~A8#=7)*0l(=rkfDPxKr^~>&*MzJoT}Nu0<8bsm=!i3qCNIZRh?nRr2jU^Fo}+ zvBb2g_ndMWGuVy)VG-bGQgmdy8dBvBwj=DU7X8Y9BYO=(1Gi}S>k zHYyb2GdV?VqRYEtr&5&HJqFj9B6*RzH2-Bp!S%z^;7oXve%Czu|T*iU(-(xu|(D&#wMHy|1A%}X(zXW}n> z*YF2px&?ccaSsZETX12K&XfbcLa%^8h)tPhe6mFlSe1qX9>?2-g(^Kw-|qbzeY-b$ zB)5K&QWL`+TCJ>hIz4p7>3;%Maf)# zL4vpY(lo=`x;$T=hD!Vk$UsR8$Rb?rJkpK;Vy1X#u9x@{uxg-s3GaMuQKopa zW$34lXLaCgj!Z)6|L%h@yhP-0y>p_bJ@455zC;ATkIxpq)QvTt9hDNWMBLxMW`AA% zTV1*1(u31`X=MsXf|b=SUdDoT{;*vxU#*Pj`i$orUz8VTR{@Wh+3P#slxD)#gGbgG zcQtmIc@?%e_L+|Sfmukgy@PMo46;0D!}#Tg-G34J6Q7!8Ca|e) z1{)7iHEjHSnpL)CxINi8h5XJx?j?GQXbj5DE7cbFh8-U18*HHesPV@3GZ&Y5p<|QP zX6WPuD6E$+G=>0>7p+UopR-z%4iBTzk5k2lhyUs!n#{iX;WDyE;@J*qqC>~TZXc$| zk=W*|aFUkJSn|bVdoMLm8Lb7X07?&^GgF|S<&srX92t25BiEmj*=0mavu zd~!;rS^<~NGj6GXH`Q(!f9(Ol!L>wM_U_(1ETZ{%@tH-`>oFE*l?0Oy6L!8cto_>1 zGwQTs8YG!cDKdkhb0nq>{?=VZS3#Rj+R4Iry#@Woqo3Ef>aQEtF^rN4uHl3DI)SWq?>;&?axzRA%a0K-0vCI0Fw-K9qsUX3@`_EtW7OqIN;-8fx1mBx zH4CjV&G`(gXG=;3T6CvXnH<=sub~~*Q#zOpGj$U=kI!#EK7u1=gL7fx?k8A&cEg=dA#}!_r<$#>DRmuwt9#pznGe-c44S(`o=x;x5w7ozqA*vR#2z zmy9I8nUEr~)PaDcFX83k#jRBKg!)*^Drn@CIV6Haw%b=D;NEu<8+DPG_ zV+j7N#e*=+FsDLKJCL!E(2t_Y5#0W_tHeOHNQ9Hba4mt#gnu42E-1;8^xuawod#XL zDouKYuE>AdUw@=<8CtAkH8@Tzl%^YS*P`faBKgaEi-mI&uhbY{T=L%9BPK&+*65z4 zP=*(YqqW zq7Wy8pFV8%C~j#kZ|K6FxQae%b?+@j(JbJo@}0)1&Zmaku^nvZ{67Vv3iY;7cHT{Y zB27F$^#N6rDOTQ)_3YrdwYaRru(iCR+u+*QK`v-Ai=`UniRyd%Q=d6GTMsdVAbS=Y zYMq{d3-q&q-Q(BJ9nWLoC#hQm=kGtzD+8l=JC;yrHTAEa!a5-+M(l~yhfONf^f zV+56+YMVYH)FZVnA6sG+0Co;|op3&VUtE)fjCfQeudg7(p!q!h82q>Yv;X|SidJDL zZPH&v@aGPw-R;KKUL`4UZSf;^Qk5Duhq(el?v@AMuW?Vo==!OE(`0`=vSx2~0d z>?Ql#mErxfEeea6r>-sV0OzWJw14$+WdhzZjG+Y-BkH@}WVr)`3=R(&+h#w{hN)aL zajE39of29~V; zZpBE1q;{&JC&!CzdLs0P`j(k4K3s(V9-0V>X-|2%%@cYpH3};I7xW5a$1^?3I%4ib zP=YU^Z{q9Nbo^}06N`ZcHWI{m0I-(3eBnP&M;^;|j`hKlbtLW5Z@}Kj?_IHO))Z6L zphyhxSjO)^d?wC9hA7BwD>N?w6TYtBti;=(N*F+Hx9g(?1(c3$STI>b*L!{9M(FlJ zOdzD7^oG3~4918YMjh)5l40cD#I1yb0p$0mUTfYd>3b{@5vUJwUnES9(Z&>j5$r-k(l znEZuUKcBqpVGUZZeW5Wi63Q}YK&W-_WmBAcxfy-vz{Aa6Ti>NP5E(FPK=0zY9_qT4 zWQ4s{l*?8e_s6gr6u2X5O;ZJwmKcK5Qb$%5?In%^N1TNT-$pVp7Z24&7X{P1j(_Rl z?3pWfS=yMT2e{d&`MB9EHy;~woG<9AE0ypZx`akIsZF{xB+LAGZ|NH(kx{|Y-+7dG zK!1cc=`GL;Cn+Ps>Q9tIobVJEj2z`1%?Q=#0w79NP{PB2dmkZ(7BoRkJ?JM7>4Fkxm8#e7zdkR67F1DG<6KGKid_;Zi-Hg2Et@3fKdRIhtIy16XPf6t z3&`MK@M3y8qDYjZ3&L@I1onO#wFUA+LzRIbsMkRm2Kw0|#JId){lS;J1V1r}iibWX z3Lq12d&<+s5^qSX^5@AbmK|_G@WtBww={MZjUR6CPS{!pO8oszyeWckBpW#iZfhY# zY%+Vu;=uBYs$FCa)U=6DS8d)Q|n2kH1m;(OAR6271B#qPI>7*F^}D z0$-)Qi+;j#N%4eCFIEFU+td#-~LA)6Q_vbUfqpEGK4pZ(*j;zrqRnepCW)t)fJ==@h6 zc4c6GhhzZ+qnTP3Q6J+CeL@8g(J8*kfyS_?l~&QZ3#7GOiFCi(fR>02ya&?GEztbL ztI!ghKUuNR?#_;$N5A}#34PkyXKN<9(x!U2bq$bby4QeMnf^vj9~gt4x1TWc)x&dD zx|o%Zg-LHZBotm9Dc|V#{&QpmJ}|g>*ZGNxfH33J)wRme*wC#*gp8drZQ!izTVp6oi1!Z$*k@w+9Z37`bvwol-u{}x>r#ZLpYo@Z+Rt}8g z1cczn=HhT}|2H30>w^^GG!1tb`{*rIpm1<_LW8;m)yC6}4b4vhg%;W{sfsi2E6QgO z*Fn8{Pmc5a7$q@TJ=B@&@p?rg1podc(_0K6Aa1-yOVd~aQfNIe(OG)=2f^yJpzPhe zhI5!KgLAE(Klyk`-);t|bH^-g=qMl=T=4DEd_~$;T2F%FkkMs#?^42%(L%bT25ezb z9~mKTms6HoY1`mx0A+Fi&toels)m=amv0{3(?qxVXOuOTH&(sln&f2P1iwV-l%{ei z?LnJgemRtbgzM}8B&uhSDfh>)d&UdEUz%A%lmH>+)zw}ZBG2|syExp0DX2}`9fAP0 z+{XTW_tH;LZ}x?|YfAiuAFO>M`Zr#PfH%WeH7MA(q-zxdxfIYK@kX)ui$t~5w}iH7 zG&Y{Kc!p{q(Nq{xlX?1b4@9LmxHfFJahK=9i5bi%Ff_sni-XU(q{#X% z`gGf>eao7l&=-kbWoM{tyRmF#&Pd@^aAJ31D1v6tmalhp7(Dx7@gc~pwafOT-n|zN z+grMDzxR;lnrr2_$tQLCksrb0r%8($pu@)d=UW5uJ%yQPh4MsqnX~gc+UNT1DJJIpmJ2lUwPmqz+ALQ zOcZ?t07C(rjum2@#kWtuk$)g=u=P@Q0aBe|pZ`iX3O^u`!(K;*yqhESYbh?cKamxL zl~zJsO`SmPxzmT{??3X6#_Ft$xgN7lPA#6ba1MrC@9Fc)bFN;!5{~6s;|F#5yQ~xT z%kDCJo+6(*=3NUhD=5MRc%6rC6$qSo$_fN-oC$V&_HO>UOqlh)2DG;dld-zP5;RhU z+kQc(_}@PesqKLo2FVaW;!n7Qze>l#jW24{ak)_Q*0TA4y?_e=tg6v>C?esTVz$V4 zG(i%eq*Ace;{oq6|I!cmhIq9cX~8Q2{Do?9#!GE~lXiAB%!l(%%#514oQ|yvIS7*x zo5C(>(Fh1c<|y{~Bt!71?S%uej;DGjCPXWRM;aV2nr$haN=wrOuNvOA_QqKMvML92ecaZjV)g61P+4nx(=K6G*HA|P*w zX+Bb7$dEYq$YfODC3=c7)}LDSQhS+7{K_eN_}NP)Z1cV;?c0Y~?wzDzhw}8-n+59C zB4_Ds`@(Kxe5HCibhCGfsCxLRUe*>1bRr4wj30Hbh2Aei51*%7+knph61W!nrS0w3 z%e`+>GYMs8Z|GA+?m`*+UwXV>nuxF8x5S1RYN8ewnFxxpn=UZvsyFv|X<}&sI16MY zi5_)!BHo=JurjHiBwKxW=k?b?B1^mjOuWl%`KuL}+|Y+%$p^a8aCOWR+;!!gH&cp! zJ=g17r4t+NYvV?Gxvt$Qt-LSN+z zlu~>nGw26cA?CWRS?SXt^5v|;Z#_H_&OW^500ZYDsRS+h@tfhzt-8~ySW25-EWBkU z(vjj%()fOsUIM@s)5HOztpB8f8`K|ZAz``tK9=JBg|^ZQXnx~dbbK;=N7pb4}!&Y;eP=FCfyiGsYljk5Wo@*-~g8Ds}8K6wkiyk-Ojbo-RFcGXy6D5Uxo?>~tj`s@X3hg+g*iOi2xmABIr9-%l|RPdokT0JQt)JM zjHzg=TAl*|Y=r&yA=yBfe2*Vt{Yi22TA?DR99<3Ggd}&Ia(`7}yQ@V@W)L$%L)`vk z=q1Q%RP{42rZtg)*mSsaE3yZ zOSd`wP083<3m?6P)Dn1QLM?8!Aik5S!1H`w38)BEN%@h-sNWnefHOmVYpR7-V?sts z+7OkWk*->T5SKvG*R1xXIeatiV};puN%V%0hD~`8c??bgx(X1zD20Sy$>FU(Q&1)FH|pWA%M1l1gEQ2yHAL=W@DhKxa0hg&SnfD@#;pv@qvy zdvFd)0-Q=YaBZ@dZ~r^$;(b?mD(GCvut8>80b>m|`n1LJw;msQ0#-}%HhjN17QO5J z!VohCwiJ2e&f$(TDcU#M|FW#nkWD_Qahhijz`d~)(w3$qX9eg|VSt5!g|}aqO-0TS zbFPI87-S#R-5vZ(KZpnF41rfyXB=$nDGbu39Davje;C#1lj;$X{gF;cvDT`*a>i}< zlJM#M!p=y36ELkIY( zR~d-)(ita9c{rpyj*^x!V`oI*ZUOmByxv{}8U(=5J)E|0+uiUTYiEI)vJ6n$4&o=j zfG1N@MTiJl>V8R+Q1b!y7U8KjejzfZvS&n~#rMXgAe* zgmR2WY^)~mtzl-TYQf#&-c_?$$j;00X*n>%JZAVtPxP9sIY3dIciNQWk_aA=Zfj{# z?u5h!s^WnH3-%eJc9}rc{!btA#1t;*B-l0;=quF#&Oy+E1S^M>L3w&BVK8H(b_PYB zmm@Y6zwF~hk;M;ZhEp=me=yfsO1XLhflgg}D@Qwn}=jr{?n;pBX1! z@gZXaW`pGu^m|bJt)xzf)qT4(#z}l9V|U&mluQ3H((SI4{l+6i6g?US00^r9P*o>a z81+;dqRfAz2N8U5JA9VQwK5D>>>*;-&vQMc}9m}yOCX6pU zrC#m+cOIY7#tg&fB2XC!1keU#Z@|?E%wY*c8<`GoHVqW7jj^*GsH|c9x3ifWZY!Yx zw@Ie;z_0Jf1sO3&1`@SseiBp{{!8GByE^w1GCns@T6W0+ew&_RKep9rhT|m%--iT%2uGD6GO%7j zv4`<@mgQ0KhfG`8rsAf5;tfcyVU`+t@hyX*oDU>` zJYbz^=6&Be;CdVDCLtE%zfe1%mX^R3s%*v@khn&;F!e#@}e4^R<@G2>gd&M8}{izimO!uE>uC!&07RY)|wFD?O zK{MeHw^z6jY8u4^>{QKVX)010eEsEX;g0Z%(v6ioeQzN8W2GVg!tsHczES``j8(0>9|<9nP! zku@Z3zH_03jgYJN^=OEYwhQh%3|Ir*L*DjFKaib+AfCo64EMWDCfYPbPS-#$5z^Yc|_&SiBR_BJKW262m^K4%mC@xB6SkCGwtahsB|1 zI8*M4MG1dI#o{Slg95(cG=`y7p5amRC$&;eIkYj!3vi_t;8opdeg zZGS;zq(YSeUN8fy8)U+V5~udCzYG1Ts8$`(m`>ad^-9%d=3k53&h<=ATA|pdA-OdB zV}JJ%7dElkjBv&iDedeKK*%DcjNPwZfv_U@<3NKIJEAu3_+4C*T@fq#N+Ksa&lm>HA+Dn^{!)b899-0kC#?Dqaf8RU-6#m*I)2smnc@5{3)e)FM6N>`{6mSj<> zcUZA*s=Rga=!qB5z(CiWxA&hK$mYVq?YMfj)R$09T=GOhW#TK%!8vZUZ(-_>7?{bn z+c}XDsJ-@=rSRzikAL~m`#oS<14m~3guZE}!^Y^dsbgFCEN03ZsT*(KS_uNqv++e- z@wJN~PC95zH*Ymr;q{VjqD!N2x2H#(sWKI_G%EF1flv&gDccK`%FpD)%O*hRjV z34@rJyO!$t>PD9F^NINmBlLWI_SWo1_4_nz4XKDVdH=Kxn28rYm87}Pa`-zB$T~hn zbzZWBiE6zqXhnEiH%V`iA`V^6S9#UHseuteD_Iml6ty`g>}^)7z%u9+M6<|BCG8sR zH72h@r-8p3*cZXXN5362<>z~F?=NIE+rTD-?|=D0lU~x8@9Mpd@%=3;=dXDMv#zi9 z1Sn65qH?*X9D4&Gav_}!^yiF=%w8`kK#4UrlOBnF_pwu;%hMjXsDLtUY@C-usXx5AaT|eKi4~|Q z6|?5=gyxZiBA$|cA#23+L;)#J6l4F|13EmDc)lUWY>(mmNJ5r>%Ya#kNBcxAz?@&7 zM%)$E`P7bEkb;#a?-a;!gHY=LM9R>>;VfyKEN+hTAJ4W^vqqMnCh3E=7Q21^!2_38 z%=bR$7pI-##)1Mid}ZMa?xq~dx2AxA1^(p`628wKt%0cyBF_}aAz7B}0$d!GGA9Ce2ow3f;Ghm#gM6M= zWoqq2D(USPA|*T&j1WQF;U5wPI!mhC8VCwfxEsQ$5m#&{&lzCZU65@20%;Sk;VGK()$C%+&nfTpS9*AWP)o^lnQ zE1K^qH$(=54+>!*8N)7G?|)GI0p9(w|MBsAfBJ7cEZ~4SJp;^aGwn(l`Jn7;U~@Di z8LW2?uk8T&{8{h;xNm6Jw#?dC)ilq7D7D;J=iirUCX8n1etQBed)2@6P=QTRUdlT1 z+!n|XT{L2*$Din(tJ)Zi3%QSz!}baa&C-MQ4%;vPJHTKFIoIo4-%Q%qv{cV z(mx+F3&g$FeW+rr`tmv8HO*o@HC?Ip^v&Q!O__2wMZ-e67#?zK7&sP!7Uzf$(V{@R zJA6>%G3;IC1j^%cvLxodxp>k%m45=-7jJKrqIK6eX?L-kQOr~-<#AJmYH)yu+?LE*XVJH zO`P(6nJyn{TD>odQdAzADohG({O41Q!9RJ}?p2L8_jy;kX%;mR12Rmk*s%JZr`ze1 z)>o$k&1Xf(FZ=UL8x%=q`o%KiUYazjXt4|rY{}HSRyK#p=otn3sD2N+U_kFw_6Bl) zIso?Rk0&q6`(kp^8J)S5M}zo_5BnQgK)uZe;uVorV)wxA#OOSk|g^h zmL{J7g7GB)5^c7!R1 z#};9pxnGnraBo27zV}83V(&>BKpP_8(32zKp&-Bh&GC9}DryO9Qt8P*c-&8?b|_#6 zxt&ZQpKGP;@ZR`E!>6j@mLSt7yHSK7B~L#hw1~;^jWqAcW>qTTPH)xVlOTOQ%@1ha zvsRrzd6vfS*9{yp2pSB=CjHi9NY(EnY=PV3OAYdgt7g`2V(%(qa$6BC;o`{v-J3mn z_d95n(;-(sw0Hg7=+6Nt3rTU3fn^via@*Ha_rATsbVVxx0D?}-s8J9kzxv=jnybyD zFGpI(<9Aa#y!xef4sxx4brHqqh3(#;ercHithRJ#*r?V8d>VDnzLz%*jAYKz>Ts}@ zK2bz4_ksAg>a(H^zUTrR(!Af;n=1~+E&~%-NeF=OSO&yJrSFHrVhQ(jthTFOi}A(r zp;@fLv4U?puoK9tp+*2kk-_FiT8Z1TP7RI<$?7RIMu*swdw@e65|D(`0i)X;OI;*& z4fqm>kX{;<5Hs`(JcQRw$x9KC%O0h)4;<~Z58N<5>jO9TAL)b(ICfR1ke2`V@pYC8 zFAEK3@TCU=0{Q_?-(CYz# zCShv3gY>VbSp7Vx=Q=bO+XVjh&Txg$*1 zRfOg|^(6NMkxihDO!VOH&o9l<&>;iHF+5)v{WGHcEQjWGrue9hK*)VwMS}kSCB0e5 zG!38D_w`2@Le|| zH20LEpI6+bZ-lYh!$UB9?;u@F`29;!Setr~b0dM35F?sT7a(_5O+grI_b^8VOUz4D zxONE=DLevMrr%F9k!&BFbOSqvcM&m3tev#}JL+$~{7BfFJvssu1f5|vEo}79mUL*f zW~?Rhbx$YG5ZU1$U)sF8hH+vO8`cYZO2X!j%CAZ&gfH^&A}tXue^NuFo%@B$oXvxA0gJL;eRn4Q|J; z?E~=8<}tEq55h33SQl$PUesT|5Y2(X-Ra&89`uGU4j!2q-N*je(q>sX@eUl1r&g&; zJLJfBydh%9D5EA4h*iu*a;rLF5L1J*$y|~$FLt*yt8FfpefTa`byx}(7Te%x<<>h!(llaT& ziFuh^L*UZKwSjp!XM&T_u#M(A>F7dolD)9>Rx4Tskc4_}e%yiQW-kMMa~>L3KW z(ZoyPxQOts*L8LJ=2w5pv&QV?#eVftYn;0Jtw%2y^)9dTjd4JR@F?0Q&drcd*uy^; zaNpLw;mQZPnEo-7vQ$i?dTYuAquW_@IpS{ECP*8`NaLofnR#O14?> zSLC#OjbcKj{6_L#yMyIaL(1JSj_LRr1enN61zX7he_dc#_$luMArfH722DLq(Yya4 ze&r(9Nacb$t3L0@9}T?L+v|LaU|~>Udq0iySHI7T(#VHkV;jEc1O4@3js{m&5MkYZ z<)kv2v0MB=i{a>t(71)Rn1k2m;DpQ;uK{a zHITySoy!3AMk(J@qt?sg4c^d$BA}7y`a8U)Eh_@U9Jc12cHp>c+TXm87THH{$=$o< zx`Fsl+WnT)uo4dFg%Xfl?o;rk+b)jDyhTK5?ZbZMJi_-WrQlg}CC2HmTfyR4EQr}B zAcq>q$!K{}p&&mzGwXls#oS-+`@`5?*IhN2I)8+el#)pkX6znm&+sTmH9_WDisJL0 z-oK*t|92`CxMX5%Cj5dptbgxH$B0+HYND)r8YR zpSs^Dbq}a|wf-eVrS}DVjq8Y`$?fOo3nwI*E(L#rf`Xs@)ZysO!XMH`0}Vt*a{{w> zK(~B8ho~7aMY*yAPBozd#X>o$w95)ac8W+I!42J@?8~Wjrr_@=m^U(a%3^gk#xF?U zEuVlz+2J|JyN8m^2rylmyD9oS94Xo3Y5X>y>#iYt`a2+6yJm3vcZPd{_!aSpUk>eS zUZy(g^cg2c^FLuPyt(f)1Jxh{COP2wbl56ey;PnrD%1RYJ}=h#P|Cs+s35$*#rK;T zRVkdyg(t+OBGBu%l(`Vv?t@oF=($x?!Dn_X}w2;pG89eVNA4?e| zYd$cxsMz5#PT!hB?6L6GJNW*0D?a-n7U@L4H7qla!|K`0X@-)~jSohyj=l>8mR~ zmW6zS&p)^%9p5saF{0P24B|~Pk26MBK~CqLw~M+v-&DH~r>Ja$9cEbhLxB(W9OZ?f z30mPAv@z6Y#l@t)SE`ITUg6wTxPJ>Z`0r9LiUCxyw9s4(Eu-*mnI+`MGw{#}mM=%O z;^e>yxRh@e)FSlVef>|Dx?b$1DAGcldi5w$+)cxPv>AtT?(1ZT+s(O>_+x9@&%=g_ zdlbwOeE(u*dd8P|`8?q3Ow>B|`1%DA+x?NQW#Zj>z~ zszAn^JYITW$rF}Ri6iad(Vb@@?MYW}7;Yj+=sK;g&VEU!?g9=}D`b^313@rXQ9=d= ze%2N#ciUSAFm@$CCHVw~|7af_hFk|~Iwxus;J1kLVZrLtfa{zT5ta{lW%@rZOa3rA ziV-9*l9^TNy|*MB+r?#wegQ5So0dN&TYg&oe zCYjz|#`Hy= zY1`wTW&ntFgg7pQf#J(Hd38?}1v|E^DMOb>43OAYi)t}x!2CfkUZ|)+~NpC0< z-Zu6`h!?!oEwGR7&ZfHC3Kg{+owt+J;*_iJ>Y7e@Z|WRt)*K_i&PiGC5**KzkB%eZ zo?VxwK1iC7e$)qPMH0@n55vPnPyLjpb|wyWx?#0{^3?E2WPo2L@z zpv|1+MG=XzS}aK*fWjgm0Pac zMPL3_W&7ZL3FAoEM0u|3z%VFqi$<+hk#c>s=xccpH;mqe37|;MKe-HGNPCG9IU7o1*$=RG41gy9 z7Jx7lyvHu(5PTnZ1fvEvi^FVMK$XN<&%v(>{BRpc9P$i2)Am=YCv`L&K>EaWraa^kI--Ah z5k77TE^qO+?BJ6!^rg|X76}ODDb5zFa6`Wn!10M#+L6_JqkBg|-N`5U9Bu~FDw+ao zlt;kTr@|Kdu+Nqw-YISTP%(Z8iYx8KsGD!`6Z+?W5)-d3gS-Ltv@+&)jv80l2Xi2! z{rpk|2#8hn?ENgW9p4Py`nH8>3$h^GRxr*g34PXHK*BUaAW_8Xy57$74z3cnFLe*Q zfR4vISlS#`(tqOuKtJ!Z?goEe^Xu0+t?j!>>z%?Z_3>RKt zMkTx@k3*|}dtb|c=@~!!RlSCw1$%znvH~p9`}4d~kpOT(bQgZ*LgEYIC0`Sl&X*$s zXVRfsISXvL`Nn%w?7n%P`6>V~Rxerc$dsPQ7yiYZ%+0g#zjFz$L0(C4`l*P5RMJlB z5{GoXibJh5x`?WNj@ks(Ea>2TRed=reSMVZ?R-@5w-Ly^B@UWaOZQb1?X*+dPt~)2 z>!P8R+;rQvKDM(@>ut%q!43 zT&)tUqMm8Mj_xa6hdNR|%{L?S>vn`QJ`dYQct{VdT=zWAFda6DFMfMkYdjAo>==G% z;Zt~XZgkv}*g6cT@%xt@2U!M~Bc*tuc_xO&auzF>oHFFpl(=WXk2PpoZ-^o3%MS#p ztV9gl5*if-{K8!XWhHC!VggJ$)KJy8{XHV^jujDx>`dE7%T zeV2Ah&qJncJKxY)t2hidt8%9Q)EJD`_raDAL{3qRZeG8@HNvm_l=!}_R%a{$5P-o} zpEu%?5-HtW%56MQUPndc*%r7Q)iC>KNT)sS`N-{Rb(dFYmq`Xg;ER#nWJEuh9c+wg z#O5p@CWp$S*PSk5VRLCLh2LeT@9Gxb!xuEShM-n^h5U)6i1mj{w)r7&zj`S!wf^p1 zrN$ViGVu~+uQIs!8Vq`Caf2GWEpWgy>rFyeCyl^0?MtoRGv=lUBL zew-gNSwvkSiltjT!@CKR5Ptivu+dgwSEw0%xv;5%J^3Q4=i9d8PbC1qPsH^rLT>mq$@l5}hsJE_dhs?%bzbZ;*$w~k`oD2tLTni zKiExTv2XkxUq#1Xo#ZRzXv=^i)6-wC#U{B--7&M?@_Xom!&|{>I+TWGji`kOs7m!B z|1e4j%kiHOc!!GqIVb0Y?)NU9jIwvIerrKK;r6(>qtVraewL!Tz>YX==V!;a1@`$K zJm*i40OGkmYx%_vuazO$miL)<@qYIO8r3KGIQo}L4BQm^zO!ut+d&^0Rz34F?mX=D z#^4a`U%9w|$EE%E`9>k%C(o_$iNT-2J4Ogo7V#@)_h8e^F!7@L5V=7Vrx9K!~&$vzo{%Gr_w*7Mv#a zN3W1S4g`c5Aik5Z&Og88T_uL$-BMw(EHvUiX)vbu2GUvLBITHL+_7E(ZX2FRE^2h= zodrys4z%nRL+Tb-Z;V0rz(o6Wyx9WS2~XeA+;NRa$~nPK*w3~rpYxgE;PSGZ?5NL6 z+0QSf-q=~_3vmJJpcRmVA>IT>hUXjS`dr_g13ZiCUE-qKlPfeJ4V%e-Uyqr zU#zQ9>57OsLNOhBEmF*t{@{|bu`9{o14+3@c%5Y!K)H!^WF2-EDHtPRw$x_Q&9@cL zc!I#}(;7rkF6NSx%uJ8o4OYK>nZYqea6s+Qv^0)6gzbQ5$imijdS3|*0m)bXHHidH zR50}%0suL@Liv+yscW<$G{9>bY1aMUFE1kGS2s2-Ke^u00Vk^gaE{8dT8N3HqpI}| z%{OngI|!eYy?1?|qiXP2npuhT`Rc9nw3|OHqxjvT1rwON0+W4p;!7N}CO5fCOv;aV z@19|0VJ2EEe0;M3qPUqLn?oM)lZ}YC{%e=>I^o;}0Ry3wdwqmWnSYoU$oSyjJE6z> zwl=E+oaQgBV;EHySHH>5^IZ8VaP|@I5F+p@C!YiVJ&#Fv^yU-aLh@(R^HGNd&oRvF z_X^mGmSh<2J-qNY4jOU_9)qImr)n`}1p~gc1ahgxJ-zG(cNPGHM)7fAi46AKL^Jzg zpuM>qJ6*YfDf;?2&!fd?a9UNr^?=!IA07@%zq-(?i=Ok~~FW^WVp%p8>Y zLtONawFiU^4!}J(E5==cX5BNtc1cs)mFyo{y$bakj>qMja!44i2fLV&5kHqP{-|weq5u$1OK>=)Fn7aciWn(B>HoaKN+Lr!*I3)GI*jSr4G8n z@N00RfL_V^pgy@=!aUz!CM_3~51bTZ43xMl(t#?har?T5lnmaA5Akb@Y%{0|WwI2k z>r4L%8Z_C{ucyrhbm2_1WehzvL5Z1`L!|j*=EXD*vG`fsIZ%;yLc~NOCQCp0OPl%P z@1Z@t;#XBGxu*nE%vPEBl%XPj5rL4esEA~p8Yw1%-_f646!D~D`(0}dhKw}&-qlwo z2CTwFq#j&nNmv5a^Yu($AyEWfn5-wTUH?F+!+lMzTUZt)5P@c#K?9t7W;$>nW!yyj z_Zk8DHAt4LMkW%Kd>VTAV^`$RvzJFo`oFypLp8DvmDwLGv1k<8r)v6UHnjqySORfM z&<}w<-x zZj0Fm@^JpLK}W~-GTBS)RwC1Zegf@~5DQZ0E4MmzN%s85OlMK@b9QETMp--Hi@BdWVhv^qs=Bi}i($O@G|F#^ zg7I2jFn$|sckUNHd}B>@mX=X>G-WEfLRUfRhv`p;0foj}@*u*1?=#Q}bb#Z?`c~<% zYC3Ge88@FW0}EIXBnBr@OFci@|8>EuIPM}>g^A&CP4~M2K8OxU?r%8$M_Ni}W8!IG zsMXn|ptQT0#LcEn@ueFDc*B@%+X>Q-AQvRQk!ap~+gS{s)uNEGxA)rvb2bcNSO(Zn zxZ*p7{g=Yo{ld#w9wX#qz}buU!va-6cCTP&A$IXbZ1Z-DJ~Q}12^$IiEofl1H_WK! z7SXMwN$=&$y6V6>j%TepE9J??V5P9^GtL!Q8p_T=bF4DzH`6|ZSBa5FT0o?QtDrXH=@Xc| zzRZDD@1ddUlInhb$>hi9$!gm70S-%|#>;cMhi^d?=paDdh0k{E&pY;X6-oY76*1TE zkk$Yxs2opA$!zdHWIahm0Q(K`|NMAC&;NLPQvcEsx*@(0zHpVstLTzK+n?G_1Y{Io zWrQXT0A_!1fpO@E#@BQGYbEl~kLBKzemR~O#aAdio+?2~=OrEQGo_SPrVp?^036q& zHEC{}18`E!GD&3gu)$Y}2?!St@kHbHo}8<&=F%N42d!Rnw*B9@{Mfi$$9wMd*e`G3 zHIkkM;2OWF>lBSX#Cyb-AcocwQhnD)W+Ga&`8Xo9q88zPa%x7JA&^m8qyNR_UB=%4 z>5ePYOa01(Uzl@?JPQ|XS3Sl)iDKFQ0&o$}Wes|TzV^UHBt0n!9GDplu@BaQt9JFx z7ymsYjC4yt5H=d?Py%S1K3W=L~-+fj!Q4CoUK-oWI1rSt)!UUfzG@&v2&i-rltOQe;E;Qszr)=N zJQT(ncc0^M&=3edkQ=Vdi`p6d9!go!mU3u5AqnW(dTh^jWPW_7?)8a-I?-JF>`tD3 zuUyC1c>o!`-j{etz~EjMp~)rS21f&)QYa-@)Mv>&`zXVICn00|Z#qzqu%I4n7KG z$Y2ux9;Y9@UdxOp8K%(we|;>CEXOJEoIOK6!y0+H%1afw?aFaq$trZ{iC=!`_0VOJ!fGSgQRdr zQL*WP0+)G~WnL}XA1};3V#~H?Mz1~zUF!6cwd_%T<-k__?TfeyXiK>-!dpiPo=iY~ z10R+yu}06OK#l{D)jyZVpHDspSP)*vAvXcB=F8p9TTA*2J~z0G8q(Q4 z=Lq0F&rkQpfbDEKS7n6%cQ1O5f4hQyfvQ~z)0Fb{tM^W9_sd4kvjs>c%6s5wl9vgX zCM`NGUeBc^co0}=SHQHknJP zuq$rnb*rvlJPQHlhMZfP{z<@pg@l;j#%5#x=5iKM=Mosa?^&fnTL!5sFc}Qgc}!;T z%_kkk4fo@UljOUrnm6iL{_a!?6EAK3`065iZXSzo_mj;0j7a^dyT5zU4&4$WHtq)%L)yZ2xiFpk^DUPkund0lvvcsVm;!90 zytO*V6JGvKzJz4HrX0Da59%(VEWVMWz1b{Rm0xM<|J_BS-jC<=IA#rQ4WiegU$f88 zt)xRlhzbLstA4SVXSK?0x}#P?U>Juc3=t03c>wdW0?R_s~vDB(4O-XdMjS@|9W|-yslMW zru%?@8P*Cegh_?=RxHQc%04}rOj`-QPQV1+=5dYncz6&N`Bd{jrvc@DuE(z*{CHAr zx2n#AL3-$pjEVpBtu_nbHF`AeKu3Ty|f3| z1d%cH0UD&bJeXqX0*U}5FHJ{yrEU513ldHqP3s$#Jq5V)gaa6O9smGNZDSC=FPj3j zJS4jeD!!hCZU&e~6TzwUi?02}+Yy-}%rD_h7LVc9f3_a2PrC{uMytSgH!JhMGE_l- zklLH=Q2PVE%MqZQifFR5rIQ1=ytOL*2EPqZ}SiHku zfBUjO=6YSYL>k^q$1gW@2Xf#BBAUSm|W=>(Or#sCYK)&%y|p)`kWqlchwPj28G4NXXak`w_+#odeXBBs*;t%3`8}YQ zFWfq_w2Skq=swc2#BOC)+f+n$$Cz{d;{b|GWcVqX=pX3vBU`fW5cVYp?ifPsz{`NF zE=Avj$z-eAx~(sfY*wi)es+n0l>JSL>F79Ec*yoK)8TuM6#a4iOf7FGu-%a9ECqrp z!Q66SbCM&VMJE^XXxu(=*kJv&3kkd6`lgnz(K}2_fSHAI9yiL71wx@>_)gL{+sd?d zm#<1isB@Q){PDuh*F#psNbeD+a`5AWkHEt$VQ>HDg}l;N@acoVz+EK zZ}jhZ2R-U;j=J<~ju6N(3KTVxplo$YmB+mYFh?uc9JIJ8EdYuL^=gA}`R^`J*5C`l zHU@m)X*qo)hz0^xfsFV5lQCtBn5d#bhx>=>XXd;f@#8IcuItN!NF--{-s-kTTKc-u z>E~ycw}*?*SAo1>Ng~|#JI}GB{6ln*N%w$=ryA3&vvr%Eu~YMvdS~VBVJS4rhv$hJ zv&}y{S4EU$z`*^t{8a7*m?}f`1}3oxKILvk7x$6YK9T;* z;dRC57?cqxb(UJx{X7AYXZ0X!fsx`T{|?<(3|-p8d5-(iw zV0Pl4T==gu>L&MM9oT`@VAh4lK><}*S-Xgf6xV~Z3{us+@&R7p@4|mQ2pg?!{;>iw z5Dke0X!c}VDq(U%&9@^=pa*yXp8!JCH+8;CDbWWp=>wQSiJ}pl7Ev0|`T=u)CuBO4D6=yo=)%iVFZ}V6)zqPNA7JU^4gS zL0L7|EpU_9|J`NnqT@5$!!Btf!sF_xr)tU1!v!=X=+e3Ht^(8mqC$xt_F!M!dUIq| zZ4uXTS}fs@#!h`mqkO&YpvAremHHhTXG@25y>Oo7@GE*jRF(WHK^vXXtzBMOG99g! z?~Uv4T-L=60r%mza%+!A$W%o9bpbWj_1+y(+_hL;Cyb0DU$sZcFk}&wO$_zQeOk(L z&u>=<-9&*l0whUb4CQp<`i3|jP*=AMBVSQ0XGehkp%%e|s&2`*pqu_wg{%zGs$_&{ zrm+RS@~MAI?t+XzaEWQuzNGz ziI&j?%R#qasZ0<#?TBjB!}(~%z5k~`nes$hKxKAZxxs*gT6fbRO@p~PxEQ<$-#x?B zudd@EfDNHuyyLLXPC!8b>`&rs1C)R$z8g@fQ;Wy8YU)gZO)kG zi40&F1?hG!0L}#@x_!>-+?=mLY@Z|(k`2o|3fK9I)Eqxl$t?{` zFOhqfhSPMS>Hq@QV#e2v-((W;HF&)1RfU z8KhO^VB(^_q5_j5%&SCr_2R>XU}^;+hnNIS{JvkoXV<4=+`IU$jqhTTHwv9SeR!{czq_ zwNV;zjlCU<-`7LNCVu#eXF zt}PE0pPr}G_Aj(v>F?CpI+C^>ZTYCqP#dDiB6222FwRD z@Qj-~cUAW35kR6UIqqMpOWV)80g7q;9@G%)N#|URl2RwP4*0W^MrD5g0wNnQ6KCuL zXg&B5MgsOJczKuQ&li<~4~}@Aet+Tya!%G$46>eG#LUAMeEEGvSaQmLu&s;-;YaLbL^NHr4{{Sie%<^Nc&cXMx0I7iA(E0g? zko`LU?xj5kD$()@)kDTb%!bHI81!{cQ-X_f020;@x9KdWs*Mv_S*Jt6e;GE&!TSMD zI3Du%+9w(&@9vOkqX%y%;o^|D1fot>ynmH^AV0}HxxHBMmFvX>Ag?1#rBaZf){Ojg z^n~kGh@vZi8aA>t?gt|S`cV)+4Tu*T3voQsh{fkWx&U8D6#(+*j}W>E5ehp8aDgQs z0Ju2bU|~dm!R!X%gmfz?NBH3$$Y)ewce?JIt0LnKKg)$F=I;i`Ilj*$>4aH+;sX4u z>sV2X?mO-xdJjlqm}~L$p`=SkN6&#Tc`Mo63H;F>AbWcfJtvjCAwP%v^!=l~^GK`k zv8ZS25l^q0$-;c(4hIRiUfxtwGn&~#b~U6&m)14nyj6mzE(tV9IFKEyg!=-lk|Pk8 z^Fg7CbFa;(X~(HSj0rzRaoh8P_w7pq*0r1lpw7Q{A+y?P!|+;zZkXI&khN>Ihx|lA zy0$`e^F4Im;!GCLhr0qb3`)Tui9WD51O&!?*B?+}(5a}7wE-4<#+vle>TVRo!nm8l zWcWVoAbkKDA}>xm@DXmTitS$r`Sz5NC!f$Kil7{G(ZxNk4LJGbQTtACJ-^iuPZUR5 zT$3MVLqBLLeQ*kgknKN=e*)gQ_1^(g3lEE?gK5vzyp?$9y@PKEQO_U$&$kYJRj-?{MXfLQ+PBua+t_}h z6n0G5^A9eqN(&&>zW3LUMnm!q&XX9Qr*YdN@A2Uc4@YFe+$PHekRFHNVk>l9QgEPA z608DPV+kQ>lDGPUI@{$1P(NNIWSU*CPW4VcUP^)Nn$f=iVhm8pT1>=;ss66~?5A4t z{!%;pX~z+-1&x?$v42ONM?ucUb9pMTnNg{$Q8HByPJle4vQc!%70~%R?9PBp%0uvL zgKg)vww^vv8T#PyLuBQ>rLe&Ov<#iM7(SKQNAau~rNmBO{pXeS}*q=2T+r_T+o}YJUD<7sG9YD5|-YPbfdCaWE zZZH(f#E-}P(rd@kA3(6UB|J4PdAr_Ahqm$dc6M50ME&jsKQ~$L$Lre5*SfF;?F&Oq z*8)lCp#l4RREm;ca_lTC&Tkpv^MD$QY(5jvd3WT9#|7vY3E1Dj1-;~cV|8~Psm$d~ zCyogUd?|vOjbsNDN?Dn7qU_OafBgcLwir^7tI~KiQ7>ccRztROz#-Ia{6}<)??ycS zVX)lSa?sLp9!kpsBb%tU+%#!#<#fu~yN-bhf&A|;!||fMC2=(_njwICw@l=DhQufF z12H<-ToJfrQ1XX>VHG`JA+!(w;vl)IH1}&QEhL|IqPna%PiW0I#nXQS@Q}Suc0?w# zS4mQ;*|t%DYU|W)Q+ZHH%yT+N^yhn`LrNvV4=G4l)?N2vP?`7J>VAc5-^C+ zc=9(N45lvc-rTSgzbFexv(`i@9(L>C!*!x>iD7OmfwRlG1Aj+{O}fPW!;5?qUeyQ! zX!@WO8=kShHvJ3a$}Ix9tQa&8L$$?L^nBq6`T!fCS4=7U=0Is568cROIJh|$H( zD|-X12ZcY)S@i+h{QX}pkf^1Bk`M@hY`f>p%^twspRd9@gp-S@8D$?h_tQTy;fPCD zf-42zJwUyPEII%zjMXiubI=WUrB+6r-7Q-&*=X20^3Gl`Kpl4E-?+RIZmh*Ru+yR{ zAck+h1!Rd3&uRzp3VipYDEH3<3_wFc{Jfz8Qo%gPaRRI!8;iT4bW*e8BTC>c@5+$u z1>V*-jQVoxJr!8dPOw2pi{8y`e3*5bn41K9L83Shg4g4A1(ezIQ$p&iAnKWA&{X^M zu3@(z#PthaTJ^78EC|76NW}t>LWXp*`l&q$Kav<7>a#>+T@kKQqj9(bo2&;^g>W5%uS|4+ zsqGuEaGHRg+(xd(^&Jw#?cN_DQJVoljC79>8k&kdDYoJ_k^&9d|9uG|aVdB4uVUYA zg?V}a_69n(QPT<5!TM`G`+fg9gSe<*QT|mkh(z=g?ChtkP?nnh9UQWJ$vw~gq_xt|K=S4Eh#aH8C zDv%Sior1uV*Rb~YQj;uVDIv&3wQ=f#TyFpW!}FJg_BD+ z9$;eUbJCo2Wd2;glF66L`~ZcPPh{LedBS2+krhv>f{H7I@}4NVWRz<29DohdgFt-! zR(f+)BDB8!^NTh)6yRISwYeDsWI{TkA)y zq+&>I4F++R`KJoQQTd$f?#o)Lg0s9nlUQ9N5B@&>F+%KYd1%f&AJGnkxaal=E1V+!C8ejj@ABP=~H%FJGV8Z8K<4Uoq?dNs= z6s@PSjk;19Pq~~jgL3ueycP&qWy=%R2>k5%{TNlwF@6%Fhz| zE5iZ{epit59!+3159An+dHg7^vs_l#@t<9oLxprMzR{vNH~1mLyO}L)Q~=Wa^)bic^uu07V_&%GIqxG ztsb;lxpGtfnilC+ zlj^M%X*whxGEzYSnz7>smMeF_!dW@Sd6n$gauE2@n_IipYnTMk(>*8RNk z0wkKskKs#K8#+cNqq4Fuiz3g~pW>Om+GOF6agM`c`?E%`zhLd!uU-7TSmI>?f$VL@ zW8ns0D^U7%A(@!rVF?OB7#y}1YIZwdk|Dr;`UMGjoIOCmaulqVrV{Vt=|ENh+{wSb zK1e^iaQRDHfWl6J9Pga&&5A5R7_@c3^6#L*UUj{yU+Dr*gX_vQp@S0(M!>X9kC{3(5A52dS`0g@e$*p58_K#*xmXzD6F{ad0aa z32^BvedOVLyK~P(qAdsuU*yMzA)vAKEq|PuhR1|+pa3Ws2Xbr4{2yPe?ulk@2tsZ6 zL#JSrkwBnD)PM%78;%*K!XM&6t6twz?V6-$+iNcbwr+&r7ydCb*m2O`JFGdh*K!Q) zSoT8Tiw)gfOPzti;eP?Cqwp?4?0ETs9nM$7ufUaWPrzv$kz| z79Nk^dv)DDngxs4=teCs4#g+1M4O5xy-g4+E_RpjH=~04I*h_+V1lxlhEwXtyClQNtPVx~C^|NrDCN=JpS+*K+EMwu2I*DoAq>T&>War8 z(px|C@FPb5d@af$_D)S0Z?)%R#d|?7Fw?-CZ z1i#)lxMip!h1k7=-RoHnU6H0wWh%ETI3+-{=YUTCeZSSr&fGpDj>59nw+aY7&7Vj6 z&KvSVEul%_IZ7o!vJAhpmNKV!p%eb7=~1q1h! b|M|cD&E`M+%fWB|^mm2oU;oGd Date: Fri, 5 Jul 2024 20:15:19 +0200 Subject: [PATCH 29/41] Use Common/Base64.h --- src/Server/HTTP/authenticateUserByHTTP.cpp | 29 +--------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 3bcf6552e54..40ac5f82657 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -4,16 +4,13 @@ #include #include #include +#include #include #include #include #include #include -#include -#include -#include -#include #include #if USE_SSL @@ -32,30 +29,6 @@ namespace ErrorCodes } -namespace -{ - String base64Decode(const String & encoded) - { - String decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; - } - - String base64Encode(const String & decoded) - { - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); - } -} - - bool authenticateUserByHTTP( const HTTPServerRequest & request, const HTMLForm & params, From c42b5eddd4eaf2e8a185a7b768aab9bd557547ab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 21:10:39 +0200 Subject: [PATCH 30/41] Fix error messages in authenticateUserByHTTP() when multiple authentication methods are used simultaneously. --- src/Server/HTTP/authenticateUserByHTTP.cpp | 69 ++++++++++++---------- 1 file changed, 38 insertions(+), 31 deletions(-) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 40ac5f82657..8929c912601 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -29,6 +29,17 @@ namespace ErrorCodes } +namespace +{ + /// Throws an exception that multiple authorization schemes are used simultaneously. + [[noreturn]] void throwMultipleAuthenticationMethods(std::string_view method1, std::string_view method2) + { + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed to use {} and {} simultaneously", method1, method2); + } +} + + bool authenticateUserByHTTP( const HTTPServerRequest & request, const HTMLForm & params, @@ -46,11 +57,11 @@ bool authenticateUserByHTTP( std::string user = request.get("X-ClickHouse-User", ""); std::string password = request.get("X-ClickHouse-Key", ""); std::string quota_key = request.get("X-ClickHouse-Quota", ""); + bool has_auth_headers = !user.empty() || !password.empty(); /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name /// extracted from the SSL certificate used for this connection instead of checking password. bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); - bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; /// User name and password can be passed using HTTP Basic auth or query parameters /// (both methods are insecure). @@ -60,45 +71,41 @@ bool authenticateUserByHTTP( std::string spnego_challenge; SSLCertificateSubjects certificate_subjects; - if (has_auth_headers) + if (has_ssl_certificate_auth) + { +#if USE_SSL + /// It is prohibited to mix different authorization schemes. + if (!password.empty()) + throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password"); + if (has_http_credentials) + throwMultipleAuthenticationMethods("SSL certificate authentication", "Authorization HTTP header"); + if (has_credentials_in_query_params) + throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via parameters"); + + if (request.havePeerCertificate()) + certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); + + if (certificate_subjects.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); +#endif + } + else if (has_auth_headers) { /// It is prohibited to mix different authorization schemes. if (has_http_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header"); if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); - - if (has_ssl_certificate_auth) - { -#if USE_SSL - if (!password.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); - - if (request.havePeerCertificate()) - certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); - - if (certificate_subjects.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); -#endif - } + throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "authentication via parameters"); } else if (has_http_credentials) { /// It is prohibited to mix different authorization schemes. if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); + throwMultipleAuthenticationMethods("Authorization HTTP header", "authentication via parameters"); std::string scheme; std::string auth_info; From eb7c2e43412b4ba7274d993e2a8fc658beeab9b2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 21:42:54 +0200 Subject: [PATCH 31/41] Remove unnecessary "const" from return type. --- base/poco/Net/include/Poco/Net/NameValueCollection.h | 2 +- base/poco/Net/src/NameValueCollection.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/NameValueCollection.h b/base/poco/Net/include/Poco/Net/NameValueCollection.h index b66819840b1..2337535bd11 100644 --- a/base/poco/Net/include/Poco/Net/NameValueCollection.h +++ b/base/poco/Net/include/Poco/Net/NameValueCollection.h @@ -79,7 +79,7 @@ namespace Net /// Returns the value of the first name-value pair with the given name. /// If no value with the given name has been found, the defaultValue is returned. - const std::vector getAll(const std::string & name) const; + std::vector getAll(const std::string & name) const; /// Returns all values of all name-value pairs with the given name. /// /// Returns an empty vector if there are no name-value pairs with the given name. diff --git a/base/poco/Net/src/NameValueCollection.cpp b/base/poco/Net/src/NameValueCollection.cpp index 0cb31924579..e35d66d3bde 100644 --- a/base/poco/Net/src/NameValueCollection.cpp +++ b/base/poco/Net/src/NameValueCollection.cpp @@ -102,7 +102,7 @@ const std::string& NameValueCollection::get(const std::string& name, const std:: return defaultValue; } -const std::vector NameValueCollection::getAll(const std::string& name) const +std::vector NameValueCollection::getAll(const std::string& name) const { std::vector values; for (ConstIterator it = _map.find(name); it != _map.end(); it++) From fdd7353c92a4fc9bcf517633e049fcb301fdf6c0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 5 Jul 2024 23:47:11 +0200 Subject: [PATCH 32/41] update initCap --- .../functions/string-functions.md | 216 ++++++++++++++---- 1 file changed, 167 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c068b0e9d17..1e801cb1887 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -12,9 +12,7 @@ Functions for [searching](string-search-functions.md) in strings and for [replac ## empty -Checks whether the input string is empty. - -A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. +Checks whether the input string is empty. A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. The function is also available for [arrays](array-functions.md#function-empty) and [UUIDs](uuid-functions.md#empty). @@ -48,9 +46,7 @@ Result: ## notEmpty -Checks whether the input string is non-empty. - -A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. +Checks whether the input string is non-empty. A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. The function is also available for [arrays](array-functions.md#function-notempty) and [UUIDs](uuid-functions.md#notempty). @@ -96,7 +92,7 @@ length(s) **Parameters** -- `s`: An input string or array. [String](../data-types/string)/[Array](../data-types/array). +- `s` — An input string or array. [String](../data-types/string)/[Array](../data-types/array). **Returned value** @@ -149,7 +145,7 @@ lengthUTF8(s) **Parameters** -- `s`: String containing valid UTF-8 encoded text. [String](../data-types/string). +- `s` — String containing valid UTF-8 encoded text. [String](../data-types/string). **Returned value** @@ -183,8 +179,8 @@ left(s, offset) **Parameters** -- `s`: The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -230,8 +226,8 @@ leftUTF8(s, offset) **Parameters** -- `s`: The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -347,8 +343,8 @@ right(s, offset) **Parameters** -- `s`: The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -394,8 +390,8 @@ rightUTF8(s, offset) **Parameters** -- `s`: The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -547,7 +543,7 @@ Alias: `ucase` **Parameters** -- `input`: A string type [String](../data-types/string.md). +- `input` — A string type [String](../data-types/string.md). **Returned value** @@ -571,17 +567,47 @@ SELECT upper('clickhouse'); Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). +:::note +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: -If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +**Syntax** + +```sql +lowerUTF8(input) +``` + +**Parameters** + +- `input` — A string type [String](../data-types/string.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +Query: + +``` sql +SELECT lowerUTF8('MÜNCHEN') as Lowerutf8; +``` + +Result: + +``` response +┌─Lowerutf8─┐ +│ münchen │ +└───────────┘ +``` ## upperUTF8 Converts a string to uppercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). - -If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +:::note +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: **Syntax** @@ -591,7 +617,7 @@ upperUTF8(input) **Parameters** -- `input`: A string type [String](../data-types/string.md). +- `input` — A string type [String](../data-types/string.md). **Returned value** @@ -605,6 +631,8 @@ Query: SELECT upperUTF8('München') as Upperutf8; ``` +Result: + ``` response ┌─Upperutf8─┐ │ MÜNCHEN │ @@ -615,6 +643,34 @@ SELECT upperUTF8('München') as Upperutf8; Returns 1, if the set of bytes constitutes valid UTF-8-encoded text, otherwise 0. +**Syntax** + +``` sql +isValidUTF8(input) +``` + +**Parameters** + +- `input` — A string type [String](../data-types/string.md). + +**Returned value** + +- Returns `1`, if the set of bytes constitutes valid UTF-8-encoded text, otherwise `0`. + +Query: + +``` sql +SELECT isValidUTF8('\xc3\xb1') AS valid, isValidUTF8('\xc3\x28') AS invalid; +``` + +Result: + +``` response +┌─valid─┬─invalid─┐ +│ 1 │ 0 │ +└───────┴─────────┘ +``` + ## toValidUTF8 Replaces invalid UTF-8 characters by the `�` (U+FFFD) character. All running in a row invalid characters are collapsed into the one replacement character. @@ -884,7 +940,7 @@ Returns the substring of a string `s` which starts at the specified byte index ` substring(s, offset[, length]) ``` -Alias: +Aliases: - `substr` - `mid` - `byteSlice` @@ -927,9 +983,9 @@ substringUTF8(s, offset[, length]) **Arguments** -- `s`: The string to calculate a substring from. [String](../data-types/string.md), [FixedString](../data-types/fixedstring.md) or [Enum](../data-types/enum.md) -- `offset`: The starting position of the substring in `s` . [(U)Int*](../data-types/int-uint.md). -- `length`: The maximum length of the substring. [(U)Int*](../data-types/int-uint.md). Optional. +- `s` — The string to calculate a substring from. [String](../data-types/string.md), [FixedString](../data-types/fixedstring.md) or [Enum](../data-types/enum.md) +- `offset` — The starting position of the substring in `s` . [(U)Int*](../data-types/int-uint.md). +- `length` — The maximum length of the substring. [(U)Int*](../data-types/int-uint.md). Optional. **Returned value** @@ -965,9 +1021,9 @@ Alias: `SUBSTRING_INDEX` **Arguments** -- s: The string to extract substring from. [String](../data-types/string.md). -- delim: The character to split. [String](../data-types/string.md). -- count: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) +- s — The string to extract substring from. [String](../data-types/string.md). +- delim — The character to split. [String](../data-types/string.md). +- count — The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) **Example** @@ -996,9 +1052,9 @@ substringIndexUTF8(s, delim, count) **Arguments** -- `s`: The string to extract substring from. [String](../data-types/string.md). -- `delim`: The character to split. [String](../data-types/string.md). -- `count`: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) +- `s` — The string to extract substring from. [String](../data-types/string.md). +- `delim` — The character to split. [String](../data-types/string.md). +- `count` — The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) **Returned value** @@ -1278,7 +1334,7 @@ tryBase64Decode(encoded) **Arguments** -- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. +- `encoded` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. **Returned value** @@ -1310,7 +1366,7 @@ tryBase64URLDecode(encodedUrl) **Parameters** -- `encodedURL`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. +- `encodedURL` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. **Returned value** @@ -1936,7 +1992,7 @@ soundex(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -1969,7 +2025,7 @@ punycodeEncode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2002,7 +2058,7 @@ punycodeEncode(val) **Arguments** -- `val` - Punycode-encoded string. [String](../data-types/string.md) +- `val` — Punycode-encoded string. [String](../data-types/string.md) **Returned value** @@ -2028,7 +2084,7 @@ Like `punycodeDecode` but returns an empty string if no valid Punycode-encoded s ## idnaEncode -Returns the the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +Returns the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. The input string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown. Note: No percent decoding or trimming of tabs, spaces or control characters is performed. @@ -2040,7 +2096,7 @@ idnaEncode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2066,7 +2122,7 @@ Like `idnaEncode` but returns an empty string in case of an error instead of thr ## idnaDecode -Returns the the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +Returns the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. In case of an error (e.g. because the input is invalid), the input string is returned. Note that repeated application of `idnaEncode()` and `idnaDecode()` does not necessarily return the original string due to case normalization. @@ -2078,7 +2134,7 @@ idnaDecode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2122,7 +2178,7 @@ Result: └───────────────────────────────────────────┘ ``` -Alias: mismatches +Alias: `mismatches` ## stringJaccardIndex @@ -2176,7 +2232,7 @@ Result: └─────────────────────────────────────┘ ``` -Alias: levenshteinDistance +Alias: `levenshteinDistance` ## editDistanceUTF8 @@ -2202,7 +2258,7 @@ Result: └─────────────────────────────────────┘ ``` -Alias: levenshteinDistanceUTF8 +Alias: `levenshteinDistanceUTF8` ## damerauLevenshteinDistance @@ -2280,13 +2336,75 @@ Result: Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. +**Syntax** + +```sql +initcap(val) +``` + +**Arguments** + +- `val` — Input value. [String](../data-types/string.md). + +**Returned value** + +- `val` with the first letter of each word converted to upper case. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT initcap('building for fast'); +``` + +Result: + +```text +┌─initcap('building for fast')─┐ +│ Building For Fast │ +└──────────────────────────────┘ +``` + ## initcapUTF8 -Like [initcap](#initcap), assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. - -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). +Like [initcap](#initcap), сonvert the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. +If this assumption is violated, no exception is thrown and the result is undefined. +:::note +This function does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: + +**Syntax** + +```sql +initcapUTF8(val) +``` + +**Arguments** + +- `val` — Input value. [String](../data-types/string.md). + +**Returned value** + +- `val` with the first letter of each word converted to upper case. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT initcapUTF8('не тормозит'); +``` + +Result: + +```text +┌─initcapUTF8('не тормозит')─┐ +│ Не Тормозит │ +└────────────────────────────┘ +``` ## firstLine @@ -2300,7 +2418,7 @@ firstLine(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** From 03d6f37607b44a1c9bbbaae70a30d6077b913a15 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 11:17:21 +0200 Subject: [PATCH 33/41] add note on known issues --- .../functions/string-functions.md | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d451f763e18..6734630cd10 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -568,7 +568,7 @@ SELECT upper('clickhouse'); Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point (such as `ẞ` and `ß`), the result may be incorrect for this code point. ::: **Syntax** @@ -606,7 +606,7 @@ Result: Converts a string to uppercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point (such as `ẞ` and `ß`), the result may be incorrect for this code point. ::: **Syntax** @@ -2336,6 +2336,24 @@ Result: Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. +:::note +Because `initCap` converts only the first letter of each word to upper case you may observe unexpected behaviour for words containing apostrophes or capital letters. For example: + +```sql +SELECT initCap('mother''s daughter'), initCap('joe McAdam'); +``` + +will return + +```response +┌─initCap('mother\'s daughter')─┬─initCap('joe McAdam')─┐ +│ Mother'S Daughter │ Joe Mcadam │ +└───────────────────────────────┴───────────────────────┘ +``` + +This is a known behaviour, with no plans currently to fix it. +::: + **Syntax** ```sql From 7f2bfa221e552957d45cc19b4137bf1d4638ee24 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 19:37:00 +0200 Subject: [PATCH 34/41] Update formatting of maxMap and minMap and add missing aliases --- .../aggregate-functions/reference/maxmap.md | 34 +++++++++++++++---- .../aggregate-functions/reference/minmap.md | 34 +++++++++++++++---- 2 files changed, 56 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md index c9c6913249c..73075c0823d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md @@ -5,23 +5,45 @@ sidebar_position: 165 # maxMap -Syntax: `maxMap(key, value)` or `maxMap(Tuple(key, value))` - Calculates the maximum from `value` array according to the keys specified in the `key` array. -Passing a tuple of keys and value arrays is identical to passing two arrays of keys and values. +**Syntax** -The number of elements in `key` and `value` must be the same for each row that is totaled. +```sql +maxMap(key, value) +``` +or +```sql +maxMap(Tuple(key, value)) +``` -Returns a tuple of two arrays: keys and values calculated for the corresponding keys. +Alias: `maxMappedArrays` -Example: +:::note +- Passing a tuple of keys and value arrays is identical to passing two arrays of keys and values. +- The number of elements in `key` and `value` must be the same for each row that is totaled. +::: + +**Parameters** + +- `key` — Array of keys. [Array](../../data-types/array.md). +- `value` — Array of values. [Array](../../data-types/array.md). + +**Returned value** + +- Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. [Tuple](../../data-types/tuple.md)([Array](../../data-types/array.md), [Array](../../data-types/array.md)). + +**Example** + +Query: ``` sql SELECT maxMap(a, b) FROM values('a Array(Char), b Array(Int64)', (['x', 'y'], [2, 2]), (['y', 'z'], [3, 1])) ``` +Result: + ``` text ┌─maxMap(a, b)───────────┐ │ [['x','y','z'],[2,3,1]]│ diff --git a/docs/en/sql-reference/aggregate-functions/reference/minmap.md b/docs/en/sql-reference/aggregate-functions/reference/minmap.md index b1fbb9e49f3..c0f340b3f3f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/minmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/minmap.md @@ -5,23 +5,45 @@ sidebar_position: 169 # minMap -Syntax: `minMap(key, value)` or `minMap(Tuple(key, value))` - Calculates the minimum from `value` array according to the keys specified in the `key` array. -Passing a tuple of keys and value ​​arrays is identical to passing two arrays of keys and values. +**Syntax** -The number of elements in `key` and `value` must be the same for each row that is totaled. +```sql +`minMap(key, value)` +``` +or +```sql +minMap(Tuple(key, value)) +``` -Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. +Alias: `minMappedArrays` -Example: +:::note +- Passing a tuple of keys and value arrays is identical to passing an array of keys and an array of values. +- The number of elements in `key` and `value` must be the same for each row that is totaled. +::: + +**Parameters** + +- `key` — Array of keys. [Array](../../data-types/array.md). +- `value` — Array of values. [Array](../../data-types/array.md). + +**Returned value** + +- Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. [Tuple](../../data-types/tuple.md)([Array](../../data-types/array.md), [Array](../../data-types/array.md)). + +**Example** + +Query: ``` sql SELECT minMap(a, b) FROM values('a Array(Int32), b Array(Int64)', ([1, 2], [2, 2]), ([2, 3], [1, 1])) ``` +Result: + ``` text ┌─minMap(a, b)──────┐ │ ([1,2,3],[2,1,1]) │ From 3eb2de3d856a3e67eafde36fa34388242e1cbc2c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 19:41:58 +0200 Subject: [PATCH 35/41] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..eb19b3d2871 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1996,6 +1996,7 @@ maxMap maxintersections maxintersectionsposition maxmap +minMappedArrays maxmind mdadm meanZTest @@ -2013,6 +2014,7 @@ metrica metroHash mfedotov minMap +minMappedArrays minSampleSizeContinuous minSampleSizeConversion mindsdb From 82e1d82cb354de6ece3d525c4c71b7d08bfcf6d9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jul 2024 09:07:18 +0000 Subject: [PATCH 36/41] Cosmetics --- .../functions/string-replace-functions.md | 2 +- src/Functions/ReplaceRegexpImpl.h | 79 ++++++++++--------- 2 files changed, 43 insertions(+), 38 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 7aeb1f5b2a7..8793ebdd1a3 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -34,7 +34,7 @@ Alias: `replace`. Replaces the first occurrence of the substring matching the regular expression `pattern` (in [re2 syntax](https://github.com/google/re2/wiki/Syntax)) in `haystack` by the `replacement` string. -`replacement` can containing substitutions `\0-\9`. +`replacement` can contain substitutions `\0-\9`. Substitutions `\1-\9` correspond to the 1st to 9th capturing group (submatch), substitution `\0` corresponds to the entire match. To use a verbatim `\` character in the `pattern` or `replacement` strings, escape it using `\`. diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 24a40c45c6e..c5095e3039f 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -48,42 +48,57 @@ struct ReplaceRegexpImpl static constexpr int max_captures = 10; - static Instructions createInstructions(std::string_view replacement, int num_captures) + /// The replacement string references must not contain non-existing capturing groups. + static void checkSubstitutions(std::string_view replacement, int num_captures) { - Instructions instructions; - - String literals; for (size_t i = 0; i < replacement.size(); ++i) { if (replacement[i] == '\\' && i + 1 < replacement.size()) { - if (isNumericASCII(replacement[i + 1])) /// Substitution + if (isNumericASCII(replacement[i + 1])) /// substitution + { + int substitution_num = replacement[i + 1] - '0'; + if (substitution_num >= num_captures) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Substitution '\\{}' in replacement argument is invalid, regexp has only {} capturing groups", substitution_num, num_captures - 1); + } + } + } + } + + static Instructions createInstructions(std::string_view replacement, int num_captures) + { + checkSubstitutions(replacement, num_captures); + + Instructions instructions; + + String literals; + literals.reserve(replacement.size()); + + for (size_t i = 0; i < replacement.size(); ++i) + { + if (replacement[i] == '\\' && i + 1 < replacement.size()) + { + if (isNumericASCII(replacement[i + 1])) /// substitution { if (!literals.empty()) { instructions.emplace_back(literals); literals = ""; } - instructions.emplace_back(replacement[i + 1] - '0'); + int substitution_num = replacement[i + 1] - '0'; + instructions.emplace_back(substitution_num); } else - literals += replacement[i + 1]; /// Escaping + literals += replacement[i + 1]; /// escaping ++i; } else - literals += replacement[i]; /// Plain character + literals += replacement[i]; /// plain character } if (!literals.empty()) instructions.emplace_back(literals); - for (const auto & instr : instructions) - if (instr.substitution_num >= num_captures) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Id {} in replacement string is an invalid substitution, regexp has only {} capturing groups", - instr.substitution_num, num_captures - 1); - return instructions; } @@ -124,7 +139,7 @@ struct ReplaceRegexpImpl { std::string_view replacement; if (instr.substitution_num >= 0) - replacement = std::string_view(matches[instr.substitution_num].data(), matches[instr.substitution_num].size()); + replacement = {matches[instr.substitution_num].data(), matches[instr.substitution_num].size()}; else replacement = instr.literal; res_data.resize(res_data.size() + replacement.size()); @@ -179,19 +194,15 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(replacement, num_captures); - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -221,10 +232,8 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -242,6 +251,7 @@ struct ReplaceRegexpImpl re2::RE2 searcher(needle, regexp_options); if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); + int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); Instructions instructions = createInstructions(replacement, num_captures); @@ -270,17 +280,14 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -290,8 +297,9 @@ struct ReplaceRegexpImpl size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0; const char * repl_data = reinterpret_cast(replacement_data.data() + repl_from); const size_t repl_length = static_cast(replacement_offsets[i] - repl_from - 1); + std::string_view replacement(repl_data, repl_length); - Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures); + Instructions instructions = createInstructions(replacement, num_captures); processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions); res_offsets[i] = res_offset; @@ -317,10 +325,8 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -338,12 +344,14 @@ struct ReplaceRegexpImpl size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0; const char * repl_data = reinterpret_cast(replacement_data.data() + repl_from); const size_t repl_length = static_cast(replacement_offsets[i] - repl_from - 1); + std::string_view replacement(repl_data, repl_length); re2::RE2 searcher(needle, regexp_options); if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); + int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures); + Instructions instructions = createInstructions(replacement, num_captures); processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions); res_offsets[i] = res_offset; @@ -367,16 +375,13 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(replacement, num_captures); for (size_t i = 0; i < haystack_size; ++i) From 948565f5544d29b37a7368b8f61c5bfbd66905c1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jul 2024 10:43:48 +0000 Subject: [PATCH 37/41] Speed up replaceRegexp(All|One) if the pattern is trivial --- src/Functions/ReplaceRegexpImpl.h | 37 ++++++++++++++++++- tests/performance/replaceRegexp_fallback.xml | 12 ++++++ ...4_replace_regexp_string_fallback.reference | 1 + .../02864_replace_regexp_string_fallback.sql | 11 ++++++ 4 files changed, 60 insertions(+), 1 deletion(-) create mode 100644 tests/performance/replaceRegexp_fallback.xml create mode 100644 tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference create mode 100644 tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index c5095e3039f..f5fb08f71d2 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -1,9 +1,12 @@ #pragma once -#include #include +#include #include +#include +#include #include +#include namespace DB { @@ -102,6 +105,21 @@ struct ReplaceRegexpImpl return instructions; } + static bool canFallbackToStringReplacement(const String & needle, const String & replacement, const re2::RE2 & searcher, int num_captures) + { + if (searcher.NumberOfCapturingGroups()) + return false; + + checkSubstitutions(replacement, num_captures); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; + std::vector alternatives; + OptimizedRegularExpression::analyze(needle, required_substring, is_trivial, required_substring_is_prefix, alternatives); + return is_trivial && required_substring_is_prefix && required_substring == needle; + } + static void processString( const char * haystack_data, size_t haystack_length, @@ -201,6 +219,23 @@ struct ReplaceRegexpImpl throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); + + /// Try to use non-regexp string replacement. This shortcut is implemented only for const-needles + const-replacement as + /// pattern analysis incurs some cost too. + if (canFallbackToStringReplacement(needle, replacement, searcher, num_captures)) + { + auto convertTrait = [](ReplaceRegexpTraits::Replace first_or_all) + { + switch (first_or_all) + { + case ReplaceRegexpTraits::Replace::First: return ReplaceStringTraits::Replace::First; + case ReplaceRegexpTraits::Replace::All: return ReplaceStringTraits::Replace::All; + } + }; + ReplaceStringImpl::vectorConstantConstant(haystack_data, haystack_offsets, needle, replacement, res_data, res_offsets); + return; + } + Instructions instructions = createInstructions(replacement, num_captures); for (size_t i = 0; i < haystack_size; ++i) diff --git a/tests/performance/replaceRegexp_fallback.xml b/tests/performance/replaceRegexp_fallback.xml new file mode 100644 index 00000000000..926e66c702f --- /dev/null +++ b/tests/performance/replaceRegexp_fallback.xml @@ -0,0 +1,12 @@ +> +> + + > + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null + + > + > + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null + diff --git a/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference new file mode 100644 index 00000000000..dd52d49eea3 --- /dev/null +++ b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference @@ -0,0 +1 @@ +Hello l x Hexlo Hexxo diff --git a/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql new file mode 100644 index 00000000000..917c11fe8dd --- /dev/null +++ b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql @@ -0,0 +1,11 @@ +-- Tests functions replaceRegexpAll and replaceRegexpOne with trivial patterns. These trigger internally a fallback to simple string replacement. + +-- _materialize_ because the shortcut is only implemented for non-const haystack + const needle + const replacement strings + +SELECT 'Hello' AS haystack, 'l' AS needle, 'x' AS replacement, replaceRegexpOne(materialize(haystack), needle, replacement), replaceRegexpAll(materialize(haystack), needle, replacement); + +-- negative tests + +-- Even if the fallback is used, invalid substitutions must throw an exception. +SELECT 'Hello' AS haystack, 'l' AS needle, '\\1' AS replacement, replaceRegexpOne(materialize(haystack), needle, replacement); -- { serverError BAD_ARGUMENTS } +SELECT 'Hello' AS haystack, 'l' AS needle, '\\1' AS replacement, replaceRegexpAll(materialize(haystack), needle, replacement); -- { serverError BAD_ARGUMENTS } From 8f604c892ade829ebff91961faf550ec45f6c1a9 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Jul 2024 12:58:29 +0200 Subject: [PATCH 38/41] Fix typo --- docs/en/sql-reference/functions/string-functions.md | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 6734630cd10..8259d780577 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -2386,7 +2386,7 @@ Result: ## initcapUTF8 -Like [initcap](#initcap), сonvert the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. +Like [initcap](#initcap), `initcapUTF8` converts the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fa2bfef935a..4d615340357 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 2758 +personal_ws-1.1 en 2942 AArch ACLs ALTERs @@ -1656,9 +1656,9 @@ fsync func fuzzBits fuzzJSON +fuzzQuery fuzzer fuzzers -fuzzQuery gRPC gccMurmurHash gcem From 5ac5a4b2fe91ee5efa37a602f109bf6534e12de6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 22:15:32 +0200 Subject: [PATCH 39/41] Add check that user name is not empty to authenticateUserByHTTP(). --- src/Server/HTTP/authenticateUserByHTTP.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 8929c912601..ac43bfd64c0 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -37,6 +37,13 @@ namespace throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: it is not allowed to use {} and {} simultaneously", method1, method2); } + + /// Checks that a specified user name is not empty, and throws an exception if it's empty. + void checkUserNameNotEmpty(const String & user_name, std::string_view method) + { + if (user_name.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Got an empty user name from {}", method); + } } @@ -74,6 +81,9 @@ bool authenticateUserByHTTP( if (has_ssl_certificate_auth) { #if USE_SSL + /// For SSL certificate authentication we extract the user name from the "X-ClickHouse-User" HTTP header. + checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers"); + /// It is prohibited to mix different authorization schemes. if (!password.empty()) throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password"); @@ -95,6 +105,8 @@ bool authenticateUserByHTTP( } else if (has_auth_headers) { + checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers"); + /// It is prohibited to mix different authorization schemes. if (has_http_credentials) throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header"); @@ -116,6 +128,7 @@ bool authenticateUserByHTTP( Poco::Net::HTTPBasicCredentials credentials(auth_info); user = credentials.getUsername(); password = credentials.getPassword(); + checkUserNameNotEmpty(user, "Authorization HTTP header"); } else if (Poco::icompare(scheme, "Negotiate") == 0) { @@ -134,10 +147,12 @@ bool authenticateUserByHTTP( /// If the user name is not set we assume it's the 'default' user. user = params.get("user", "default"); password = params.get("password", ""); + checkUserNameNotEmpty(user, "authentication via parameters"); } if (!certificate_subjects.empty()) { + chassert(!user.empty()); if (!current_credentials) current_credentials = std::make_unique(user, std::move(certificate_subjects)); @@ -183,6 +198,7 @@ bool authenticateUserByHTTP( if (!basic_credentials) throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); + chassert(!user.empty()); basic_credentials->setUserName(user); basic_credentials->setPassword(password); } From 486ac7f287f0486fedb60f0339b81b2688af6760 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 8 Jul 2024 09:30:25 -0300 Subject: [PATCH 40/41] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d74a63b972..ed0b29aa851 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1170,6 +1170,10 @@ Data in the VALUES clause of INSERT queries is processed by a separate stream pa Default value: 262144 (= 256 KiB). +:::note +`max_query_size` cannot be set within an SQL query (e.g., `SELECT now() SETTINGS max_query_size=10000`) because ClickHouse needs to allocate a buffer to parse the query, and this buffer size is determined by the `max_query_size` setting, which must be configured before the query is executed. +::: + ## max_parser_depth {#max_parser_depth} Limits maximum recursion depth in the recursive descent parser. Allows controlling the stack size. From 708c81f6e2a81b530a47f543795bb8ff6ec663ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 9 Jul 2024 01:20:31 +0200 Subject: [PATCH 41/41] Update 00504_mergetree_arrays_rw.sql --- tests/queries/0_stateless/00504_mergetree_arrays_rw.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql index 7c939d060ea..14929045356 100644 --- a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql +++ b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql @@ -1,5 +1,8 @@ set allow_deprecated_syntax_for_merge_tree=1; +set max_threads = 1; +set max_insert_threads = 1; + drop table if exists test_ins_arr; create table test_ins_arr (date Date, val Array(UInt64)) engine = MergeTree(date, (date), 8192); insert into test_ins_arr select toDate('2017-10-02'), [number, 42] from system.numbers limit 10000;