From c2f85c6fd062dde095ee34178450dc94c245e691 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:43:02 +0800 Subject: [PATCH 01/23] support map type as first argument type --- .../functions/tuple-map-functions.md | 2 +- src/Functions/map.cpp | 131 +++++++++++------- .../0_stateless/01651_map_functions.reference | 3 + .../0_stateless/01651_map_functions.sql | 9 +- 4 files changed, 89 insertions(+), 56 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 24b356eca87..ae23387f6e5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 66cd10a3f0b..5319390fb70 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -1,14 +1,17 @@ -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include @@ -178,22 +181,28 @@ public: getName(), arguments.size()); - /// The first argument should always be Array. - /// Because key type can not be nested type of Map, which is Tuple - DataTypePtr key_type; - if (const auto * keys_type = checkAndGetDataType(arguments[0].get())) - key_type = keys_type->getNestedType(); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); + auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + { + DataTypePtr nested; + if (const auto * array_type = checkAndGetDataType(type.get())) + nested = array_type->getNestedType(); + else if (const auto * map_type = checkAndGetDataType(type.get())) + nested = std::make_shared(map_type->getKeyValueTypes()); + else + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument types of function {} must be Array or Map, but {} is given", + getName(), + type->getName()); - DataTypePtr value_type; - if (const auto * value_array_type = checkAndGetDataType(arguments[1].get())) - value_type = value_array_type->getNestedType(); - else if (const auto * value_map_type = checkAndGetDataType(arguments[1].get())) - value_type = std::make_shared(value_map_type->getKeyValueTypes()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName()); + return nested; + }; + auto key_type = get_nested_type(arguments[0]); + auto value_type = get_nested_type(arguments[1]); + + /// Remove Nullable from key_type if needed for map key must not be Nullable + key_type = removeNullableOrLowCardinalityNullable(key_type); DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -201,44 +210,62 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - bool is_keys_const = isColumnConst(*arguments[0].column); - ColumnPtr holder_keys; - const ColumnArray * col_keys; - if (is_keys_const) + auto get_array_column = [this](const ColumnPtr & column) -> std::pair { - holder_keys = arguments[0].column->convertToFullColumnIfConst(); - col_keys = checkAndGetColumn(holder_keys.get()); - } - else + bool is_const = isColumnConst(*column); + ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; + + const ColumnArray * col_res = nullptr; + if (const auto * col_array = checkAndGetColumn(holder.get())) + col_res = col_array; + else if (const auto * col_map = checkAndGetColumn(holder.get())) + col_res = &col_map->getNestedColumn(); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument columns of function {} must be Array or Map, but {} is given", + getName(), + holder->getName()); + + return {col_res, holder}; + }; + + auto [col_keys, key_holder] = get_array_column(arguments[0].column); + + /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + ColumnPtr data_keys = col_keys->getDataPtr(); + if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - col_keys = checkAndGetColumn(arguments[0].column.get()); + std::cout << "data keys is nullable" << std::endl; + const NullMap * null_map = nullptr; + if (const auto * nullable = checkAndGetColumn(data_keys.get())) + { + null_map = &nullable->getNullMapData(); + data_keys = nullable->getNestedColumnPtr(); + } + else if (const auto * low_cardinality = checkAndGetColumn(data_keys.get())) + { + if (const auto * nullable_dict = checkAndGetColumn(low_cardinality->getDictionaryPtr().get())) + { + null_map = &nullable_dict->getNullMapData(); + data_keys = ColumnLowCardinality::create(nullable_dict->getNestedColumnPtr(), low_cardinality->getIndexesPtr()); + } + } + + if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); } - if (!col_keys) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName()); - - bool is_values_const = isColumnConst(*arguments[1].column); - ColumnPtr holder_values; - if (is_values_const) - holder_values = arguments[1].column->convertToFullColumnIfConst(); - else - holder_values = arguments[1].column; - - const ColumnArray * col_values; - if (const auto * col_values_array = checkAndGetColumn(holder_values.get())) - col_values = col_values_array; - else if (const auto * col_values_map = checkAndGetColumn(holder_values.get())) - col_values = &col_values_map->getNestedColumn(); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName()); - + auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName()); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); - const auto & data_keys = col_keys->getDataPtr(); const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets); + std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; + auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); + std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 471da5586b7..9114aa419b1 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -52,3 +52,6 @@ {1:4,2:5} {1:4,2:5} {1:4,2:5} +{1:3,2:4} +{1:3,2:4} +{1:3,2:4} {(1,3):'a',(2,4):'b'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index cf2460fce2c..4604ddd6db1 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -67,12 +67,15 @@ select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} +select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); -select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], [4, 5]); select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, 5])) from numbers(2); + +select mapFromArrays([1,2], [3,4]); +select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); +select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); From e181ccd0173c46d31867097532f64df0be3944da Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 17:53:51 +0800 Subject: [PATCH 02/23] update doc --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index ae23387f6e5..db66188b1f5 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -62,7 +62,7 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). - `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** From a32c702caa142d15bc3e5bc51ca90240d5d010a9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 18:23:47 +0800 Subject: [PATCH 03/23] fix style --- src/Functions/map.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 5319390fb70..a8e5f7ad90e 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -236,7 +236,6 @@ public: ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { - std::cout << "data keys is nullable" << std::endl; const NullMap * null_map = nullptr; if (const auto * nullable = checkAndGetColumn(data_keys.get())) { @@ -263,9 +262,7 @@ public: const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - std::cout << "before create array:" << "offsets:" << offsets->getName() << std::endl; auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets); - std::cout << "after create array:" << "offsets:" << offsets->getName() << std::endl; return ColumnMap::create(nested_column); } }; From 60acbe7720ce31a55e3f846536ad2e1572f541ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 01:40:05 +0200 Subject: [PATCH 04/23] Fix stack overflow in too deep JSON --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/IO/ReadHelpers.cpp | 25 +++++++++++++------ .../0_stateless/03213_deep_json.reference | 0 tests/queries/0_stateless/03213_deep_json.sql | 5 ++++ 7 files changed, 27 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03213_deep_json.reference create mode 100644 tests/queries/0_stateless/03213_deep_json.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86e04b2ab4e..6c8e09f38f9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1126,6 +1126,7 @@ class IColumn; M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ M(Bool, input_format_json_case_insensitive_column_matching, false, "Ignore case when matching JSON keys with CH columns", 0) \ + M(UInt64, input_format_json_max_depth, 1000, "Maximum depth of a field in JSON. This is not a strict limit, it does not have to be applied precisely.", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9faf77e9087..b340091798c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,7 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.8", {{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."}}}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 501343d2ca9..695f15e3fdb 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -123,6 +123,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.import_nested_json = settings.input_format_import_nested_json; format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; + format_settings.json.max_depth = settings.input_format_json_max_depth; format_settings.json.array_of_rows = settings.output_format_json_array_of_rows; format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; format_settings.json.write_named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 72d1515b7e1..32676970011 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -205,6 +205,7 @@ struct FormatSettings struct JSON { + size_t max_depth = 1000; bool array_of_rows = false; bool quote_64bit_integers = true; bool quote_64bit_floats = false; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..bc9c7bb110c 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -1494,10 +1495,20 @@ template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const template -ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings) +ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings, size_t current_depth) { static constexpr bool throw_exception = std::is_same_v; + if (unlikely(current_depth > settings.max_depth)) + { + if constexpr (throw_exception) + throw Exception(ErrorCodes::INCORRECT_DATA, "JSON is too deep for key '{}'", name_of_field.toString()); + return ReturnType(false); + } + + if (unlikely(current_depth > 0 && current_depth % 1024 == 0)) + checkStackSize(); + if (buf.eof()) { if constexpr (throw_exception) @@ -1560,8 +1571,8 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo while (true) { if constexpr (throw_exception) - skipJSONFieldImpl(buf, name_of_field, settings); - else if (!skipJSONFieldImpl(buf, name_of_field, settings)) + skipJSONFieldImpl(buf, name_of_field, settings, current_depth + 1); + else if (!skipJSONFieldImpl(buf, name_of_field, settings, current_depth + 1)) return ReturnType(false); skipWhitespaceIfAny(buf); @@ -1619,8 +1630,8 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo skipWhitespaceIfAny(buf); if constexpr (throw_exception) - skipJSONFieldImpl(buf, name_of_field, settings); - else if (!skipJSONFieldImpl(buf, name_of_field, settings)) + skipJSONFieldImpl(buf, name_of_field, settings, current_depth + 1); + else if (!skipJSONFieldImpl(buf, name_of_field, settings, current_depth + 1)) return ReturnType(false); skipWhitespaceIfAny(buf); @@ -1659,12 +1670,12 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo void skipJSONField(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings) { - skipJSONFieldImpl(buf, name_of_field, settings); + skipJSONFieldImpl(buf, name_of_field, settings, 0); } bool trySkipJSONField(ReadBuffer & buf, StringRef name_of_field, const FormatSettings::JSON & settings) { - return skipJSONFieldImpl(buf, name_of_field, settings); + return skipJSONFieldImpl(buf, name_of_field, settings, 0); } diff --git a/tests/queries/0_stateless/03213_deep_json.reference b/tests/queries/0_stateless/03213_deep_json.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03213_deep_json.sql b/tests/queries/0_stateless/03213_deep_json.sql new file mode 100644 index 00000000000..4f79d99eb26 --- /dev/null +++ b/tests/queries/0_stateless/03213_deep_json.sql @@ -0,0 +1,5 @@ +-- The default limit works. +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +-- Even if we relax the limit, it is also safe. +SET input_format_json_max_depth = 100000; +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } From 86f8fa5efbfd0c7bfe3f00ff37b85b2efe17cfe5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 01:40:16 +0200 Subject: [PATCH 05/23] Fix stack overflow in too deep JSON --- src/IO/ReadHelpers.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index bc9c7bb110c..9559462e62b 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -40,6 +39,7 @@ namespace ErrorCodes extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int TOO_DEEP_RECURSION; } template @@ -1502,7 +1502,7 @@ ReturnType skipJSONFieldImpl(ReadBuffer & buf, StringRef name_of_field, const Fo if (unlikely(current_depth > settings.max_depth)) { if constexpr (throw_exception) - throw Exception(ErrorCodes::INCORRECT_DATA, "JSON is too deep for key '{}'", name_of_field.toString()); + throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "JSON is too deep for key '{}'", name_of_field.toString()); return ReturnType(false); } From 161e794618a86f39ad4f5e6435306e384c85d01d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 29 Jul 2024 12:57:36 +0200 Subject: [PATCH 06/23] Add no-distributed-cache tag --- docker/test/stateless/run.sh | 4 ++++ tests/clickhouse-test | 10 ++++++++++ tests/queries/0_stateless/02995_index_9.sh | 2 +- 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index f9f96c76d59..da17b82d91b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -212,6 +212,10 @@ function run_tests() ADDITIONAL_OPTIONS+=('--shared-catalog') fi + if [[ "$USE_DISTRIBUTED_CACHE" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--distributed-cache') + fi + if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') # Too many tests fail for DatabaseReplicated in parallel. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a29c786e998..f810a8714bd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -681,6 +681,7 @@ class FailureReason(enum.Enum): BUILD = "not running for current build" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" SHARED_MERGE_TREE = "no-shared-merge-tree" + DISTRIBUTED_CACHE = "distributed-cache" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1191,6 +1192,9 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB + elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: + return FailureReason.DISTRIBUTED_CACHE + elif ( tags and ("atomic-database" in tags) @@ -3204,6 +3208,12 @@ def parse_args(): default=False, help="Run tests over s3 storage", ) + parser.add_argument( + "--distributed-cache", + action="store_true", + default=False, + help="Run tests with enabled distributed cache", + ) parser.add_argument( "--azure-blob-storage", action="store_true", diff --git a/tests/queries/0_stateless/02995_index_9.sh b/tests/queries/0_stateless/02995_index_9.sh index 76160c62aaa..46426f22185 100755 --- a/tests/queries/0_stateless/02995_index_9.sh +++ b/tests/queries/0_stateless/02995_index_9.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8affc58c4d77eb490b6b969020d8d797bfb49cb9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 29 Jul 2024 14:10:19 +0200 Subject: [PATCH 07/23] Tag more tests --- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 2 +- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 2 +- tests/queries/0_stateless/02714_local_object_storage.sql | 2 ++ tests/queries/0_stateless/02770_async_buffer_ignore.sh | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 1cf52c0288b..3d467515b5c 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan +-- Tags: no-tsan, no-distributed-cache -- Tag no-tsan: Too long for TSan set enable_filesystem_cache=0; diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 9b96ce3e586..5f09dc423b2 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage +-- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher diff --git a/tests/queries/0_stateless/02714_local_object_storage.sql b/tests/queries/0_stateless/02714_local_object_storage.sql index 914eac1ddba..bfe3162df97 100644 --- a/tests/queries/0_stateless/02714_local_object_storage.sql +++ b/tests/queries/0_stateless/02714_local_object_storage.sql @@ -1,3 +1,5 @@ +-- Tags: no-distributed-cache + SET min_bytes_to_use_direct_io='1Gi'; -- It does not work (fixme) SET local_filesystem_read_method='pread'; -- ui_uring local_fs_method does not work here (fixme) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index 6eaf655324d..37f002767d6 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-random-settings +# Tags: no-fasttest, no-random-settings, no-distributed-cache CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 1333fe24645f36f2dd5f45a084d379de834e71a8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 29 Jul 2024 16:09:51 +0200 Subject: [PATCH 08/23] Update 01603_read_with_backoff_bug.sql --- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 3d467515b5c..8b97f3514b3 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan, no-distributed-cache +-- Tags: long, no-tsan, no-distributed-cache -- Tag no-tsan: Too long for TSan set enable_filesystem_cache=0; From af63af8ff2d5f86aa22300b1e19c37b36307dac9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 29 Jul 2024 16:44:19 +0200 Subject: [PATCH 09/23] Update a few more tests --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 1 + tests/queries/0_stateless/02995_index_2.sh | 2 +- tests/queries/0_stateless/02995_index_4.sh | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index 114f60cc393..be7cfa78492 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02995_index_2.sh b/tests/queries/0_stateless/02995_index_2.sh index e7451c7ee4b..f2f466e8ff7 100755 --- a/tests/queries/0_stateless/02995_index_2.sh +++ b/tests/queries/0_stateless/02995_index_2.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02995_index_4.sh b/tests/queries/0_stateless/02995_index_4.sh index 1a0458728f9..59be06ae17a 100755 --- a/tests/queries/0_stateless/02995_index_4.sh +++ b/tests/queries/0_stateless/02995_index_4.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 7a53a14940ae1be299305548f0d024de7f279fe3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 09:29:16 +0200 Subject: [PATCH 10/23] Update 03213_deep_json.sql --- tests/queries/0_stateless/03213_deep_json.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03213_deep_json.sql b/tests/queries/0_stateless/03213_deep_json.sql index 4f79d99eb26..2a9476381ff 100644 --- a/tests/queries/0_stateless/03213_deep_json.sql +++ b/tests/queries/0_stateless/03213_deep_json.sql @@ -1,5 +1,5 @@ -- The default limit works. -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } -- Even if we relax the limit, it is also safe. SET input_format_json_max_depth = 100000; -SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION } +SELECT * FROM format("JSONCompactEachRow", 'x UInt32, y UInt32', REPEAT('[1,1,', 100000)) SETTINGS input_format_json_compact_allow_variable_number_of_columns = 1; -- { serverError TOO_DEEP_RECURSION, INCORRECT_DATA } From fb466287dac16801518547f34b42edbc16a57fae Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:13:47 +0200 Subject: [PATCH 11/23] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index be7cfa78492..c29b604d23d 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-distributed-cache +# Tags: long, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 6372fdee6d344bd87d58ce89fa069b55750c9aba Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 13:46:05 +0200 Subject: [PATCH 12/23] Update tests --- tests/queries/0_stateless/02995_index_3.sh | 2 +- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02995_index_3.sh b/tests/queries/0_stateless/02995_index_3.sh index 506429e2696..219ae81154f 100755 --- a/tests/queries/0_stateless/02995_index_3.sh +++ b/tests/queries/0_stateless/02995_index_3.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage +# Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-sanitize-coverage, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..a927e4f1e1f 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-distributed-cache DROP TABLE IF EXISTS build; DROP TABLE IF EXISTS skewed_probe; From d69f6cccde7633214eba48c08d7647e4ea9a40da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Jul 2024 15:08:26 +0200 Subject: [PATCH 13/23] Fix --- src/Common/ThreadPoolTaskTracker.cpp | 5 ++++- src/IO/WriteBufferFromS3.cpp | 9 ++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/ThreadPoolTaskTracker.cpp b/src/Common/ThreadPoolTaskTracker.cpp index 61d34801f7a..1697a13f780 100644 --- a/src/Common/ThreadPoolTaskTracker.cpp +++ b/src/Common/ThreadPoolTaskTracker.cpp @@ -19,6 +19,10 @@ TaskTracker::TaskTracker(ThreadPoolCallbackRunnerUnsafe scheduler_, size_t TaskTracker::~TaskTracker() { + /// Tasks should be waited outside of dtor. + /// Important for WriteBufferFromS3/AzureBlobStorage, where TaskTracker is currently used. + chassert(finished_futures.empty() && futures.empty()); + safeWaitAll(); } @@ -170,4 +174,3 @@ bool TaskTracker::isAsync() const } } - diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 3682e49b018..e702b4d35ad 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -277,12 +277,10 @@ WriteBufferFromS3::~WriteBufferFromS3() "The file might not be written to S3. " "{}.", getVerboseLogDetails()); - return; } - - /// That destructor could be call with finalized=false in case of exceptions - if (!finalized && !canceled) + else if (!finalized) { + /// That destructor could be call with finalized=false in case of exceptions LOG_INFO( log, "WriteBufferFromS3 is not finalized in destructor. " @@ -291,9 +289,10 @@ WriteBufferFromS3::~WriteBufferFromS3() getVerboseLogDetails()); } + /// Wait for all tasks, because they contain reference to this write buffer. task_tracker->safeWaitAll(); - if (!multipart_upload_id.empty() && !multipart_upload_finished) + if (!canceled && !multipart_upload_id.empty() && !multipart_upload_finished) { LOG_WARNING(log, "WriteBufferFromS3 was neither finished nor aborted, try to abort upload in destructor. {}.", getVerboseLogDetails()); tryToAbortMultipartUpload(); From c81d3322b18b0eb4b45b91ac019a8c4f42d7518d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 13:39:30 +0200 Subject: [PATCH 14/23] Update 02150_index_hypothesis_race_long.sh --- tests/queries/0_stateless/02150_index_hypothesis_race_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index c29b604d23d..5c432350768 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-distributed-cache +# Tags: long, no-random-settings, no-distributed-cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 406ac2279ecbfc24913548dfcf459c55dd450723 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 14:48:33 +0200 Subject: [PATCH 15/23] Analyzer: Do not traverse unresolved subtrees --- src/Planner/findParallelReplicasQuery.cpp | 12 +++++------- src/Planner/findQueryForParallelReplicas.h | 2 +- ...5_analyzer_replace_with_dummy_tables.reference | 0 .../03215_analyzer_replace_with_dummy_tables.sql | 15 +++++++++++++++ 4 files changed, 21 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference create mode 100644 tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index c89a70be541..1140f30ad9c 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -113,13 +113,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre return res; } -class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitor +class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitorWithContext { public: - using Base = InDepthQueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * table_node = node->as(); auto * table_function_node = node->as(); @@ -134,21 +134,19 @@ public: ColumnsDescription(storage_snapshot->getColumns(get_column_options)), storage_snapshot); - auto dummy_table_node = std::make_shared(std::move(storage_dummy), context); + auto dummy_table_node = std::make_shared(std::move(storage_dummy), getContext()); dummy_table_node->setAlias(node->getAlias()); replacement_map.emplace(node.get(), std::move(dummy_table_node)); } } - ContextPtr context; std::unordered_map replacement_map; }; QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) { - ReplaceTableNodeToDummyVisitor visitor; - visitor.context = context; + ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); return query->cloneAndReplace(visitor.replacement_map); diff --git a/src/Planner/findQueryForParallelReplicas.h b/src/Planner/findQueryForParallelReplicas.h index f5dc69dfa0e..cdce4ad0b47 100644 --- a/src/Planner/findQueryForParallelReplicas.h +++ b/src/Planner/findQueryForParallelReplicas.h @@ -13,7 +13,7 @@ using QueryTreeNodePtr = std::shared_ptr; struct SelectQueryOptions; -/// Find a qury which can be executed with parallel replicas up to WithMergableStage. +/// Find a query which can be executed with parallel replicas up to WithMergableStage. /// Returned query will always contain some (>1) subqueries, possibly with joins. const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options); diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql new file mode 100644 index 00000000000..12d2bd627a7 --- /dev/null +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -0,0 +1,15 @@ +create table t (number UInt64) engine MergeTree order by number; + +SELECT 1 +FROM +( + SELECT number IN ( + SELECT number + FROM view( + SELECT number + FROM numbers(1) + ) + ) + FROM t +) +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; From d6de2be4395e1bcc62ab32ad1d5b02e9db080303 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:08:18 +0200 Subject: [PATCH 16/23] Fix build --- src/Planner/findParallelReplicasQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 1140f30ad9c..39edb1e6516 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -144,7 +144,7 @@ public: std::unordered_map replacement_map; }; -QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context) +QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const ContextPtr & context) { ReplaceTableNodeToDummyVisitor visitor(context); visitor.visit(query); From e31569a065d4c81cdea671727c39983d7f3a84e5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 Jul 2024 16:32:37 +0200 Subject: [PATCH 17/23] Expect an unknown cluster --- .../0_stateless/03215_analyzer_replace_with_dummy_tables.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql index 12d2bd627a7..6d084c2ac50 100644 --- a/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql +++ b/tests/queries/0_stateless/03215_analyzer_replace_with_dummy_tables.sql @@ -12,4 +12,4 @@ FROM ) FROM t ) -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, allow_experimental_analyzer = 1; -- { serverError CLUSTER_DOESNT_EXIST } From 2b79da36c0701bb9ca392fddd9129a7e0e04ef3f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 Jul 2024 19:26:45 +0200 Subject: [PATCH 18/23] Update 01605_adaptive_granularity_block_borders.sql --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 5f09dc423b2..f9b8bb1c1c6 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,4 @@ --- Tags: no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache +-- Tags: long, no-random-merge-tree-settings, no-tsan, no-debug, no-object-storage, no-distributed-cache -- no-tsan: too slow -- no-object-storage: for remote tables we use thread pool even when reading with one stream, so memory consumption is higher From 5564489cca1c14e95e7c543e03c508849abaf079 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 15:31:54 +0800 Subject: [PATCH 19/23] change as request --- .../functions/tuple-map-functions.md | 18 +++++++++++--- src/Functions/map.cpp | 24 ++++++++++--------- .../0_stateless/01651_map_functions.reference | 4 ++++ .../0_stateless/01651_map_functions.sql | 7 +++++- 4 files changed, 38 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index db66188b1f5..d670ed42a2a 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -43,7 +43,7 @@ Result: ## mapFromArrays -Creates a map from an array of keys and an array of values. +Creates a map from an array or map of keys and an array or map of values. The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`. For example, instead of writing @@ -62,8 +62,8 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Array or map of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type, or [Map](../data-types/map.md). -- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). +- `keys` — Array or map of keys to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). If `keys` is an array, we accept `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as its type as long as it doesn't contain NULL value. +- `values` - Array or map of values to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** @@ -99,6 +99,18 @@ Result: └───────────────────────────────────────────────────────┘ ``` +```sql +SELECT mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3]) +``` + +Result: + +``` +┌─mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])─┐ +│ {('a',1):1,('b',2):2,('c',3):3} │ +└───────────────────────────────────────────────────────┘ +``` + ## extractKeyValuePairs Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md). diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index a8e5f7ad90e..738c61164a3 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } namespace @@ -157,7 +158,7 @@ private: bool use_variant_as_common_type = false; }; -/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays +/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays or maps class FunctionMapFromArrays : public IFunction { public: @@ -181,13 +182,13 @@ public: getName(), arguments.size()); - auto get_nested_type = [this](const DataTypePtr & type) -> DataTypePtr + auto get_nested_type = [&](const DataTypePtr & type) { DataTypePtr nested; - if (const auto * array_type = checkAndGetDataType(type.get())) - nested = array_type->getNestedType(); - else if (const auto * map_type = checkAndGetDataType(type.get())) - nested = std::make_shared(map_type->getKeyValueTypes()); + if (const auto * type_as_array = checkAndGetDataType(type.get())) + nested = type_as_array->getNestedType(); + else if (const auto * type_as_map = checkAndGetDataType(type.get())) + nested = std::make_shared(type_as_map->getKeyValueTypes()); else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -201,8 +202,9 @@ public: auto key_type = get_nested_type(arguments[0]); auto value_type = get_nested_type(arguments[1]); - /// Remove Nullable from key_type if needed for map key must not be Nullable + /// We accept Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as key types as long as the actual array doesn't contain NULL value(this is checked in executeImpl). key_type = removeNullableOrLowCardinalityNullable(key_type); + DataTypes key_value_types{key_type, value_type}; return std::make_shared(key_value_types); } @@ -210,7 +212,7 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - auto get_array_column = [this](const ColumnPtr & column) -> std::pair + auto get_array_column = [&](const ColumnPtr & column) -> std::pair { bool is_const = isColumnConst(*column); ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column; @@ -231,8 +233,9 @@ public: }; auto [col_keys, key_holder] = get_array_column(arguments[0].column); + auto [col_values, values_holder] = get_array_column(arguments[1].column); - /// Check if nested column of first argument contains NULL value in case its nested type is Nullable(T) type. + /// Nullable(T) or LowCardinality(Nullable(T)) are okay as nested key types but actual NULL values are not okay. ColumnPtr data_keys = col_keys->getDataPtr(); if (isColumnNullableOrLowCardinalityNullable(*data_keys)) { @@ -253,10 +256,9 @@ public: if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size())) throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "The nested column of first argument in function {} must not contain NULLs", getName()); + ErrorCodes::BAD_ARGUMENTS, "The nested column of first argument in function {} must not contain NULLs", getName()); } - auto [col_values, values_holder] = get_array_column(arguments[1].column); if (!col_keys->hasEqualOffsets(*col_values)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of function {} must have equal sizes", getName()); diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 9114aa419b1..e336a02318d 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -55,3 +55,7 @@ {1:3,2:4} {1:3,2:4} {1:3,2:4} {(1,3):'a',(2,4):'b'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} +{(1,'a'):'c',(2,'b'):'d'} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 4604ddd6db1..dc93a38b265 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -68,7 +68,7 @@ select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAY select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE} select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE} select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError ILLEGAL_COLUMN } +select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError BAD_ARGUMENTS } select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5)); select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2); @@ -79,3 +79,8 @@ select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, select mapFromArrays([1,2], [3,4]); select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]); select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']); + +select mapFromArrays(map(1, 'a', 2, 'b'), array('c', 'd')); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), array('c', 'd')); +select mapFromArrays(map(1, 'a', 2, 'b'), materialize(array('c', 'd'))); +select mapFromArrays(materialize(map(1, 'a', 2, 'b')), materialize(array('c', 'd'))); From 681441e170202bc3963fb3fa1d7b7785192dbd2e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 1 Aug 2024 16:01:39 +0800 Subject: [PATCH 20/23] fix style --- src/Functions/map.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 738c61164a3..6e389f39dec 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -192,7 +192,7 @@ public: else throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument types of function {} must be Array or Map, but {} is given", + "Arguments of function {} must be Array or Map, but {} is given", getName(), type->getName()); @@ -275,10 +275,7 @@ public: static constexpr auto name = "mapUpdate"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -287,9 +284,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 2", - getName(), arguments.size()); + getName(), + arguments.size()); const auto * left = checkAndGetDataType(arguments[0].type.get()); const auto * right = checkAndGetDataType(arguments[1].type.get()); @@ -405,7 +404,6 @@ public: return ColumnMap::create(nested_column); } }; - } REGISTER_FUNCTION(Map) From 8562a6106c286882f26383086e52e399106893be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:45:18 +0200 Subject: [PATCH 21/23] Better safety thresholds in `arrayWithConstant` --- src/Functions/array/arrayWithConstant.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayWithConstant.cpp b/src/Functions/array/arrayWithConstant.cpp index 48262870553..4cbc6404b9b 100644 --- a/src/Functions/array/arrayWithConstant.cpp +++ b/src/Functions/array/arrayWithConstant.cpp @@ -1,9 +1,9 @@ #include -#include #include #include #include #include +#include namespace DB @@ -15,7 +15,8 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } -/// Reasonable threshold. +/// Reasonable thresholds. +static constexpr Int64 max_array_size_in_columns_bytes = 1000000000; static constexpr size_t max_arrays_size_in_columns = 1000000000; @@ -63,12 +64,19 @@ public: auto array_size = col_num->getInt(i); if (unlikely(array_size < 0)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size cannot be negative: while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} cannot be negative: while executing function {}", array_size, getName()); + + Int64 estimated_size = 0; + if (unlikely(common::mulOverflow(array_size, col_value->byteSize(), estimated_size))) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); + + if (unlikely(estimated_size > max_array_size_in_columns_bytes)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size {} with element size {} bytes is too large: while executing function {}", array_size, col_value->byteSize(), getName()); offset += array_size; if (unlikely(offset > max_arrays_size_in_columns)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size while executing function {}", getName()); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size {} (will generate at least {} elements) while executing function {}", array_size, offset, getName()); offsets.push_back(offset); } From 7a066a6505108b14bd49da8766c1bc473a978b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 23:48:51 +0200 Subject: [PATCH 22/23] Add a test --- .../0_stateless/03216_arrayWithConstant_limits.reference | 1 + tests/queries/0_stateless/03216_arrayWithConstant_limits.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.reference create mode 100644 tests/queries/0_stateless/03216_arrayWithConstant_limits.sql diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference new file mode 100644 index 00000000000..825319e1c5b --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.reference @@ -0,0 +1 @@ +10000000 diff --git a/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql new file mode 100644 index 00000000000..c46524c50e6 --- /dev/null +++ b/tests/queries/0_stateless/03216_arrayWithConstant_limits.sql @@ -0,0 +1,3 @@ +SELECT arrayWithConstant(96142475, ['qMUF']); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT arrayWithConstant(100000000, materialize([[[[[[[[[['Hello, world!']]]]]]]]]])); -- { serverError TOO_LARGE_ARRAY_SIZE } +SELECT length(arrayWithConstant(10000000, materialize([[[[[[[[[['Hello world']]]]]]]]]]))); From 185b6a54da8f24a97f130bdebe7bb1ec2bd266c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 00:34:58 +0200 Subject: [PATCH 23/23] Merge with master --- src/Core/SettingsChangesHistory.cpp | 261 +--------------------------- 1 file changed, 1 insertion(+), 260 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3f07bfdb933..5b94391bade 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.8", {{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."}}}, - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_reader_time_zone_name", "GMT", "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -335,6 +75,7 @@ static std::initializer_list