mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into tests/issue_26473
This commit is contained in:
commit
3e422d4ec6
@ -63,10 +63,6 @@
|
||||
"name": "clickhouse/integration-tests-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/testflows/runner": {
|
||||
"name": "clickhouse/testflows-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/fasttest": {
|
||||
"name": "clickhouse/fasttest",
|
||||
"dependent": []
|
||||
|
@ -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="22.12.1.1752"
|
||||
ARG VERSION="22.12.2.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -21,7 +21,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="22.12.1.1752"
|
||||
ARG VERSION="22.12.2.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# set non-empty deb_location_url url to create a docker image
|
||||
|
@ -8,6 +8,7 @@ RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
|
||||
|
||||
RUN apt-get update \
|
||||
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
|
||||
adduser \
|
||||
ca-certificates \
|
||||
bash \
|
||||
btrfs-progs \
|
||||
|
29
docs/changelogs/v22.12.2.25-stable.md
Normal file
29
docs/changelogs/v22.12.2.25-stable.md
Normal file
@ -0,0 +1,29 @@
|
||||
---
|
||||
sidebar_position: 1
|
||||
sidebar_label: 2023
|
||||
---
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### ClickHouse release v22.12.2.25-stable (c790cfd4465) FIXME as compared to v22.12.1.1752-stable (688e488e930)
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
* Backported in [#44381](https://github.com/ClickHouse/ClickHouse/issues/44381): In rare cases, we don't rebuild binaries, because another task with a similar prefix succeeded. E.g. `binary_darwin` didn't restart because `binary_darwin_aarch64`. [#44311](https://github.com/ClickHouse/ClickHouse/pull/44311) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Backported in [#44561](https://github.com/ClickHouse/ClickHouse/issues/44561): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
|
||||
|
||||
* Backported in [#44739](https://github.com/ClickHouse/ClickHouse/issues/44739): [#40651](https://github.com/ClickHouse/ClickHouse/issues/40651) [#41404](https://github.com/ClickHouse/ClickHouse/issues/41404). [#42126](https://github.com/ClickHouse/ClickHouse/pull/42126) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
* Backported in [#44764](https://github.com/ClickHouse/ClickHouse/issues/44764): Fix parsing of bad version from compatibility setting. [#44224](https://github.com/ClickHouse/ClickHouse/pull/44224) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Backported in [#44435](https://github.com/ClickHouse/ClickHouse/issues/44435): Fix possible crash in case function `IN` with constant arguments was used as a constant argument together with `LowCardinality`. Fixes [#44221](https://github.com/ClickHouse/ClickHouse/issues/44221). [#44346](https://github.com/ClickHouse/ClickHouse/pull/44346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
|
||||
#### NOT FOR CHANGELOG / INSIGNIFICANT
|
||||
|
||||
* Bump libdivide (to gain some new optimizations) [#44132](https://github.com/ClickHouse/ClickHouse/pull/44132) ([Azat Khuzhin](https://github.com/azat)).
|
||||
* Add some settings under `compatibility` [#44209](https://github.com/ClickHouse/ClickHouse/pull/44209) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Fix deadlock in StorageSystemDatabases [#44272](https://github.com/ClickHouse/ClickHouse/pull/44272) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
* Get rid of global Git object [#44273](https://github.com/ClickHouse/ClickHouse/pull/44273) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Add check for submodules sanity [#44386](https://github.com/ClickHouse/ClickHouse/pull/44386) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Fix crash on delete from materialized view [#44705](https://github.com/ClickHouse/ClickHouse/pull/44705) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
|
@ -154,10 +154,6 @@ Runs [stateful functional tests](tests.md#functional-tests). Treat them in the s
|
||||
Runs [integration tests](tests.md#integration-tests).
|
||||
|
||||
|
||||
## Testflows Check
|
||||
Runs some tests using Testflows test system. See [here](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally) how to run them locally.
|
||||
|
||||
|
||||
## Stress Test
|
||||
Runs stateless functional tests concurrently from several clients to detect
|
||||
concurrency-related errors. If it fails:
|
||||
|
@ -281,10 +281,6 @@ We also track test coverage but only for functional tests and only for clickhous
|
||||
|
||||
There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky.
|
||||
|
||||
## Testflows
|
||||
|
||||
[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse.
|
||||
|
||||
## Test Automation {#test-automation}
|
||||
|
||||
We run tests with [GitHub Actions](https://github.com/features/actions).
|
||||
|
@ -19,7 +19,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
(
|
||||
Path String,
|
||||
Time DateTime,
|
||||
Value <Numeric_type>,
|
||||
Value Float64,
|
||||
Version <Numeric_type>
|
||||
...
|
||||
) ENGINE = GraphiteMergeTree(config_section)
|
||||
@ -37,7 +37,7 @@ A table for the Graphite data should have the following columns for the followin
|
||||
|
||||
- Time of measuring the metric. Data type: `DateTime`.
|
||||
|
||||
- Value of the metric. Data type: any numeric.
|
||||
- Value of the metric. Data type: `Float64`.
|
||||
|
||||
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
|
||||
|
||||
@ -65,7 +65,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
EventDate Date,
|
||||
Path String,
|
||||
Time DateTime,
|
||||
Value <Numeric_type>,
|
||||
Value Float64,
|
||||
Version <Numeric_type>
|
||||
...
|
||||
) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section)
|
||||
|
@ -86,7 +86,18 @@ When creating a table, the following settings are applied:
|
||||
[join_any_take_last_row](/docs/en/operations/settings/settings.md/#settings-join_any_take_last_row)
|
||||
#### join_use_nulls
|
||||
|
||||
[persistent](/docs/en/operations/settings/settings.md/#persistent)
|
||||
#### persistent
|
||||
|
||||
Disables persistency for the Join and [Set](/docs/en/engines/table-engines/special/set.md) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations.
|
||||
|
||||
|
@ -19,5 +19,16 @@ For a rough server restart, the block of data on the disk might be lost or damag
|
||||
|
||||
When creating a table, the following settings are applied:
|
||||
|
||||
- [persistent](../../../operations/settings/settings.md#persistent)
|
||||
#### persistent
|
||||
|
||||
Disables persistency for the Set and [Join](/docs/en/engines/table-engines/special/join.md/#join) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
|
@ -2441,19 +2441,6 @@ Result
|
||||
└──────────────────────────┴───────┴───────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## persistent {#persistent}
|
||||
|
||||
Disables persistency for the [Set](../../engines/table-engines/special/set.md/#set) and [Join](../../engines/table-engines/special/join.md/#join) table engines.
|
||||
|
||||
Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 1 — Enabled.
|
||||
- 0 — Disabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
## allow_nullable_key {#allow-nullable-key}
|
||||
|
||||
Allows using of the [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engines-mergetree) tables.
|
||||
|
@ -806,7 +806,8 @@ Aliases: `dateSub`, `DATE_SUB`.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `unit` — The type of interval to subtract. [String](../../sql-reference/data-types/string.md).
|
||||
- `unit` — The type of interval to subtract. Note: The unit should be unquoted.
|
||||
|
||||
Possible values:
|
||||
|
||||
- `second`
|
||||
|
@ -110,9 +110,6 @@ git push
|
||||
## 集成测试 {#integration-tests}
|
||||
运行[集成测试](./tests.md#integration-tests).
|
||||
|
||||
## Testflows 检查{#testflows-check}
|
||||
使用Testflows测试系统去运行一些测试, 在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们.
|
||||
|
||||
## 压力测试 {#stress-test}
|
||||
从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败:
|
||||
```
|
||||
|
@ -281,10 +281,6 @@ We also track test coverage but only for functional tests and only for clickhous
|
||||
|
||||
There is automated check for flaky tests. It runs all new tests 100 times (for functional tests) or 10 times (for integration tests). If at least single time the test failed, it is considered flaky.
|
||||
|
||||
## Testflows
|
||||
|
||||
[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse.
|
||||
|
||||
## Test Automation {#test-automation}
|
||||
|
||||
We run tests with [GitHub Actions](https://github.com/features/actions).
|
||||
|
@ -12,8 +12,6 @@ priority: "optional"
|
||||
|
||||
conflicts:
|
||||
- clickhouse-server
|
||||
depends:
|
||||
- adduser
|
||||
suggests:
|
||||
- clickhouse-keeper-dbg
|
||||
|
||||
|
@ -12,8 +12,6 @@ priority: "optional"
|
||||
|
||||
conflicts:
|
||||
- clickhouse-keeper
|
||||
depends:
|
||||
- adduser
|
||||
replaces:
|
||||
- clickhouse-server-common
|
||||
- clickhouse-server-base
|
||||
|
@ -138,6 +138,8 @@ public:
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkAndGetDataType<DataTypeIPv6>(arguments[0].get()))
|
||||
{
|
||||
const auto * ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH)
|
||||
@ -145,6 +147,7 @@ public:
|
||||
" of argument 1 of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (!WhichDataType(arguments[1]).isUInt8())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() +
|
||||
@ -162,7 +165,7 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_type_name = arguments[0];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
@ -172,11 +175,17 @@ public:
|
||||
const auto & col_ipv4_zeroed_tail_bytes_type = arguments[2];
|
||||
const auto & col_ipv4_zeroed_tail_bytes = col_ipv4_zeroed_tail_bytes_type.column;
|
||||
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != IPV6_BINARY_LENGTH)
|
||||
const auto * col_in_str = checkAndGetColumn<ColumnFixedString>(column.get());
|
||||
const auto * col_in_ip = checkAndGetColumn<ColumnIPv6>(column.get());
|
||||
|
||||
if (!col_in_str && !col_in_ip)
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (col_in_str && col_in_str->getN() != IPV6_BINARY_LENGTH)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" of column " + col_in_str->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -205,36 +214,44 @@ public:
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1));
|
||||
offsets_res.resize(size);
|
||||
vec_res.resize(input_rows_count * (IPV6_MAX_TEXT_LENGTH + 1));
|
||||
offsets_res.resize(input_rows_count);
|
||||
|
||||
auto * begin = reinterpret_cast<char *>(vec_res.data());
|
||||
auto * pos = begin;
|
||||
|
||||
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += IPV6_BINARY_LENGTH, ++i)
|
||||
if (col_in_str)
|
||||
{
|
||||
const auto & vec_in = col_in_str->getChars();
|
||||
|
||||
for (size_t offset = 0, i = 0; i < input_rows_count; offset += IPV6_BINARY_LENGTH, ++i)
|
||||
{
|
||||
const auto * address = &vec_in[offset];
|
||||
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||
cutAddress(reinterpret_cast<const unsigned char *>(address), pos, zeroed_tail_bytes_count);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & vec_in = col_in_ip->getData();
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
const auto * address = reinterpret_cast<const UInt8 *>(&vec_in[i]);
|
||||
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||
cutAddress(reinterpret_cast<const unsigned char *>(address), pos, zeroed_tail_bytes_count);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
static bool isIPv4Mapped(const UInt8 * address)
|
||||
|
@ -1136,7 +1136,7 @@ private:
|
||||
else if (which.isInt128()) executeBigIntType<Int128, first>(icolumn, vec_to);
|
||||
else if (which.isInt256()) executeBigIntType<Int256, first>(icolumn, vec_to);
|
||||
else if (which.isUUID()) executeBigIntType<UUID, first>(icolumn, vec_to);
|
||||
else if (which.isIPv4()) executeBigIntType<IPv4, first>(icolumn, vec_to);
|
||||
else if (which.isIPv4()) executeIntType<IPv4, first>(icolumn, vec_to);
|
||||
else if (which.isIPv6()) executeBigIntType<IPv6, first>(icolumn, vec_to);
|
||||
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
|
@ -65,10 +65,6 @@
|
||||
"name": "clickhouse/integration-tests-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/testflows/runner": {
|
||||
"name": "clickhouse/testflows-runner",
|
||||
"dependent": []
|
||||
},
|
||||
"docker/test/fasttest": {
|
||||
"name": "clickhouse/fasttest",
|
||||
"dependent": []
|
||||
|
@ -0,0 +1,4 @@
|
||||
::ffff:127.0.0.1
|
||||
::ffff:127.0.0.1
|
||||
::ffff:127.0.0.1
|
||||
::ffff:127.0.0.1
|
10
tests/queries/0_stateless/02184_ipv6_select_parsing.sql
Normal file
10
tests/queries/0_stateless/02184_ipv6_select_parsing.sql
Normal file
@ -0,0 +1,10 @@
|
||||
drop table if exists ips_v6;
|
||||
create table ips_v6(i IPv6) Engine=Memory;
|
||||
|
||||
INSERT INTO ips_v6 SELECT toIPv6('::ffff:127.0.0.1');
|
||||
INSERT INTO ips_v6 values ('::ffff:127.0.0.1');
|
||||
INSERT INTO ips_v6 FORMAT TSV ::ffff:127.0.0.1
|
||||
INSERT INTO ips_v6 SELECT ('::ffff:127.0.0.1');
|
||||
|
||||
SELECT * FROM ips_v6;
|
||||
drop table ips_v6;
|
@ -244,7 +244,7 @@ find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null |
|
||||
find $ROOT_PATH -not -path $ROOT_PATH'/tests/ci*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:"
|
||||
|
||||
# There shouldn't be any docker compose files outside docker directory
|
||||
#find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:"
|
||||
find $ROOT_PATH -name '*compose*.yml' -type f -not -path $ROOT_PATH'/docker' -not -path $ROOT_PATH'/tests/integration*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' 2>/dev/null | grep -vP $EXCLUDE_DIRS | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to the 'docker' or 'tests' directory:"
|
||||
|
||||
# Check that every header file has #pragma once in first line
|
||||
find $ROOT_PATH/{src,programs,utils} -name '*.h' |
|
||||
|
@ -1,3 +1,4 @@
|
||||
v22.12.2.25-stable 2023-01-06
|
||||
v22.12.1.1752-stable 2022-12-15
|
||||
v22.11.2.30-stable 2022-12-02
|
||||
v22.11.1.1360-stable 2022-11-17
|
||||
|
|
Loading…
Reference in New Issue
Block a user