Merge branch 'master' into suppress_hung_check_with_ubsan

This commit is contained in:
Alexander Tokmakov 2023-03-10 23:09:26 +03:00 committed by GitHub
commit 30b75c98f5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 168 additions and 48 deletions

View File

@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \
esac
ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release"
ARG VERSION="23.2.3.17"
ARG VERSION="23.2.4.12"
ARG PACKAGES="clickhouse-keeper"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="23.2.3.17"
ARG VERSION="23.2.4.12"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="23.2.3.17"
ARG VERSION="23.2.4.12"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -0,0 +1,20 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.2.4.12-stable (8fe866cb035) FIXME as compared to v23.2.3.17-stable (dec18bf7281)
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Backported in [#47277](https://github.com/ClickHouse/ClickHouse/issues/47277): Fix IPv4/IPv6 serialization/deserialization in binary formats that was broken in https://github.com/ClickHouse/ClickHouse/pull/43221. Closes [#46522](https://github.com/ClickHouse/ClickHouse/issues/46522). [#46616](https://github.com/ClickHouse/ClickHouse/pull/46616) ([Kruglov Pavel](https://github.com/Avogar)).
* Backported in [#47212](https://github.com/ClickHouse/ClickHouse/issues/47212): `INSERT` queries through native TCP protocol and HTTP protocol were not canceled correctly in some cases. It could lead to a partially applied query if a client canceled the query, or if a client died or, in rare cases, on network errors. As a result, it could lead to not working deduplication. Fixes [#27667](https://github.com/ClickHouse/ClickHouse/issues/27667) and [#45377](https://github.com/ClickHouse/ClickHouse/issues/45377). [#46681](https://github.com/ClickHouse/ClickHouse/pull/46681) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Backported in [#47363](https://github.com/ClickHouse/ClickHouse/issues/47363): Fix possible deadlock on distributed query cancellation. [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -1492,7 +1492,7 @@ Default value: `65505`.
The name of table that will be used in the output INSERT statement.
Default value: `'table''`.
Default value: `table`.
### output_format_sql_insert_include_column_names {#output_format_sql_insert_include_column_names}
@ -1532,4 +1532,12 @@ Disabled by default.
The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit.
Default value: `1GiB`
Default value: `1GiB`.
## Native format settings {#native-format-settings}
### input_format_native_allow_types_conversion {#input_format_native_allow_types_conversion}
Allow types conversion in Native input format between columns from input data and requested columns.
Enabled by default.

View File

@ -112,23 +112,21 @@ See also [data_type_default_nullable](../../../operations/settings/settings.md#d
## Default Values {#default_values}
The column description can specify an expression for a default value, in one of the following ways: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`.
The column description can specify a default value expression in the form of `DEFAULT expr`, `MATERIALIZED expr`, or `ALIAS expr`. Example: `URLDomain String DEFAULT domain(URL)`.
Example: `URLDomain String DEFAULT domain(URL)`.
The expression `expr` is optional. If it is omitted, the column type must be specified explicitly and the default value will be `0` for numeric columns, `''` (the empty string) for string columns, `[]` (the empty array) for array columns, `1970-01-01` for date columns, or `NULL` for nullable columns.
If an expression for the default value is not defined, the default values will be set to zeros for numbers, empty strings for strings, empty arrays for arrays, and `1970-01-01` for dates or zero unix timestamp for DateTime, NULL for Nullable.
The column type of a default value column can be omitted in which case it is infered from `expr`'s type. For example the type of column `EventDate DEFAULT toDate(EventTime)` will be date.
If the default expression is defined, the column type is optional. If there isnt an explicitly defined type, the default expression type is used. Example: `EventDate DEFAULT toDate(EventTime)` the Date type will be used for the EventDate column.
If both a data type and a default value expression are specified, an implicit type casting function inserted which converts the expression to the specified type. Example: `Hits UInt32 DEFAULT 0` is internally represented as `Hits UInt32 DEFAULT toUInt32(0)`.
If the data type and default expression are defined explicitly, this expression will be cast to the specified type using type casting functions. Example: `Hits UInt32 DEFAULT 0` means the same thing as `Hits UInt32 DEFAULT toUInt32(0)`.
Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions do not contain loops. For INSERT, it checks that expressions are resolvable that all columns they can be calculated from have been passed.
A default value expression `expr` may reference arbitrary table columns and constants. ClickHouse checks that changes of the table structure do not introduce loops in the expression calculation. For INSERT, it checks that expressions are resolvable that all columns they can be calculated from have been passed.
### DEFAULT
`DEFAULT expr`
Normal default value. If the INSERT query does not specify the corresponding column, it will be filled in by computing the corresponding expression.
Normal default value. If the value of such a column is not specified in an INSERT query, it is computed from `expr`.
Example:
@ -154,9 +152,9 @@ SELECT * FROM test;
`MATERIALIZED expr`
Materialized expression. Such a column cant be specified for INSERT, because it is always calculated.
For an INSERT without a list of columns, these columns are not considered.
In addition, this column is not substituted when using an asterisk in a SELECT query. This is to preserve the invariant that the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns.
Materialized expression. Values of such columns are always calculated, they cannot be specified in INSERT queries.
Also, default value columns of this type are not included in the result of `SELECT *`. This is to preserve the invariant that the result of a `SELECT *` can always be inserted back into the table using `INSERT`. This behavior can be disabled with setting `asterisk_include_materialized_columns`.
Example:
@ -192,8 +190,9 @@ SELECT * FROM test SETTINGS asterisk_include_materialized_columns=1;
`EPHEMERAL [expr]`
Ephemeral column. Such a column isn't stored in the table and cannot be SELECTed, but can be referenced in the defaults of CREATE statement. If `expr` is omitted type for column is required.
INSERT without list of columns will skip such column, so SELECT/INSERT invariant is preserved - the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns.
Ephemeral column. Columns of this type are not stored in the table and it is not possible to SELECT from them. The only purpose of ephemeral columns is to build default value expressions of other columns from them.
An insert without explicitly specified columns will skip columns of this type. This is to preserve the invariant that the result of a `SELECT *` can always be inserted back into the table using `INSERT`.
Example:
@ -205,7 +204,7 @@ CREATE OR REPLACE TABLE test
hexed FixedString(4) DEFAULT unhex(unhexed)
)
ENGINE = MergeTree
ORDER BY id
ORDER BY id;
INSERT INTO test (id, unhexed) Values (1, '5a90b714');
@ -227,9 +226,9 @@ hex(hexed): 5A90B714
`ALIAS expr`
Synonym. Such a column isnt stored in the table at all.
Its values cant be inserted in a table, and it is not substituted when using an asterisk in a SELECT query.
It can be used in SELECTs if the alias is expanded during query parsing.
Calculated columns (synonym). Column of this type are not stored in the table and it is not possible to INSERT values into them.
When SELECT queries explicitly reference columns of this type, the value is computed at query time from `expr`. By default, `SELECT *` excludes ALIAS columns. This behavior can be disabled with setting `asteriks_include_alias_columns`.
When using the ALTER query to add new columns, old data for these columns is not written. Instead, when reading old data that does not have values for the new columns, expressions are computed on the fly by default. However, if running the expressions requires different columns that are not indicated in the query, these columns will additionally be read, but only for the blocks of data that need it.

View File

@ -827,6 +827,8 @@ class IColumn;
M(UInt64, input_format_csv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in CSV format", 0) \
M(UInt64, input_format_tsv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in TSV format", 0) \
\
M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \
\
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
\

View File

@ -82,6 +82,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{
{"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"}}},

View File

@ -86,6 +86,6 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type);
ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column);
/// Convert column of type from_type to type to_type by converting nested LowCardinality columns.
ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
}

View File

@ -113,7 +113,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
return column;
}
ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type)
ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type)
{
if (!column)
return column;
@ -128,7 +128,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr &
if (const auto * column_const = typeid_cast<const ColumnConst *>(column.get()))
{
const auto & nested = column_const->getDataColumnPtr();
auto nested_no_lc = recursiveTypeConversion(nested, from_type, to_type);
auto nested_no_lc = recursiveLowCardinalityTypeConversion(nested, from_type, to_type);
if (nested.get() == nested_no_lc.get())
return column;
@ -164,7 +164,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr &
const auto & nested_to = to_array_type->getNestedType();
return ColumnArray::create(
recursiveTypeConversion(column_array->getDataPtr(), nested_from, nested_to),
recursiveLowCardinalityTypeConversion(column_array->getDataPtr(), nested_from, nested_to),
column_array->getOffsetsPtr());
}
}
@ -187,7 +187,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr &
for (size_t i = 0; i < columns.size(); ++i)
{
auto & element = columns[i];
auto element_no_lc = recursiveTypeConversion(element, from_elements.at(i), to_elements.at(i));
auto element_no_lc = recursiveLowCardinalityTypeConversion(element, from_elements.at(i), to_elements.at(i));
if (element.get() != element_no_lc.get())
{
element = element_no_lc;

View File

@ -194,6 +194,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string;
format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.max_binary_string_size = settings.format_binary_max_string_size;
format_settings.native.allow_types_conversion = settings.input_format_native_allow_types_conversion;
format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth;
format_settings.client_protocol_version = context->getClientProtocolVersion();

View File

@ -364,6 +364,11 @@ struct FormatSettings
bool output_string_as_string;
bool skip_fields_with_unsupported_types_in_schema_inference;
} bson;
struct
{
bool allow_types_conversion = true;
} native;
};
}

View File

@ -15,6 +15,8 @@
#include <DataTypes/Serializations/SerializationInfo.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Interpreters/castColumn.h>
namespace DB
{
@ -39,12 +41,14 @@ NativeReader::NativeReader(
UInt64 server_revision_,
bool skip_unknown_columns_,
bool null_as_default_,
bool allow_types_conversion_,
BlockMissingValues * block_missing_values_)
: istr(istr_)
, header(header_)
, server_revision(server_revision_)
, skip_unknown_columns(skip_unknown_columns_)
, null_as_default(null_as_default_)
, allow_types_conversion(allow_types_conversion_)
, block_missing_values(block_missing_values_)
{
}
@ -204,11 +208,31 @@ Block NativeReader::read()
if (null_as_default)
insertNullAsDefaultIfNeeded(column, header_column, header.getPositionByName(column.name), block_missing_values);
/// Support insert from old clients without low cardinality type.
if (!header_column.type->equals(*column.type))
{
column.column = recursiveTypeConversion(column.column, column.type, header.safeGetByPosition(i).type);
column.type = header.safeGetByPosition(i).type;
if (allow_types_conversion)
{
try
{
column.column = castColumn(column, header_column.type);
}
catch (Exception & e)
{
e.addMessage(fmt::format(
"while converting column \"{}\" from type {} to type {}",
column.name,
column.type->getName(),
header_column.type->getName()));
throw;
}
}
else
{
/// Support insert from old clients without low cardinality type.
column.column = recursiveLowCardinalityTypeConversion(column.column, column.type, header_column.type);
}
column.type = header_column.type;
}
}
else

View File

@ -30,6 +30,7 @@ public:
UInt64 server_revision_,
bool skip_unknown_columns_ = false,
bool null_as_default_ = false,
bool allow_types_conversion_ = false,
BlockMissingValues * block_missing_values_ = nullptr);
/// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read.
@ -51,6 +52,7 @@ private:
UInt64 server_revision;
bool skip_unknown_columns = false;
bool null_as_default = false;
bool allow_types_conversion = false;
BlockMissingValues * block_missing_values = nullptr;
bool use_index = false;

View File

@ -984,13 +984,16 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting
if (tryReadIntText(tmp_int, buf) && buf.eof())
return std::make_shared<DataTypeInt64>();
/// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof.
buf.position() = buf.buffer().begin();
/// In case of Int64 overflow, try to infer UInt64
UInt64 tmp_uint;
if (tryReadIntText(tmp_uint, buf) && buf.eof())
return std::make_shared<DataTypeUInt64>();
}
/// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof.
/// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof.
buf.position() = buf.buffer().begin();
Float64 tmp;

View File

@ -1089,6 +1089,7 @@ static std::shared_ptr<IJoin> chooseJoinAlgorithm(
if (MergeJoin::isSupported(analyzed_join))
return std::make_shared<JoinSwitcher>(analyzed_join, right_sample_block);
return std::make_shared<HashJoin>(analyzed_join, right_sample_block);
}
throw Exception(ErrorCodes::NOT_IMPLEMENTED,

View File

@ -655,7 +655,7 @@ std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_jo
return std::make_shared<HashJoin>(table_join, right_table_expression_header);
}
if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH))
if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) && !table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section");
/// Direct JOIN with special storages that support key value access. For example JOIN with Dictionary
@ -708,7 +708,11 @@ std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_jo
}
if (table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO))
return std::make_shared<JoinSwitcher>(table_join, right_table_expression_header);
{
if (MergeJoin::isSupported(table_join))
return std::make_shared<JoinSwitcher>(table_join, right_table_expression_header);
return std::make_shared<HashJoin>(table_join, right_table_expression_header);
}
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Can't execute any of specified algorithms for specified strictness/kind and right storage type");

View File

@ -23,6 +23,7 @@ public:
0,
settings.skip_unknown_fields,
settings.null_as_default,
settings.native.allow_types_conversion,
settings.defaults_for_omitted_fields ? &block_missing_values : nullptr))
, header(header_) {}

View File

@ -117,7 +117,7 @@ static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Co
{
const auto & from_type = desc.nested_type;
const auto & to_type = desc.real_type;
columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type);
columns[desc.column_number] = recursiveLowCardinalityTypeConversion(columns[desc.column_number], from_type, to_type);
}
}

View File

@ -450,7 +450,7 @@ static void postprocessChunk(
{
const auto & from_type = desc.nested_type;
const auto & to_type = desc.real_type;
res_columns[desc.column_numbers[0]] = recursiveTypeConversion(column, from_type, to_type);
res_columns[desc.column_numbers[0]] = recursiveLowCardinalityTypeConversion(column, from_type, to_type);
}
else
res_columns[desc.column_numbers[0]] = std::move(column);

View File

@ -13,3 +13,33 @@
2 2
2 3
2 4
0 0
0 1
0 2
0 3
0 4
1 0
1 1
1 2
1 3
1 4
2 0
2 1
2 2
2 3
2 4
0 0
0 1
0 2
0 3
0 4
1 0
1 1
1 2
1 3
1 4
2 0
2 1
2 2
2 3
2 4

View File

@ -1 +1,7 @@
SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2;
SET join_algorithm = 'auto';
SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2;
SET allow_experimental_analyzer = 1;
SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2;

View File

@ -15,7 +15,7 @@ SET join_algorithm = 'partial_merge';
SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 }
-- works for a = b OR a = b because of equivalent disjunct optimization
SET join_algorithm = 'auto';
SET join_algorithm = 'grace_hash';
SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 }
-- works for a = b OR a = b because of equivalent disjunct optimization

View File

@ -1,11 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# Should correctly handle error.
${CLICKHOUSE_LOCAL} --query "SELECT toString(number) AS a, toString(number) AS a FROM numbers(10)" --output-format Native |
${CLICKHOUSE_LOCAL} --query "SELECT * FROM table" --input-format Native --structure 'a LowCardinality(String)' 2>&1 |
grep -c -F Exception

View File

@ -0,0 +1,3 @@
1
42
1

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL -q "select 42::UInt8 as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=0 2>&1 | grep "TYPE_MISMATCH" -c
$CLICKHOUSE_LOCAL -q "select 42::UInt8 as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=1
$CLICKHOUSE_LOCAL -q "select 'Hello' as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=1 2>&1 | grep 'while converting column "x" from type String to type UInt64' -c

View File

@ -0,0 +1,4 @@
c1 Nullable(Float64)
100000000000000000000
c1 Nullable(Float64)
-100000000000000000000

View File

@ -0,0 +1,5 @@
desc format('CSV', '100000000000000000000');
select * from format('CSV', '100000000000000000000');
desc format('CSV', '-100000000000000000000');
select * from format('CSV', '-100000000000000000000');

View File

@ -1,3 +1,4 @@
v23.2.4.12-stable 2023-03-10
v23.2.3.17-stable 2023-03-06
v23.2.2.20-stable 2023-03-01
v23.2.1.2537-stable 2023-02-23

1 v23.2.3.17-stable v23.2.4.12-stable 2023-03-06 2023-03-10
1 v23.2.4.12-stable 2023-03-10
2 v23.2.3.17-stable v23.2.3.17-stable 2023-03-06 2023-03-06
3 v23.2.2.20-stable v23.2.2.20-stable 2023-03-01 2023-03-01
4 v23.2.1.2537-stable v23.2.1.2537-stable 2023-02-23 2023-02-23