Merge remote-tracking branch 'origin/master' into clickhouse-help

This commit is contained in:
Yatsishin Ilya 2023-03-02 16:16:18 +00:00
commit d4304d2028
596 changed files with 9197 additions and 3059 deletions

View File

@ -209,3 +209,5 @@ CheckOptions:
# Workaround clang-tidy bug: https://github.com/llvm/llvm-project/issues/46097
- key: readability-identifier-naming.TypeTemplateParameterIgnoredRegexp
value: expr-type
- key: cppcoreguidelines-avoid-do-while.IgnoreMacros
value: true

View File

@ -13,9 +13,11 @@ on: # yamllint disable-line rule:truthy
branches:
- master
paths:
- 'CHANGELOG.md'
- 'README.md'
- 'SECURITY.md'
- 'docker/docs/**'
- 'docs/**'
- 'website/**'
- 'utils/check-style/aspell-ignore/**'
jobs:
CheckLabels:

View File

@ -154,7 +154,7 @@ jobs:
- name: Set Up Build Tools
run: |
sudo apt-get update
sudo apt-get install -yq git cmake ccache python3 ninja-build
sudo apt-get install -yq git cmake ccache ninja-build python3 yasm
sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
- name: Run build-wrapper
run: |

View File

@ -13,9 +13,11 @@ on: # yamllint disable-line rule:truthy
branches:
- master
paths-ignore:
- 'CHANGELOG.md'
- 'README.md'
- 'SECURITY.md'
- 'docker/docs/**'
- 'docs/**'
- 'website/**'
- 'utils/check-style/aspell-ignore/**'
##########################################################################################
##################################### SMALL CHECKS #######################################
@ -3103,10 +3105,10 @@ jobs:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stress_thread
TEMP_PATH=${{runner.temp}}/stress_asan
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stress test (asan)
REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse
REPO_COPY=${{runner.temp}}/stress_asan/ClickHouse
EOF
- name: Download json reports
uses: actions/download-artifact@v3
@ -3265,6 +3267,142 @@ jobs:
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
##############################################################################################
######################################### UPGRADE CHECK ######################################
##############################################################################################
UpgradeCheckAsan:
needs: [BuilderDebAsan]
runs-on: [self-hosted, stress-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/upgrade_asan
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Upgrade check (asan)
REPO_COPY=${{runner.temp}}/upgrade_asan/ClickHouse
EOF
- name: Download json reports
uses: actions/download-artifact@v3
with:
path: ${{ env.REPORTS_PATH }}
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true
- name: Upgrade check
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 upgrade_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
UpgradeCheckTsan:
needs: [BuilderDebTsan]
# same as for stress test with tsan
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/upgrade_thread
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Upgrade check (tsan)
REPO_COPY=${{runner.temp}}/upgrade_thread/ClickHouse
EOF
- name: Download json reports
uses: actions/download-artifact@v3
with:
path: ${{ env.REPORTS_PATH }}
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true
- name: Upgrade check
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 upgrade_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
UpgradeCheckMsan:
needs: [BuilderDebMsan]
runs-on: [self-hosted, stress-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/upgrade_memory
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Upgrade check (msan)
REPO_COPY=${{runner.temp}}/upgrade_memory/ClickHouse
EOF
- name: Download json reports
uses: actions/download-artifact@v3
with:
path: ${{ env.REPORTS_PATH }}
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true
- name: Upgrade check
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 upgrade_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
UpgradeCheckDebug:
needs: [BuilderDebDebug]
runs-on: [self-hosted, stress-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/upgrade_debug
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Upgrade check (debug)
REPO_COPY=${{runner.temp}}/upgrade_debug/ClickHouse
EOF
- name: Download json reports
uses: actions/download-artifact@v3
with:
path: ${{ env.REPORTS_PATH }}
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true
- name: Upgrade check
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 upgrade_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
##############################################################################################
##################################### AST FUZZERS ############################################
##############################################################################################

View File

@ -1,9 +1,192 @@
### Table of Contents
**[ClickHouse release v23.2, 2023-02-23](#232)**<br/>
**[ClickHouse release v23.1, 2023-01-25](#231)**<br/>
**[Changelog for 2022](https://clickhouse.com/docs/en/whats-new/changelog/2022/)**<br/>
# 2023 Changelog
### <a id="232"></a> ClickHouse release 23.2, 2023-02-23
#### Backward Incompatible Change
* Extend function "toDayOfWeek()" (alias: "DAYOFWEEK") with a mode argument that encodes whether the week starts on Monday or Sunday and whether counting starts at 0 or 1. For consistency with other date time functions, the mode argument was inserted between the time and the time zone arguments. This breaks existing usage of the (previously undocumented) 2-argument syntax "toDayOfWeek(time, time_zone)". A fix is to rewrite the function into "toDayOfWeek(time, 0, time_zone)". [#45233](https://github.com/ClickHouse/ClickHouse/pull/45233) ([Robert Schulze](https://github.com/rschu1ze)).
* Rename setting `max_query_cache_size` to `filesystem_cache_max_download_size`. [#45614](https://github.com/ClickHouse/ClickHouse/pull/45614) ([Kseniia Sumarokova](https://github.com/kssenii)).
* The `default` user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. `default` user will no longer be able to grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)).
* If the SETTINGS clause is specified before the FORMAT clause, the settings will be applied to formatting as well. [#46003](https://github.com/ClickHouse/ClickHouse/pull/46003) ([Azat Khuzhin](https://github.com/azat)).
* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Slightly improve performance of `countDigits` on realistic datasets. This closed [#44518](https://github.com/ClickHouse/ClickHouse/issues/44518). In previous versions, `countDigits(0)` returned `0`; now it returns `1`, which is more correct, and follows the existing documentation. [#46187](https://github.com/ClickHouse/ClickHouse/pull/46187) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Disallow creation of new columns compressed by a combination of codecs "Delta" or "DoubleDelta" followed by codecs "Gorilla" or "FPC". This can be bypassed using setting "allow_suspicious_codecs = true". [#45652](https://github.com/ClickHouse/ClickHouse/pull/45652) ([Robert Schulze](https://github.com/rschu1ze)).
#### New Feature
* Add `StorageIceberg` and table function `iceberg` to access iceberg table store on S3. [#45384](https://github.com/ClickHouse/ClickHouse/pull/45384) ([flynn](https://github.com/ucasfl)).
* Allow configuring storage as `SETTINGS disk = '<disk_name>'` (instead of `storage_policy`) and with explicit disk creation `SETTINGS disk = disk(type=s3, ...)`. [#41976](https://github.com/ClickHouse/ClickHouse/pull/41976) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Expose `ProfileEvents` counters in `system.part_log`. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)).
* Enrichment of the existing `ReplacingMergeTree` engine to allow duplicate the insertion. It leverages the power of both `ReplacingMergeTree` and `CollapsingMergeTree` in one MergeTree engine. Deleted data are not returned when queried, but not removed from disk neither. [#41005](https://github.com/ClickHouse/ClickHouse/pull/41005) ([youennL-cs](https://github.com/youennL-cs)).
* Add `generateULID` function. Closes [#36536](https://github.com/ClickHouse/ClickHouse/issues/36536). [#44662](https://github.com/ClickHouse/ClickHouse/pull/44662) ([Nikolay Degterinsky](https://github.com/evillique)).
* Add `corrMatrix` aggregate function, calculating each two columns. In addition, since Aggregatefunctions `covarSamp` and `covarPop` are similar to `corr`, I add `covarSampMatrix`, `covarPopMatrix` by the way. @alexey-milovidov closes [#44587](https://github.com/ClickHouse/ClickHouse/issues/44587). [#44680](https://github.com/ClickHouse/ClickHouse/pull/44680) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Introduce arrayShuffle function for random array permutations. [#45271](https://github.com/ClickHouse/ClickHouse/pull/45271) ([Joanna Hulboj](https://github.com/jh0x)).
* Support types `FIXED_SIZE_BINARY` type in Arrow, `FIXED_LENGTH_BYTE_ARRAY` in `Parquet` and match them to `FixedString`. Add settings `output_format_parquet_fixed_string_as_fixed_byte_array/output_format_arrow_fixed_string_as_fixed_byte_array` to control default output type for FixedString. Closes [#45326](https://github.com/ClickHouse/ClickHouse/issues/45326). [#45340](https://github.com/ClickHouse/ClickHouse/pull/45340) ([Kruglov Pavel](https://github.com/Avogar)).
* Add a new column `last_exception_time` to system.replication_queue. [#45457](https://github.com/ClickHouse/ClickHouse/pull/45457) ([Frank Chen](https://github.com/FrankChen021)).
* Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Add `JodaTime` format support for 'x','w','S'. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)).
* Support window function `ntile`. ([lgbo](https://github.com/lgbo-ustc)).
* Add setting `final` to implicitly apply the `FINAL` modifier to every table. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)).
* Added `arrayPartialSort` and `arrayPartialReverseSort` functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)).
* The new http parameter `client_protocol_version` allows setting a client protocol version for HTTP responses using the Native format. [#40397](https://github.com/ClickHouse/ClickHouse/issues/40397). [#46360](https://github.com/ClickHouse/ClickHouse/pull/46360) ([Geoff Genz](https://github.com/genzgd)).
* Add new function `regexpExtract`, like spark function `REGEXP_EXTRACT` for compatibility. It is similar to the existing function `extract`. [#46469](https://github.com/ClickHouse/ClickHouse/pull/46469) ([李扬](https://github.com/taiyang-li)).
* Add new function `JSONArrayLength`, which returns the number of elements in the outermost JSON array. The function returns NULL if the input JSON string is invalid. [#46631](https://github.com/ClickHouse/ClickHouse/pull/46631) ([李扬](https://github.com/taiyang-li)).
#### Performance Improvement
* The introduced logic works if PREWHERE condition is a conjunction of multiple conditions (cond1 AND cond2 AND ... ). It groups those conditions that require reading the same columns into steps. After each step the corresponding part of the full condition is computed and the result rows might be filtered. This allows to read fewer rows in the next steps thus saving IO bandwidth and doing less computation. This logic is disabled by default for now. It will be enabled by default in one of the future releases once it is known to not have any regressions, so it is highly encouraged to be used for testing. It can be controlled by 2 settings: "enable_multiple_prewhere_read_steps" and "move_all_conditions_to_prewhere". [#46140](https://github.com/ClickHouse/ClickHouse/pull/46140) ([Alexander Gololobov](https://github.com/davenger)).
* An option added to aggregate partitions independently if table partition key and group by key are compatible. Controlled by the setting `allow_aggregate_partitions_independently`. Disabled by default because of limited applicability (please refer to the docs). [#45364](https://github.com/ClickHouse/ClickHouse/pull/45364) ([Nikita Taranov](https://github.com/nickitat)).
* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#45681](https://github.com/ClickHouse/ClickHouse/pull/45681) [#46282](https://github.com/ClickHouse/ClickHouse/pull/46282) ([Anton Popov](https://github.com/CurtizJ)).
* Optimize `Parquet` reader by using batch reader. [#45878](https://github.com/ClickHouse/ClickHouse/pull/45878) ([LiuNeng](https://github.com/liuneng1994)).
* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#38456](https://github.com/ClickHouse/ClickHouse/pull/38456) ([Saulius Valatka](https://github.com/sauliusvl)).
* Rewrite aggregate functions with `if` expression as argument when logically equivalent. For example, `avg(if(cond, col, null))` can be rewritten to avgIf(cond, col). It is helpful in performance. [#44730](https://github.com/ClickHouse/ClickHouse/pull/44730) ([李扬](https://github.com/taiyang-li)).
* Improve lower/upper function performance with avx512 instructions. [#37894](https://github.com/ClickHouse/ClickHouse/pull/37894) ([yaqi-zhao](https://github.com/yaqi-zhao)).
* Remove the limitation that on systems with >=32 cores and SMT disabled ClickHouse uses only half of the cores (the case when you disable Hyper Threading in BIOS). [#44973](https://github.com/ClickHouse/ClickHouse/pull/44973) ([Robert Schulze](https://github.com/rschu1ze)).
* Improve performance of function `multiIf` by columnar executing, speed up by 2.3x. [#45296](https://github.com/ClickHouse/ClickHouse/pull/45296) ([李扬](https://github.com/taiyang-li)).
* Add fast path for function `position` when the needle is empty. [#45382](https://github.com/ClickHouse/ClickHouse/pull/45382) ([李扬](https://github.com/taiyang-li)).
* Enable `query_plan_remove_redundant_sorting` optimization by default. Optimization implemented in [#45420](https://github.com/ClickHouse/ClickHouse/issues/45420). [#45567](https://github.com/ClickHouse/ClickHouse/pull/45567) ([Igor Nikonov](https://github.com/devcrafter)).
* Increased HTTP Transfer Encoding chunk size to improve performance of large queries using the HTTP interface. [#45593](https://github.com/ClickHouse/ClickHouse/pull/45593) ([Geoff Genz](https://github.com/genzgd)).
* Fixed performance of short `SELECT` queries that read from tables with large number of `Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)).
* Improve performance of filtering for big integers and decimal types. [#45949](https://github.com/ClickHouse/ClickHouse/pull/45949) ([李扬](https://github.com/taiyang-li)).
* This change could effectively reduce the overhead of obtaining the filter from ColumnNullable(UInt8) and improve the overall query performance. To evaluate the impact of this change, we adopted TPC-H benchmark but revised the column types from non-nullable to nullable, and we measured the QPS of its queries as the performance indicator. [#45962](https://github.com/ClickHouse/ClickHouse/pull/45962) ([Zhiguo Zhou](https://github.com/ZhiguoZh)).
* Make the `_part` and `_partition_id` virtual column be `LowCardinality(String)` type. Closes [#45964](https://github.com/ClickHouse/ClickHouse/issues/45964). [#45975](https://github.com/ClickHouse/ClickHouse/pull/45975) ([flynn](https://github.com/ucasfl)).
* Improve the performance of Decimal conversion when the scale does not change. [#46095](https://github.com/ClickHouse/ClickHouse/pull/46095) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Allow to increase prefetching for read data. [#46168](https://github.com/ClickHouse/ClickHouse/pull/46168) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Rewrite `arrayExists(x -> x = 1, arr)` -> `has(arr, 1)`, which improve performance by 1.34x. [#46188](https://github.com/ClickHouse/ClickHouse/pull/46188) ([李扬](https://github.com/taiyang-li)).
* Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Update zstd to v1.5.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge/mutation is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#46280](https://github.com/ClickHouse/ClickHouse/pull/46280) ([Raúl Marín](https://github.com/Algunenano)).
* Fix performance degradation caused by [#39737](https://github.com/ClickHouse/ClickHouse/issues/39737). [#46309](https://github.com/ClickHouse/ClickHouse/pull/46309) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The `replicas_status` handle will answer quickly even in case of a large replication queue. [#46310](https://github.com/ClickHouse/ClickHouse/pull/46310) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add avx512 support for aggregate function `sum`, function unary arithmetic, function comparison. [#37870](https://github.com/ClickHouse/ClickHouse/pull/37870) ([zhao zhou](https://github.com/zzachimed)).
* Rewrote the code around marks distribution and the overall coordination of the reading in order to achieve the maximum performance improvement. This closes [#34527](https://github.com/ClickHouse/ClickHouse/issues/34527). [#43772](https://github.com/ClickHouse/ClickHouse/pull/43772) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Remove redundant DISTINCT clauses in query (subqueries). Implemented on top of query plan. It does similar optimization as `optimize_duplicate_order_by_and_distinct` regarding DISTINCT clauses. Can be enabled via `query_plan_remove_redundant_distinct` setting. Related to [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#44176](https://github.com/ClickHouse/ClickHouse/pull/44176) ([Igor Nikonov](https://github.com/devcrafter)).
* A few query rewrite optimizations: `sumIf(123, cond) -> 123 * countIf(1, cond)`, `sum(if(cond, 123, 0)) -> 123 * countIf(cond)`, `sum(if(cond, 0, 123)) -> 123 * countIf(not(cond))` [#44728](https://github.com/ClickHouse/ClickHouse/pull/44728) ([李扬](https://github.com/taiyang-li)).
* Improved how memory bound merging and aggregation in order on top query plan interact. Previously we fell back to explicit sorting for AIO in some cases when it wasn't actually needed. [#45892](https://github.com/ClickHouse/ClickHouse/pull/45892) ([Nikita Taranov](https://github.com/nickitat)).
* Concurrent merges are scheduled using round-robin by default to ensure fair and starvation-free operation. Previously in heavily overloaded shards, big merges could possibly be starved by smaller merges due to the use of strict priority scheduling. Added `background_merges_mutations_scheduling_policy` server config option to select scheduling algorithm (`round_robin` or `shortest_task_first`). [#46247](https://github.com/ClickHouse/ClickHouse/pull/46247) ([Sergei Trifonov](https://github.com/serxa)).
#### Improvement
* Enable retries for INSERT by default in case of ZooKeeper session loss. We already use it in production. [#46308](https://github.com/ClickHouse/ClickHouse/pull/46308) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add ability to ignore unknown keys in JSON object for named tuples (`input_format_json_ignore_unknown_keys_in_named_tuple`). [#45678](https://github.com/ClickHouse/ClickHouse/pull/45678) ([Azat Khuzhin](https://github.com/azat)).
* Support optimizing the `where` clause with sorting key expression move to `prewhere` for query with `final`. [#38893](https://github.com/ClickHouse/ClickHouse/issues/38893). [#38950](https://github.com/ClickHouse/ClickHouse/pull/38950) ([hexiaoting](https://github.com/hexiaoting)).
* Add new metrics for backups: num_processed_files and processed_files_size described actual number of processed files. [#42244](https://github.com/ClickHouse/ClickHouse/pull/42244) ([Aleksandr](https://github.com/AVMusorin)).
* Added retries on interserver DNS errors. [#43179](https://github.com/ClickHouse/ClickHouse/pull/43179) ([Anton Kozlov](https://github.com/tonickkozlov)).
* Keeper improvement: try preallocating space on the disk to avoid undefined out-of-space issues. Introduce setting `max_log_file_size` for the maximum size of Keeper's Raft log files. [#44370](https://github.com/ClickHouse/ClickHouse/pull/44370) ([Antonio Andelic](https://github.com/antonio2368)).
* Optimize behavior for a replica delay api logic in case the replica is read-only. [#45148](https://github.com/ClickHouse/ClickHouse/pull/45148) ([mateng915](https://github.com/mateng0915)).
* Ask for the password in clickhouse-client interactively in a case when the empty password is wrong. Closes [#46702](https://github.com/ClickHouse/ClickHouse/issues/46702). [#46730](https://github.com/ClickHouse/ClickHouse/pull/46730) ([Nikolay Degterinsky](https://github.com/evillique)).
* Mark `Gorilla` compression on columns of non-Float* type as suspicious. [#45376](https://github.com/ClickHouse/ClickHouse/pull/45376) ([Robert Schulze](https://github.com/rschu1ze)).
* Show replica name that is executing a merge in the `postpone_reason` column. [#45458](https://github.com/ClickHouse/ClickHouse/pull/45458) ([Frank Chen](https://github.com/FrankChen021)).
* Save exception stack trace in part_log. [#45459](https://github.com/ClickHouse/ClickHouse/pull/45459) ([Frank Chen](https://github.com/FrankChen021)).
* The `regexp_tree` dictionary is polished and now it is compatible with https://github.com/ua-parser/uap-core. [#45631](https://github.com/ClickHouse/ClickHouse/pull/45631) ([Han Fei](https://github.com/hanfei1991)).
* Updated checking of `SYSTEM SYNC REPLICA`, resolves [#45508](https://github.com/ClickHouse/ClickHouse/issues/45508) [#45648](https://github.com/ClickHouse/ClickHouse/pull/45648) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Rename setting `replication_alter_partitions_sync` to `alter_sync`. [#45659](https://github.com/ClickHouse/ClickHouse/pull/45659) ([Antonio Andelic](https://github.com/antonio2368)).
* The `generateRandom` table function and the engine now support `LowCardinality` data types. This is useful for testing, for example you can write `INSERT INTO table SELECT * FROM generateRandom() LIMIT 1000`. This is needed to debug [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590). [#45661](https://github.com/ClickHouse/ClickHouse/pull/45661) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The experimental query result cache now provides more modular configuration settings. [#45679](https://github.com/ClickHouse/ClickHouse/pull/45679) ([Robert Schulze](https://github.com/rschu1ze)).
* Renamed "query result cache" to "query cache". [#45682](https://github.com/ClickHouse/ClickHouse/pull/45682) ([Robert Schulze](https://github.com/rschu1ze)).
* add `SYSTEM SYNC FILE CACHE` command. It will do the `sync` syscall. [#8921](https://github.com/ClickHouse/ClickHouse/issues/8921). [#45685](https://github.com/ClickHouse/ClickHouse/pull/45685) ([DR](https://github.com/freedomDR)).
* Add a new S3 setting `allow_head_object_request`. This PR makes usage of `GetObjectAttributes` request instead of `HeadObject` introduced in https://github.com/ClickHouse/ClickHouse/pull/45288 optional (and disabled by default). [#45701](https://github.com/ClickHouse/ClickHouse/pull/45701) ([Vitaly Baranov](https://github.com/vitlibar)).
* Add ability to override connection settings based on connection names (that said that now you can forget about storing password for each connection, you can simply put everything into `~/.clickhouse-client/config.xml` and even use different history files for them, which can be also useful). [#45715](https://github.com/ClickHouse/ClickHouse/pull/45715) ([Azat Khuzhin](https://github.com/azat)).
* Arrow format: support the duration type. Closes [#45669](https://github.com/ClickHouse/ClickHouse/issues/45669). [#45750](https://github.com/ClickHouse/ClickHouse/pull/45750) ([flynn](https://github.com/ucasfl)).
* Extend the logging in the Query Cache to improve investigations of the caching behavior. [#45751](https://github.com/ClickHouse/ClickHouse/pull/45751) ([Robert Schulze](https://github.com/rschu1ze)).
* The query cache's server-level settings are now reconfigurable at runtime. [#45758](https://github.com/ClickHouse/ClickHouse/pull/45758) ([Robert Schulze](https://github.com/rschu1ze)).
* Hide password in logs when a table function's arguments are specified with a named collection. [#45774](https://github.com/ClickHouse/ClickHouse/pull/45774) ([Vitaly Baranov](https://github.com/vitlibar)).
* Improve internal S3 client to correctly deduce regions and redirections for different types of URLs. [#45783](https://github.com/ClickHouse/ClickHouse/pull/45783) ([Antonio Andelic](https://github.com/antonio2368)).
* Add support for Map, IPv4 and IPv6 types in generateRandom. Mostly useful for testing. [#45785](https://github.com/ClickHouse/ClickHouse/pull/45785) ([Raúl Marín](https://github.com/Algunenano)).
* Support empty/notEmpty for IP types. [#45799](https://github.com/ClickHouse/ClickHouse/pull/45799) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* The column `num_processed_files` was split into two columns: `num_files` (for BACKUP) and `files_read` (for RESTORE). The column `processed_files_size` was split into two columns: `total_size` (for BACKUP) and `bytes_read` (for RESTORE). [#45800](https://github.com/ClickHouse/ClickHouse/pull/45800) ([Vitaly Baranov](https://github.com/vitlibar)).
* Add support for `SHOW ENGINES` query for MySQL compatibility. [#45859](https://github.com/ClickHouse/ClickHouse/pull/45859) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Improved how the obfuscator deals with queries. [#45867](https://github.com/ClickHouse/ClickHouse/pull/45867) ([Raúl Marín](https://github.com/Algunenano)).
* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#46042](https://github.com/ClickHouse/ClickHouse/pull/46042) [#45914](https://github.com/ClickHouse/ClickHouse/pull/45914) ([Joanna Hulboj](https://github.com/jh0x)).
* Add setting `check_referential_table_dependencies` to check referential dependencies on `DROP TABLE`. This PR solves [#38326](https://github.com/ClickHouse/ClickHouse/issues/38326). [#45936](https://github.com/ClickHouse/ClickHouse/pull/45936) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix `tupleElement` to return `Null` when having `Null` argument. Closes [#45894](https://github.com/ClickHouse/ClickHouse/issues/45894). [#45952](https://github.com/ClickHouse/ClickHouse/pull/45952) ([flynn](https://github.com/ucasfl)).
* Throw an error on no files satisfying the S3 wildcard. Closes [#45587](https://github.com/ClickHouse/ClickHouse/issues/45587). [#45957](https://github.com/ClickHouse/ClickHouse/pull/45957) ([chen](https://github.com/xiedeyantu)).
* Use cluster state data to check concurrent backup/restore. [#45982](https://github.com/ClickHouse/ClickHouse/pull/45982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* ClickHouse Client: Use "exact" matching for fuzzy search, which has correct case ignorance and more appropriate algorithm for matching SQL queries. [#46000](https://github.com/ClickHouse/ClickHouse/pull/46000) ([Azat Khuzhin](https://github.com/azat)).
* Forbid wrong create View syntax `CREATE View X TO Y AS SELECT`. Closes [#4331](https://github.com/ClickHouse/ClickHouse/issues/4331). [#46043](https://github.com/ClickHouse/ClickHouse/pull/46043) ([flynn](https://github.com/ucasfl)).
* Storage `Log` family support setting the `storage_policy`. Closes [#43421](https://github.com/ClickHouse/ClickHouse/issues/43421). [#46044](https://github.com/ClickHouse/ClickHouse/pull/46044) ([flynn](https://github.com/ucasfl)).
* Improve `JSONColumns` format when the result is empty. Closes [#46024](https://github.com/ClickHouse/ClickHouse/issues/46024). [#46053](https://github.com/ClickHouse/ClickHouse/pull/46053) ([flynn](https://github.com/ucasfl)).
* Add reference implementation for SipHash128. [#46065](https://github.com/ClickHouse/ClickHouse/pull/46065) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Add a new metric to record allocations times and bytes using mmap. [#46068](https://github.com/ClickHouse/ClickHouse/pull/46068) ([李扬](https://github.com/taiyang-li)).
* Currently for functions like `leftPad`, `rightPad`, `leftPadUTF8`, `rightPadUTF8`, the second argument `length` must be UInt8|16|32|64|128|256. Which is too strict for clickhouse users, besides, it is not consistent with other similar functions like `arrayResize`, `substring` and so on. [#46103](https://github.com/ClickHouse/ClickHouse/pull/46103) ([李扬](https://github.com/taiyang-li)).
* Fix assertion in the `welchTTest` function in debug build when the resulting statistics is NaN. Unified the behavior with other similar functions. Change the behavior of `studentTTest` to return NaN instead of throwing an exception because the previous behavior was inconvenient. This closes [#41176](https://github.com/ClickHouse/ClickHouse/issues/41176) This closes [#42162](https://github.com/ClickHouse/ClickHouse/issues/42162). [#46141](https://github.com/ClickHouse/ClickHouse/pull/46141) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* More convenient usage of big integers and ORDER BY WITH FILL. Allow using plain integers for start and end points in WITH FILL when ORDER BY big (128-bit and 256-bit) integers. Fix the wrong result for big integers with negative start or end points. This closes [#16733](https://github.com/ClickHouse/ClickHouse/issues/16733). [#46152](https://github.com/ClickHouse/ClickHouse/pull/46152) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add `parts`, `active_parts` and `total_marks` columns to `system.tables` on [issue](https://github.com/ClickHouse/ClickHouse/issues/44336). [#46161](https://github.com/ClickHouse/ClickHouse/pull/46161) ([attack204](https://github.com/attack204)).
* Functions "multi[Fuzzy]Match(Any|AnyIndex|AllIndices}" now reject regexes which will likely evaluate very slowly in vectorscan. [#46167](https://github.com/ClickHouse/ClickHouse/pull/46167) ([Robert Schulze](https://github.com/rschu1ze)).
* When `insert_null_as_default` is enabled and column doesn't have defined default value, the default of column type will be used. Also this PR fixes using default values on nulls in case of LowCardinality columns. [#46171](https://github.com/ClickHouse/ClickHouse/pull/46171) ([Kruglov Pavel](https://github.com/Avogar)).
* Prefer explicitly defined access keys for S3 clients. If `use_environment_credentials` is set to `true`, and the user has provided the access key through query or config, they will be used instead of the ones from the environment variable. [#46191](https://github.com/ClickHouse/ClickHouse/pull/46191) ([Antonio Andelic](https://github.com/antonio2368)).
* Add an alias "DATE_FORMAT()" for function "formatDateTime()" to improve compatibility with MySQL's SQL dialect, extend function `formatDateTime` with substitutions "a", "b", "c", "h", "i", "k", "l" "r", "s", "W". ### Documentation entry for user-facing changes User-readable short description: `DATE_FORMAT` is an alias of `formatDateTime`. Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. (Provide link to [formatDateTime](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#formatdatetime)). [#46302](https://github.com/ClickHouse/ClickHouse/pull/46302) ([Jake Bamrah](https://github.com/JakeBamrah)).
* Add `ProfileEvents` and `CurrentMetrics` about the callback tasks for parallel replicas (`s3Cluster` and `MergeTree` tables). [#46313](https://github.com/ClickHouse/ClickHouse/pull/46313) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add support for `DELETE` and `UPDATE` for tables using `KeeperMap` storage engine. [#46330](https://github.com/ClickHouse/ClickHouse/pull/46330) ([Antonio Andelic](https://github.com/antonio2368)).
* Allow writing RENAME queries with query parameters. Resolves [#45778](https://github.com/ClickHouse/ClickHouse/issues/45778). [#46407](https://github.com/ClickHouse/ClickHouse/pull/46407) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix parameterized SELECT queries with REPLACE transformer. Resolves [#33002](https://github.com/ClickHouse/ClickHouse/issues/33002). [#46420](https://github.com/ClickHouse/ClickHouse/pull/46420) ([Nikolay Degterinsky](https://github.com/evillique)).
* Exclude the internal database used for temporary/external tables from the calculation of asynchronous metric "NumberOfDatabases". This makes the behavior consistent with system table "system.databases". [#46435](https://github.com/ClickHouse/ClickHouse/pull/46435) ([Robert Schulze](https://github.com/rschu1ze)).
* Added `last_exception_time` column into distribution_queue table. [#46564](https://github.com/ClickHouse/ClickHouse/pull/46564) ([Aleksandr](https://github.com/AVMusorin)).
* Support for IN clause with parameter in parameterized views. [#46583](https://github.com/ClickHouse/ClickHouse/pull/46583) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Do not load named collections on server startup (load them on first access instead). [#46607](https://github.com/ClickHouse/ClickHouse/pull/46607) ([Kseniia Sumarokova](https://github.com/kssenii)).
#### Build/Testing/Packaging Improvement
* Introduce GWP-ASan implemented by the LLVM runtime. This closes [#27039](https://github.com/ClickHouse/ClickHouse/issues/27039). [#45226](https://github.com/ClickHouse/ClickHouse/pull/45226) ([Han Fei](https://github.com/hanfei1991)).
* We want to make our tests less stable and more flaky: add randomization for merge tree settings in tests. [#38983](https://github.com/ClickHouse/ClickHouse/pull/38983) ([Anton Popov](https://github.com/CurtizJ)).
* Enable the HDFS support in PowerPC and which helps to fixes the following functional tests 02113_hdfs_assert.sh, 02244_hdfs_cluster.sql and 02368_cancel_write_into_hdfs.sh. [#44949](https://github.com/ClickHouse/ClickHouse/pull/44949) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)).
* Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* ClickHouse's fork of poco was moved from "contrib/" to "base/poco/". [#46075](https://github.com/ClickHouse/ClickHouse/pull/46075) ([Robert Schulze](https://github.com/rschu1ze)).
* Add an option for `clickhouse-watchdog` to restart the child process. This does not make a lot of use. [#46312](https://github.com/ClickHouse/ClickHouse/pull/46312) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* If the environment variable `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` is set to 1, the Docker container will run `clickhouse-server` as a child instead of the first process, and restart it when it exited. [#46391](https://github.com/ClickHouse/ClickHouse/pull/46391) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix Systemd service file. [#46461](https://github.com/ClickHouse/ClickHouse/pull/46461) ([SuperDJY](https://github.com/cmsxbc)).
* Raised the minimum Clang version needed to build ClickHouse from 12 to 15. [#46710](https://github.com/ClickHouse/ClickHouse/pull/46710) ([Robert Schulze](https://github.com/rschu1ze)).
* Upgrade Intel QPL from v0.3.0 to v1.0.0 2. Build libaccel-config and link it statically to QPL library instead of dynamically. [#45809](https://github.com/ClickHouse/ClickHouse/pull/45809) ([jasperzhu](https://github.com/jinjunzh)).
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)).
* Fix functions (quantilesExactExclusive, quantilesExactInclusive) return unsorted array element. [#45379](https://github.com/ClickHouse/ClickHouse/pull/45379) ([wujunfu](https://github.com/wujunfu)).
* Fix uncaught exception in HTTPHandler when open telemetry is enabled. [#45456](https://github.com/ClickHouse/ClickHouse/pull/45456) ([Frank Chen](https://github.com/FrankChen021)).
* Don't infer Dates from 8 digit numbers. It could lead to wrong data to be read. [#45581](https://github.com/ClickHouse/ClickHouse/pull/45581) ([Kruglov Pavel](https://github.com/Avogar)).
* Fixes to correctly use `odbc_bridge_use_connection_pooling` setting. [#45591](https://github.com/ClickHouse/ClickHouse/pull/45591) ([Bharat Nallan](https://github.com/bharatnc)).
* When the callback in the cache is called, it is possible that this cache is destructed. To keep it safe, we capture members by value. It's also safe for task schedule because it will be deactivated before storage is destroyed. Resolve [#45548](https://github.com/ClickHouse/ClickHouse/issues/45548). [#45601](https://github.com/ClickHouse/ClickHouse/pull/45601) ([Han Fei](https://github.com/hanfei1991)).
* Fix data corruption when codecs Delta or DoubleDelta are combined with codec Gorilla. [#45615](https://github.com/ClickHouse/ClickHouse/pull/45615) ([Robert Schulze](https://github.com/rschu1ze)).
* Correctly check types when using N-gram bloom filter index to avoid invalid reads. [#45617](https://github.com/ClickHouse/ClickHouse/pull/45617) ([Antonio Andelic](https://github.com/antonio2368)).
* A couple of segfaults have been reported around `c-ares`. They were introduced in my previous pull requests. I have fixed them with the help of Alexander Tokmakov. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)).
* Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)).
* Set compression method and level for backup Closes [#45690](https://github.com/ClickHouse/ClickHouse/issues/45690). [#45737](https://github.com/ClickHouse/ClickHouse/pull/45737) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
* Should use `select_query_typed.limitByOffset()` instead of `select_query_typed.limitOffset()`. [#45817](https://github.com/ClickHouse/ClickHouse/pull/45817) ([刘陶峰](https://github.com/taofengliu)).
* When use experimental analyzer, queries like `SELECT number FROM numbers(100) LIMIT 10 OFFSET 10;` get wrong results (empty result for this sql). That is caused by an unnecessary offset step added by planner. [#45822](https://github.com/ClickHouse/ClickHouse/pull/45822) ([刘陶峰](https://github.com/taofengliu)).
* Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Add the `query_kind` column to the `system.processes` table and the `SHOW PROCESSLIST` query. Remove duplicate code. It fixes a bug: the global configuration parameter `max_concurrent_select_queries` was not respected to queries with `INTERSECT` or `EXCEPT` chains. [#45872](https://github.com/ClickHouse/ClickHouse/pull/45872) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix crash in a function `stochasticLinearRegression`. Found by WingFuzz. [#45985](https://github.com/ClickHouse/ClickHouse/pull/45985) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization`). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)).
* Fix read in order optimization for DESC sorting with FINAL, close [#45815](https://github.com/ClickHouse/ClickHouse/issues/45815). [#46009](https://github.com/ClickHouse/ClickHouse/pull/46009) ([Vladimir C](https://github.com/vdimir)).
* Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)).
* Fix elapsed column in system.processes (10x error). [#46047](https://github.com/ClickHouse/ClickHouse/pull/46047) ([Azat Khuzhin](https://github.com/azat)).
* Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. [#46087](https://github.com/ClickHouse/ClickHouse/pull/46087) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix environment variable substitution in the configuration when a parameter already has a value. This closes [#46131](https://github.com/ClickHouse/ClickHouse/issues/46131). This closes [#9547](https://github.com/ClickHouse/ClickHouse/issues/9547). [#46144](https://github.com/ClickHouse/ClickHouse/pull/46144) ([pufit](https://github.com/pufit)).
* Fix incorrect predicate push down with grouping sets. Closes [#45947](https://github.com/ClickHouse/ClickHouse/issues/45947). [#46151](https://github.com/ClickHouse/ClickHouse/pull/46151) ([flynn](https://github.com/ucasfl)).
* Fix possible pipeline stuck error on `fulls_sorting_join` with constant keys. [#46175](https://github.com/ClickHouse/ClickHouse/pull/46175) ([Vladimir C](https://github.com/vdimir)).
* Never rewrite tuple functions as literals during formatting to avoid incorrect results. [#46232](https://github.com/ClickHouse/ClickHouse/pull/46232) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Fix possible out of bounds error while reading LowCardinality(Nullable) in Arrow format. [#46270](https://github.com/ClickHouse/ClickHouse/pull/46270) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Fix possible crash which can be caused by an integer overflow while deserializing aggregating state of a function that stores HashTable. [#46349](https://github.com/ClickHouse/ClickHouse/pull/46349) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)).
* Fixed a LOGICAL_ERROR on an attempt to execute `ALTER ... MOVE PART ... TO TABLE`. This type of query was never actually supported. [#46359](https://github.com/ClickHouse/ClickHouse/pull/46359) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix s3Cluster schema inference in parallel distributed insert select when `parallel_distributed_insert_select` is enabled. [#46381](https://github.com/ClickHouse/ClickHouse/pull/46381) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix queries like `ALTER TABLE ... UPDATE nested.arr1 = nested.arr2 ...`, where `arr1` and `arr2` are fields of the same `Nested` column. [#46387](https://github.com/ClickHouse/ClickHouse/pull/46387) ([Anton Popov](https://github.com/CurtizJ)).
* Scheduler may fail to schedule a task. If it happens, the whole MulityPartUpload should be aborted and `UploadHelper` must wait for already scheduled tasks. [#46451](https://github.com/ClickHouse/ClickHouse/pull/46451) ([Dmitry Novik](https://github.com/novikd)).
* Fix PREWHERE for Merge with different default types (fixes some `NOT_FOUND_COLUMN_IN_BLOCK` when the default type for the column differs, also allow `PREWHERE` when the type of column is the same across tables, and prohibit it, only if it differs). [#46454](https://github.com/ClickHouse/ClickHouse/pull/46454) ([Azat Khuzhin](https://github.com/azat)).
* Fix a crash that could happen when constant values are used in `ORDER BY`. Fixes [#46466](https://github.com/ClickHouse/ClickHouse/issues/46466). [#46493](https://github.com/ClickHouse/ClickHouse/pull/46493) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Do not throw exception if `disk` setting was specified on query level, but `storage_policy` was specified in config merge tree settings section. `disk` will override setting from config. [#46533](https://github.com/ClickHouse/ClickHouse/pull/46533) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* fixes [#46557](https://github.com/ClickHouse/ClickHouse/issues/46557). [#46611](https://github.com/ClickHouse/ClickHouse/pull/46611) ([Alexander Gololobov](https://github.com/davenger)).
* Fix endless restarts of clickhouse-server systemd unit if server cannot start within 1m30sec (Disable timeout logic for starting clickhouse-server from systemd service). [#46613](https://github.com/ClickHouse/ClickHouse/pull/46613) ([Azat Khuzhin](https://github.com/azat)).
* Allocated during asynchronous inserts memory buffers were deallocated in the global context and MemoryTracker counters for corresponding user and query were not updated correctly. That led to false positive OOM exceptions. [#46622](https://github.com/ClickHouse/ClickHouse/pull/46622) ([Dmitry Novik](https://github.com/novikd)).
* Updated to not clear on_expression from table_join as its used by future analyze runs resolves [#45185](https://github.com/ClickHouse/ClickHouse/issues/45185). [#46487](https://github.com/ClickHouse/ClickHouse/pull/46487) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
### <a id="231"></a> ClickHouse release 23.1, 2023-01-26
### ClickHouse release 23.1

View File

@ -391,10 +391,12 @@ if (COMPILER_CLANG)
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fstrict-vtable-pointers")
# Set new experimental pass manager, it's a performance, build time and binary size win.
# Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang.
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager")
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager")
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 16)
# Set new experimental pass manager, it's a performance, build time and binary size win.
# Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang.
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager")
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager")
endif ()
# We cannot afford to use LTO when compiling unit tests, and it's not enough
# to only supply -fno-lto at the final linking stage. So we disable it

View File

@ -14,14 +14,13 @@ curl https://clickhouse.com/ | sh
* [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query a small ClickHouse cluster.
* [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Slack](https://clickhousedb.slack.com/) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time.
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time.
* [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation.
* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev.
* [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any.
## Upcoming Events
* [**ClickHouse Workshop**](https://clickhouse.com/company/events/2023-02-15-clickhouse-workshop?utm_source=github&utm_medium=social&utm_campaign=workshop) - Feb 15 & 16 - In this 2-day (3 hrs per day) free training, topics range from introductory content to a deep dive on interacting with and understanding your data. There will be both live training and hands-on labs.
* [**v23.2 Release Webinar**](https://clickhouse.com/company/events/v23-2-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Feb 23 - 23.2 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release.
* [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/291485868/) - Mar 9 - The first ClickHouse Amsterdam Meetup of 2023 is here! 🎉 Join us for short lightning talks and long discussions. Food, drinks & good times on us.
* [**ClickHouse Meetup in SF Bay Area**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/291490121/) - Mar 14 - A night to meet with ClickHouse team in the San Francisco area! Food and drink are a given...but networking is the primary focus.

View File

@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s
| Version | Supported |
|:-|:-|
| 23.2 | ✔️ |
| 23.1 | ✔️ |
| 22.12 | ✔️ |
| 22.11 | ✔️ |
| 22.11 | |
| 22.10 | ❌ |
| 22.9 | ❌ |
| 22.8 | ✔️ |

View File

@ -195,7 +195,6 @@ long splice(int fd_in, off_t *off_in, int fd_out, off_t *off_out, size_t len, un
#include <sys/stat.h>
#include <stdint.h>
#if !defined(__aarch64__)
struct statx {
uint32_t stx_mask;
uint32_t stx_blksize;
@ -226,7 +225,6 @@ int statx(int fd, const char *restrict path, int flag,
{
return syscall(SYS_statx, fd, path, flag, mask, statxbuf);
}
#endif
#include <syscall.h>

View File

@ -8,3 +8,8 @@ int fallocate(int fd, int mode, off_t base, off_t len)
{
return syscall(SYS_fallocate, fd, mode, base, len);
}
int fallocate64(int fd, int mode, off_t base, off_t len)
{
return fallocate(fd, mode, base, len);
}

View File

@ -9,3 +9,8 @@ ssize_t pwritev(int fd, const struct iovec *iov, int count, off_t ofs)
/// There was cancellable syscall (syscall_cp), but I don't care.
return syscall(SYS_pwritev, fd, iov, count, (long)(ofs), (long)(ofs>>32));
}
ssize_t pwritev64(int fd, const struct iovec *iov, int count, off_t ofs)
{
return pwritev(fd, iov, count, ofs);
}

View File

@ -136,7 +136,6 @@ struct AlignedCharArrayImpl;
// MSVC requires special handling here.
# ifdef POCO_COMPILER_CLANG
# if __has_feature(cxx_alignas)
# define POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(x) \
@ -148,17 +147,6 @@ struct AlignedCharArrayImpl;
# define POCO_HAVE_ALIGNMENT
# endif
# elif defined(__GNUC__) || defined(__IBM_ATTRIBUTES)
# define POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(x) \
template <> \
struct AlignedCharArrayImpl<x> \
{ \
char aligned __attribute__((aligned(x))); \
}
# define POCO_HAVE_ALIGNMENT
# endif
# ifdef POCO_HAVE_ALIGNMENT
POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(1);

View File

@ -76,7 +76,7 @@ public:
BinaryReader & operator>>(float & value);
BinaryReader & operator>>(double & value);
#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT)
#if !defined(POCO_LONG_IS_64_BIT)
BinaryReader & operator>>(Int64 & value);
BinaryReader & operator>>(UInt64 & value);
#endif
@ -106,12 +106,10 @@ public:
/// See BinaryWriter::write7BitEncoded() for a description
/// of the compression algorithm.
#if defined(POCO_HAVE_INT64)
void read7BitEncoded(UInt64 & value);
/// Reads a 64-bit unsigned integer in compressed format.
/// See BinaryWriter::write7BitEncoded() for a description
/// of the compression algorithm.
#endif
void readRaw(std::streamsize length, std::string & value);
/// Reads length bytes of raw data into value.

View File

@ -81,7 +81,7 @@ public:
BinaryWriter & operator<<(float value);
BinaryWriter & operator<<(double value);
#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT)
#if !defined(POCO_LONG_IS_64_BIT)
BinaryWriter & operator<<(Int64 value);
BinaryWriter & operator<<(UInt64 value);
#endif
@ -114,7 +114,6 @@ public:
/// written out. value is then shifted by seven bits and the next byte is written.
/// This process is repeated until the entire integer has been written.
#if defined(POCO_HAVE_INT64)
void write7BitEncoded(UInt64 value);
/// Writes a 64-bit unsigned integer in a compressed format.
/// The value written out seven bits at a time, starting
@ -125,7 +124,6 @@ public:
/// If value will not fit in seven bits, the high bit is set on the first byte and
/// written out. value is then shifted by seven bits and the next byte is written.
/// This process is repeated until the entire integer has been written.
#endif
void writeRaw(const std::string & rawData);
/// Writes the string as-is to the stream.

View File

@ -34,73 +34,55 @@ public:
static UInt16 flipBytes(UInt16 value);
static Int32 flipBytes(Int32 value);
static UInt32 flipBytes(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 flipBytes(Int64 value);
static UInt64 flipBytes(UInt64 value);
#endif
static Int16 toBigEndian(Int16 value);
static UInt16 toBigEndian(UInt16 value);
static Int32 toBigEndian(Int32 value);
static UInt32 toBigEndian(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 toBigEndian(Int64 value);
static UInt64 toBigEndian(UInt64 value);
#endif
static Int16 fromBigEndian(Int16 value);
static UInt16 fromBigEndian(UInt16 value);
static Int32 fromBigEndian(Int32 value);
static UInt32 fromBigEndian(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 fromBigEndian(Int64 value);
static UInt64 fromBigEndian(UInt64 value);
#endif
static Int16 toLittleEndian(Int16 value);
static UInt16 toLittleEndian(UInt16 value);
static Int32 toLittleEndian(Int32 value);
static UInt32 toLittleEndian(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 toLittleEndian(Int64 value);
static UInt64 toLittleEndian(UInt64 value);
#endif
static Int16 fromLittleEndian(Int16 value);
static UInt16 fromLittleEndian(UInt16 value);
static Int32 fromLittleEndian(Int32 value);
static UInt32 fromLittleEndian(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 fromLittleEndian(Int64 value);
static UInt64 fromLittleEndian(UInt64 value);
#endif
static Int16 toNetwork(Int16 value);
static UInt16 toNetwork(UInt16 value);
static Int32 toNetwork(Int32 value);
static UInt32 toNetwork(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 toNetwork(Int64 value);
static UInt64 toNetwork(UInt64 value);
#endif
static Int16 fromNetwork(Int16 value);
static UInt16 fromNetwork(UInt16 value);
static Int32 fromNetwork(Int32 value);
static UInt32 fromNetwork(UInt32 value);
#if defined(POCO_HAVE_INT64)
static Int64 fromNetwork(Int64 value);
static UInt64 fromNetwork(UInt64 value);
#endif
};
#if !defined(POCO_NO_BYTESWAP_BUILTINS)
# if defined(__clang__)
# if __has_builtin(__builtin_bswap32)
# define POCO_HAVE_GCC_BYTESWAP 1
# endif
# elif defined(__GNUC__) && (__GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 3))
# if __has_builtin(__builtin_bswap32)
# define POCO_HAVE_GCC_BYTESWAP 1
# endif
#endif
@ -143,7 +125,6 @@ inline Int32 ByteOrder::flipBytes(Int32 value)
}
#if defined(POCO_HAVE_INT64)
inline UInt64 ByteOrder::flipBytes(UInt64 value)
{
# if defined(POCO_HAVE_MSC_BYTESWAP)
@ -162,7 +143,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value)
{
return Int64(flipBytes(UInt64(value)));
}
#endif // POCO_HAVE_INT64
//
@ -180,7 +160,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value)
}
#if defined(POCO_HAVE_INT64)
# define POCO_IMPLEMENT_BYTEORDER_NOOP(op) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int16) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt16) \
@ -195,18 +174,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value)
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt32) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int64) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt64)
#else
# define POCO_IMPLEMENT_BYTEORDER_NOOP(op) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int16) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt16) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int32) \
POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt32)
# define POCO_IMPLEMENT_BYTEORDER_FLIP(op) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int16) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt16) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int32) \
POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt32)
#endif
#if defined(POCO_ARCH_BIG_ENDIAN)

View File

@ -98,10 +98,8 @@
# define POCO_DEPRECATED
#elif defined(_GNUC_)
# define POCO_DEPRECATED __attribute__((deprecated))
#elif defined(__clang__)
# define POCO_DEPRECATED __attribute__((deprecated))
#else
# define POCO_DEPRECATED
# define POCO_DEPRECATED __attribute__((deprecated))
#endif

View File

@ -151,7 +151,6 @@ public:
/// If prefix is true, "0x" prefix is prepended to the
/// resulting string.
#ifdef POCO_HAVE_INT64
# ifdef POCO_LONG_IS_64_BIT
@ -255,7 +254,6 @@ public:
# endif // ifdef POCO_LONG_IS_64_BIT
#endif // ifdef POCO_HAVE_INT64
static std::string format(float value);
/// Formats a float value in decimal floating-point notation,
@ -380,7 +378,6 @@ public:
/// right justified and zero-padded in a field having at least the
/// specified width.
#ifdef POCO_HAVE_INT64
# ifdef POCO_LONG_IS_64_BIT
@ -472,7 +469,6 @@ public:
# endif // ifdef POCO_LONG_IS_64_BIT
#endif // ifdef POCO_HAVE_INT64
static void append(std::string & str, float value);
/// Formats a float value in decimal floating-point notation,
@ -673,7 +669,6 @@ inline std::string NumberFormatter::formatHex(unsigned long value, int width, bo
}
#ifdef POCO_HAVE_INT64
# ifdef POCO_LONG_IS_64_BIT
@ -843,7 +838,6 @@ inline std::string NumberFormatter::formatHex(UInt64 value, int width, bool pref
# endif // ifdef POCO_LONG_IS_64_BIT
#endif // ifdef POCO_HAVE_INT64
inline std::string NumberFormatter::format(float value)

View File

@ -80,7 +80,6 @@ public:
/// Returns true if a valid integer has been found, false otherwise.
/// If parsing was not successful, value is undefined.
#if defined(POCO_HAVE_INT64)
static Int64 parse64(const std::string & s, char thousandSeparator = ',');
/// Parses a 64-bit integer value in decimal notation from the given string.
@ -118,7 +117,6 @@ public:
/// Returns true if a valid integer has been found, false otherwise.
/// If parsing was not successful, value is undefined.
#endif // defined(POCO_HAVE_INT64)
static double parseFloat(const std::string & s, char decimalSeparator = '.', char thousandSeparator = ',');
/// Parses a double value in decimal floating point notation

View File

@ -212,25 +212,6 @@
#endif
#if defined(__clang__)
# define POCO_COMPILER_CLANG
#elif defined(__GNUC__)
# define POCO_COMPILER_GCC
#elif defined(__MINGW32__) || defined(__MINGW64__)
# define POCO_COMPILER_MINGW
#elif defined(__INTEL_COMPILER) || defined(__ICC) || defined(__ECC) || defined(__ICL)
# define POCO_COMPILER_INTEL
#elif defined(__MWERKS__) || defined(__CWCC__)
# define POCO_COMPILER_CODEWARRIOR
#elif defined(__sgi) || defined(sgi)
# define POCO_COMPILER_SGI
#elif defined(__BORLANDC__) || defined(__CODEGEARC__)
# define POCO_COMPILER_CBUILDER
#elif defined(__DMC__)
# define POCO_COMPILER_DMARS
#endif
#ifdef __GNUC__
# define POCO_UNUSED __attribute__((unused))
#else

View File

@ -32,10 +32,8 @@
//
// Thread-safety of local static initialization
//
#if __cplusplus >= 201103L || __GNUC__ >= 4 || defined(__clang__)
# ifndef POCO_LOCAL_STATIC_INIT_IS_THREADSAFE
# define POCO_LOCAL_STATIC_INIT_IS_THREADSAFE 1
# endif
#ifndef POCO_LOCAL_STATIC_INIT_IS_THREADSAFE
# define POCO_LOCAL_STATIC_INIT_IS_THREADSAFE 1
#endif

View File

@ -38,7 +38,6 @@ public:
///
/// Returns the number of bytes copied.
#if defined(POCO_HAVE_INT64)
static Poco::UInt64 copyStream64(std::istream & istr, std::ostream & ostr, std::size_t bufferSize = 8192);
/// Writes all bytes readable from istr to ostr, using an internal buffer.
///
@ -46,14 +45,12 @@ public:
///
/// Note: the only difference to copyStream() is that a 64-bit unsigned
/// integer is used to count the number of bytes copied.
#endif
static std::streamsize copyStreamUnbuffered(std::istream & istr, std::ostream & ostr);
/// Writes all bytes readable from istr to ostr.
///
/// Returns the number of bytes copied.
#if defined(POCO_HAVE_INT64)
static Poco::UInt64 copyStreamUnbuffered64(std::istream & istr, std::ostream & ostr);
/// Writes all bytes readable from istr to ostr.
///
@ -61,14 +58,12 @@ public:
///
/// Note: the only difference to copyStreamUnbuffered() is that a 64-bit unsigned
/// integer is used to count the number of bytes copied.
#endif
static std::streamsize copyToString(std::istream & istr, std::string & str, std::size_t bufferSize = 8192);
/// Appends all bytes readable from istr to the given string, using an internal buffer.
///
/// Returns the number of bytes copied.
#if defined(POCO_HAVE_INT64)
static Poco::UInt64 copyToString64(std::istream & istr, std::string & str, std::size_t bufferSize = 8192);
/// Appends all bytes readable from istr to the given string, using an internal buffer.
///
@ -76,7 +71,6 @@ public:
///
/// Note: the only difference to copyToString() is that a 64-bit unsigned
/// integer is used to count the number of bytes copied.
#endif
};

View File

@ -67,19 +67,7 @@ public:
void swap(Timespan & timespan);
/// Swaps the Timespan with another one.
bool operator==(const Timespan & ts) const;
bool operator!=(const Timespan & ts) const;
bool operator>(const Timespan & ts) const;
bool operator>=(const Timespan & ts) const;
bool operator<(const Timespan & ts) const;
bool operator<=(const Timespan & ts) const;
bool operator==(TimeDiff microSeconds) const;
bool operator!=(TimeDiff microSeconds) const;
bool operator>(TimeDiff microSeconds) const;
bool operator>=(TimeDiff microSeconds) const;
bool operator<(TimeDiff microSeconds) const;
bool operator<=(TimeDiff microSeconds) const;
auto operator<=>(const Timespan & ts) const = default;
Timespan operator+(const Timespan & d) const;
Timespan operator-(const Timespan & d) const;
@ -215,78 +203,6 @@ inline Timespan::TimeDiff Timespan::totalMicroseconds() const
}
inline bool Timespan::operator==(const Timespan & ts) const
{
return _span == ts._span;
}
inline bool Timespan::operator!=(const Timespan & ts) const
{
return _span != ts._span;
}
inline bool Timespan::operator>(const Timespan & ts) const
{
return _span > ts._span;
}
inline bool Timespan::operator>=(const Timespan & ts) const
{
return _span >= ts._span;
}
inline bool Timespan::operator<(const Timespan & ts) const
{
return _span < ts._span;
}
inline bool Timespan::operator<=(const Timespan & ts) const
{
return _span <= ts._span;
}
inline bool Timespan::operator==(TimeDiff microSeconds) const
{
return _span == microSeconds;
}
inline bool Timespan::operator!=(TimeDiff microSeconds) const
{
return _span != microSeconds;
}
inline bool Timespan::operator>(TimeDiff microSeconds) const
{
return _span > microSeconds;
}
inline bool Timespan::operator>=(TimeDiff microSeconds) const
{
return _span >= microSeconds;
}
inline bool Timespan::operator<(TimeDiff microSeconds) const
{
return _span < microSeconds;
}
inline bool Timespan::operator<=(TimeDiff microSeconds) const
{
return _span <= microSeconds;
}
inline void swap(Timespan & s1, Timespan & s2)
{
s1.swap(s2);

View File

@ -84,13 +84,11 @@ public:
virtual std::string asString() const;
/// Returns a string representation of the token.
#if defined(POCO_HAVE_INT64)
virtual Int64 asInteger64() const;
/// Returns a 64-bit integer representation of the token.
virtual UInt64 asUnsignedInteger64() const;
/// Returns an unsigned 64-bit integer representation of the token.
#endif
virtual int asInteger() const;
/// Returns an integer representation of the token.

View File

@ -25,7 +25,6 @@ namespace Poco
{
#if defined(__GNUC__) || defined(__clang__)
//
// Unix/GCC/Clang
//
@ -46,8 +45,6 @@ typedef unsigned long UInt64;
typedef signed long long Int64;
typedef unsigned long long UInt64;
# endif
# define POCO_HAVE_INT64 1
#endif
} // namespace Poco

View File

@ -170,7 +170,7 @@ BinaryReader& BinaryReader::operator >> (double& value)
}
#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT)
#if !defined(POCO_LONG_IS_64_BIT)
BinaryReader& BinaryReader::operator >> (Int64& value)
@ -233,7 +233,6 @@ void BinaryReader::read7BitEncoded(UInt32& value)
}
#if defined(POCO_HAVE_INT64)
void BinaryReader::read7BitEncoded(UInt64& value)
@ -254,7 +253,6 @@ void BinaryReader::read7BitEncoded(UInt64& value)
}
#endif
void BinaryReader::readRaw(std::streamsize length, std::string& value)

View File

@ -212,7 +212,7 @@ BinaryWriter& BinaryWriter::operator << (double value)
}
#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT)
#if !defined(POCO_LONG_IS_64_BIT)
BinaryWriter& BinaryWriter::operator << (Int64 value)
@ -303,7 +303,6 @@ void BinaryWriter::write7BitEncoded(UInt32 value)
}
#if defined(POCO_HAVE_INT64)
void BinaryWriter::write7BitEncoded(UInt64 value)
@ -319,7 +318,6 @@ void BinaryWriter::write7BitEncoded(UInt64 value)
}
#endif
void BinaryWriter::writeRaw(const std::string& rawData)

View File

@ -234,7 +234,6 @@ void NumberFormatter::appendHex(std::string& str, unsigned long value, int width
}
#ifdef POCO_HAVE_INT64
#ifdef POCO_LONG_IS_64_BIT
@ -424,7 +423,6 @@ void NumberFormatter::appendHex(std::string& str, UInt64 value, int width)
#endif // ifdef POCO_LONG_IS_64_BIT
#endif // ifdef POCO_HAVE_INT64
void NumberFormatter::append(std::string& str, float value)

View File

@ -44,7 +44,7 @@ int NumberParser::parse(const std::string& s, char thSep)
if (tryParse(s, result, thSep))
return result;
else
throw SyntaxException("Not a valid integer", s);
throw SyntaxException("Not a valid integer", "'" + s + "'");
}
@ -60,7 +60,7 @@ unsigned NumberParser::parseUnsigned(const std::string& s, char thSep)
if (tryParseUnsigned(s, result, thSep))
return result;
else
throw SyntaxException("Not a valid unsigned integer", s);
throw SyntaxException("Not a valid unsigned integer", "'" + s + "'");
}
@ -76,7 +76,7 @@ unsigned NumberParser::parseHex(const std::string& s)
if (tryParseHex(s, result))
return result;
else
throw SyntaxException("Not a valid hexadecimal integer", s);
throw SyntaxException("Not a valid hexadecimal integer", "'" + s + "'");
}
@ -94,7 +94,7 @@ unsigned NumberParser::parseOct(const std::string& s)
if (tryParseOct(s, result))
return result;
else
throw SyntaxException("Not a valid hexadecimal integer", s);
throw SyntaxException("Not a valid hexadecimal integer", "'" + s + "'");
}
@ -104,7 +104,6 @@ bool NumberParser::tryParseOct(const std::string& s, unsigned& value)
}
#if defined(POCO_HAVE_INT64)
Int64 NumberParser::parse64(const std::string& s, char thSep)
@ -113,7 +112,7 @@ Int64 NumberParser::parse64(const std::string& s, char thSep)
if (tryParse64(s, result, thSep))
return result;
else
throw SyntaxException("Not a valid integer", s);
throw SyntaxException("Not a valid integer", "'" + s + "'");
}
@ -129,7 +128,7 @@ UInt64 NumberParser::parseUnsigned64(const std::string& s, char thSep)
if (tryParseUnsigned64(s, result, thSep))
return result;
else
throw SyntaxException("Not a valid unsigned integer", s);
throw SyntaxException("Not a valid unsigned integer", "'" + s + "'");
}
@ -145,7 +144,7 @@ UInt64 NumberParser::parseHex64(const std::string& s)
if (tryParseHex64(s, result))
return result;
else
throw SyntaxException("Not a valid hexadecimal integer", s);
throw SyntaxException("Not a valid hexadecimal integer", "'" + s + "'");
}
@ -163,7 +162,7 @@ UInt64 NumberParser::parseOct64(const std::string& s)
if (tryParseOct64(s, result))
return result;
else
throw SyntaxException("Not a valid hexadecimal integer", s);
throw SyntaxException("Not a valid hexadecimal integer", "'" + s + "'");
}
@ -173,7 +172,6 @@ bool NumberParser::tryParseOct64(const std::string& s, UInt64& value)
}
#endif // defined(POCO_HAVE_INT64)
double NumberParser::parseFloat(const std::string& s, char decSep, char thSep)
@ -182,7 +180,7 @@ double NumberParser::parseFloat(const std::string& s, char decSep, char thSep)
if (tryParseFloat(s, result, decSep, thSep))
return result;
else
throw SyntaxException("Not a valid floating-point number", s);
throw SyntaxException("Not a valid floating-point number", "'" + s + "'");
}
@ -198,7 +196,7 @@ bool NumberParser::parseBool(const std::string& s)
if (tryParseBool(s, result))
return result;
else
throw SyntaxException("Not a valid bool number", s);
throw SyntaxException("Not a valid bool number", "'" + s + "'");
}

View File

@ -42,7 +42,6 @@ std::streamsize StreamCopier::copyStream(std::istream& istr, std::ostream& ostr,
}
#if defined(POCO_HAVE_INT64)
Poco::UInt64 StreamCopier::copyStream64(std::istream& istr, std::ostream& ostr, std::size_t bufferSize)
{
poco_assert (bufferSize > 0);
@ -64,7 +63,6 @@ Poco::UInt64 StreamCopier::copyStream64(std::istream& istr, std::ostream& ostr,
}
return len;
}
#endif
std::streamsize StreamCopier::copyToString(std::istream& istr, std::string& str, std::size_t bufferSize)
@ -90,7 +88,6 @@ std::streamsize StreamCopier::copyToString(std::istream& istr, std::string& str,
}
#if defined(POCO_HAVE_INT64)
Poco::UInt64 StreamCopier::copyToString64(std::istream& istr, std::string& str, std::size_t bufferSize)
{
poco_assert (bufferSize > 0);
@ -112,7 +109,6 @@ Poco::UInt64 StreamCopier::copyToString64(std::istream& istr, std::string& str,
}
return len;
}
#endif
std::streamsize StreamCopier::copyStreamUnbuffered(std::istream& istr, std::ostream& ostr)
@ -130,7 +126,6 @@ std::streamsize StreamCopier::copyStreamUnbuffered(std::istream& istr, std::ostr
}
#if defined(POCO_HAVE_INT64)
Poco::UInt64 StreamCopier::copyStreamUnbuffered64(std::istream& istr, std::ostream& ostr)
{
char c = 0;
@ -144,7 +139,6 @@ Poco::UInt64 StreamCopier::copyStreamUnbuffered64(std::istream& istr, std::ostre
}
return len;
}
#endif
} // namespace Poco

View File

@ -54,7 +54,6 @@ std::string Token::asString() const
}
#if defined(POCO_HAVE_INT64)
Int64 Token::asInteger64() const
{
return NumberParser::parse64(_value);
@ -65,7 +64,6 @@ UInt64 Token::asUnsignedInteger64() const
{
return NumberParser::parseUnsigned64(_value);
}
#endif
int Token::asInteger() const

View File

@ -74,14 +74,12 @@ namespace JSON
/// An unsigned value is read. This will only be triggered if the
/// value cannot fit into a signed int.
#if defined(POCO_HAVE_INT64)
virtual void value(Int64 v) = 0;
/// A 64-bit integer value is read.
virtual void value(UInt64 v) = 0;
/// An unsigned 64-bit integer value is read. This will only be
/// triggered if the value cannot fit into a signed 64-bit integer.
#endif
virtual void value(const std::string & value) = 0;
/// A string value is read.

View File

@ -73,14 +73,12 @@ namespace JSON
/// An unsigned value is read. This will only be triggered if the
/// value cannot fit into a signed int.
#if defined(POCO_HAVE_INT64)
virtual void value(Int64 v);
/// A 64-bit integer value is read
virtual void value(UInt64 v);
/// An unsigned 64-bit integer value is read. This will only be
/// triggered if the value cannot fit into a signed 64-bit integer.
#endif
virtual void value(const std::string & s);
/// A string value is read.
@ -126,7 +124,6 @@ namespace JSON
}
#if defined(POCO_HAVE_INT64)
inline void ParseHandler::value(Int64 v)
{
setValue(v);
@ -137,7 +134,6 @@ namespace JSON
{
setValue(v);
}
#endif
inline void ParseHandler::value(const std::string & s)

View File

@ -81,13 +81,11 @@ namespace JSON
/// An unsigned value is read. This will only be triggered if the
/// value cannot fit into a signed int.
#if defined(POCO_HAVE_INT64)
void value(Int64 v);
/// A 64-bit integer value is read; it will be written to the output.
void value(UInt64 v);
/// An unsigned 64-bit integer value is read; it will be written to the output.
#endif
void value(const std::string & value);
/// A string value is read; it will be formatted and written to the output.

View File

@ -154,7 +154,6 @@ void PrintHandler::value(unsigned v)
}
#if defined(POCO_HAVE_INT64)
void PrintHandler::value(Int64 v)
{
arrayValue();
@ -169,7 +168,6 @@ void PrintHandler::value(UInt64 v)
_out << v;
_objStart = false;
}
#endif
void PrintHandler::value(const std::string& value)

View File

@ -43,11 +43,7 @@ namespace Net
public:
typedef HTTPBasicStreamBuf::openmode openmode;
#if defined(POCO_HAVE_INT64)
typedef Poco::Int64 ContentLength;
#else
typedef std::streamsize ContentLength;
#endif
HTTPFixedLengthStreamBuf(HTTPSession & session, ContentLength length, openmode mode);
~HTTPFixedLengthStreamBuf();

View File

@ -56,7 +56,6 @@ namespace Net
/// which may be UNKNOWN_CONTENT_LENGTH if
/// no Content-Length header is present.
#if defined(POCO_HAVE_INT64)
void setContentLength64(Poco::Int64 length);
/// Sets the Content-Length header.
///
@ -73,7 +72,6 @@ namespace Net
///
/// In contrast to getContentLength(), this method
/// always returns a 64-bit integer for content length.
#endif // defined(POCO_HAVE_INT64)
bool hasContentLength() const;
/// Returns true iff a Content-Length header is present.

View File

@ -132,14 +132,10 @@ namespace Net
/// Writes the HTTP request to the given
/// output stream.
#if __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Woverloaded-virtual"
#endif
void read(std::istream & istr);
#if __clang__
# pragma clang diagnostic pop
#endif
/// Reads the HTTP request from the
/// given input stream.

View File

@ -188,14 +188,10 @@ namespace Net
/// Writes the HTTP response to the given
/// output stream, but do not finish with \r\n delimiter.
#if __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Woverloaded-virtual"
#endif
void read(std::istream & istr);
#if __clang__
# pragma clang diagnostic pop
#endif
/// Reads the HTTP response from the
/// given input stream.
///

View File

@ -264,11 +264,7 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request)
{
Poco::CountingOutputStream cs;
request.write(cs);
#if POCO_HAVE_INT64
_pRequestStream = new HTTPFixedLengthOutputStream(*this, request.getContentLength64() + cs.chars());
#else
_pRequestStream = new HTTPFixedLengthOutputStream(*this, request.getContentLength() + cs.chars());
#endif
request.write(*_pRequestStream);
}
else if ((method != HTTPRequest::HTTP_PUT && method != HTTPRequest::HTTP_POST && method != HTTPRequest::HTTP_PATCH) || request.has(HTTPRequest::UPGRADE))
@ -334,11 +330,7 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response)
else if (response.getChunkedTransferEncoding())
_pResponseStream = new HTTPChunkedInputStream(*this);
else if (response.hasContentLength())
#if defined(POCO_HAVE_INT64)
_pResponseStream = new HTTPFixedLengthInputStream(*this, response.getContentLength64());
#else
_pResponseStream = new HTTPFixedLengthInputStream(*this, response.getContentLength());
#endif
else
_pResponseStream = new HTTPInputStream(*this);

View File

@ -89,7 +89,6 @@ std::streamsize HTTPMessage::getContentLength() const
}
#if defined(POCO_HAVE_INT64)
void HTTPMessage::setContentLength64(Poco::Int64 length)
{
if (length != UNKNOWN_CONTENT_LENGTH)
@ -108,7 +107,6 @@ Poco::Int64 HTTPMessage::getContentLength64() const
}
else return UNKNOWN_CONTENT_LENGTH;
}
#endif // defined(POCO_HAVE_INT64)
void HTTPMessage::setTransferEncoding(const std::string& transferEncoding)

View File

@ -49,11 +49,7 @@ HTTPServerRequestImpl::HTTPServerRequestImpl(HTTPServerResponseImpl& response, H
if (getChunkedTransferEncoding())
_pStream = new HTTPChunkedInputStream(session);
else if (hasContentLength())
#if defined(POCO_HAVE_INT64)
_pStream = new HTTPFixedLengthInputStream(session, getContentLength64());
#else
_pStream = new HTTPFixedLengthInputStream(session, getContentLength());
#endif
else if (getMethod() == HTTPRequest::HTTP_GET || getMethod() == HTTPRequest::HTTP_HEAD || getMethod() == HTTPRequest::HTTP_DELETE)
_pStream = new HTTPFixedLengthInputStream(session, 0);
else

View File

@ -92,11 +92,7 @@ std::ostream& HTTPServerResponseImpl::send()
{
Poco::CountingOutputStream cs;
write(cs);
#if defined(POCO_HAVE_INT64)
_pStream = new HTTPFixedLengthOutputStream(_session, getContentLength64() + cs.chars());
#else
_pStream = new HTTPFixedLengthOutputStream(_session, getContentLength() + cs.chars());
#endif
write(*_pStream);
}
else
@ -153,11 +149,7 @@ void HTTPServerResponseImpl::sendFile(const std::string& path, const std::string
Timestamp dateTime = f.getLastModified();
File::FileSize length = f.getSize();
set("Last-Modified", DateTimeFormatter::format(dateTime, DateTimeFormat::HTTP_FORMAT));
#if defined(POCO_HAVE_INT64)
setContentLength64(length);
#else
setContentLength(static_cast<int>(length));
#endif
setContentType(mediaType);
setChunkedTransferEncoding(false);

View File

@ -167,7 +167,6 @@ namespace Util
/// If the value contains references to other properties (${<property>}), these
/// are expanded.
#if defined(POCO_HAVE_INT64)
Int64 getInt64(const std::string & key) const;
/// Returns the Int64 value of the property with the given name.
@ -205,7 +204,6 @@ namespace Util
/// If the value contains references to other properties (${<property>}), these
/// are expanded.
#endif // defined(POCO_HAVE_INT64)
double getDouble(const std::string & key) const;
/// Returns the double value of the property with the given name.
@ -255,7 +253,6 @@ namespace Util
/// Sets the property with the given key to the given value.
/// An already existing value for the key is overwritten.
#if defined(POCO_HAVE_INT64)
virtual void setInt64(const std::string & key, Int64 value);
/// Sets the property with the given key to the given value.
@ -265,7 +262,6 @@ namespace Util
/// Sets the property with the given key to the given value.
/// An already existing value for the key is overwritten.
#endif // defined(POCO_HAVE_INT64)
virtual void setDouble(const std::string & key, double value);
/// Sets the property with the given key to the given value.
@ -335,12 +331,10 @@ namespace Util
static int parseInt(const std::string & value);
static unsigned parseUInt(const std::string & value);
#if defined(POCO_HAVE_INT64)
static Int64 parseInt64(const std::string & value);
static UInt64 parseUInt64(const std::string & value);
#endif // defined(POCO_HAVE_INT64)
static bool parseBool(const std::string & value);
void setRawWithEvent(const std::string & key, std::string value);

View File

@ -123,7 +123,6 @@ namespace Util
///
/// Throws a NotFoundException if the value does not exist.
#if defined(POCO_HAVE_INT64)
void setInt64(const std::string & name, Poco::Int64 value);
/// Sets the numeric (REG_QWORD) value with the given name.
@ -135,7 +134,6 @@ namespace Util
///
/// Throws a NotFoundException if the value does not exist.
#endif // POCO_HAVE_INT64
void deleteValue(const std::string & name);
/// Deletes the value with the given name.

View File

@ -163,7 +163,6 @@ unsigned AbstractConfiguration::getUInt(const std::string& key, unsigned default
}
#if defined(POCO_HAVE_INT64)
Int64 AbstractConfiguration::getInt64(const std::string& key) const
@ -214,7 +213,6 @@ UInt64 AbstractConfiguration::getUInt64(const std::string& key, UInt64 defaultVa
}
#endif // defined(POCO_HAVE_INT64)
double AbstractConfiguration::getDouble(const std::string& key) const
@ -283,7 +281,6 @@ void AbstractConfiguration::setUInt(const std::string& key, unsigned int value)
}
#if defined(POCO_HAVE_INT64)
void AbstractConfiguration::setInt64(const std::string& key, Int64 value)
@ -302,7 +299,6 @@ void AbstractConfiguration::setUInt64(const std::string& key, UInt64 value)
}
#endif // defined(POCO_HAVE_INT64)
void AbstractConfiguration::setDouble(const std::string& key, double value)

View File

@ -2,11 +2,11 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
SET(VERSION_REVISION 54471)
SET(VERSION_REVISION 54472)
SET(VERSION_MAJOR 23)
SET(VERSION_MINOR 2)
SET(VERSION_MINOR 3)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH dcaac47702510cc87ddf266bc524f6b7ce0a8e6e)
SET(VERSION_DESCRIBE v23.2.1.1-testing)
SET(VERSION_STRING 23.2.1.1)
SET(VERSION_GITHASH 52bf836e03a6ba7cf2d654eaaf73231701abc3a2)
SET(VERSION_DESCRIBE v23.3.1.2537-testing)
SET(VERSION_STRING 23.3.1.2537)
# end of autochange

View File

@ -30,7 +30,7 @@ elseif (ARCH_AARCH64)
# support it.
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=armv8+crc")
else ()
# ARMv8.2 is quite ancient but the lowest common denominator supported by both Graviton 2 and 3 processors [1]. In particular, it
# ARMv8.2 is quite ancient but the lowest common denominator supported by both Graviton 2 and 3 processors [1, 10]. In particular, it
# includes LSE (made mandatory with ARMv8.1) which provides nice speedups without having to fall back to compat flag
# "-moutline-atomics" for v8.0 [2, 3, 4] that requires a recent glibc with runtime dispatch helper, limiting our ability to run on
# old OSs.
@ -45,21 +45,39 @@ elseif (ARCH_AARCH64)
# dotprod: Scalar vector product (SDOT and UDOT instructions). Probably the most obscure extra flag with doubtful performance benefits
# but it has been activated since always, so why not enable it. It's not 100% clear in which revision this flag was
# introduced as optional, either in v8.2 [7] or in v8.4 [8].
# ldapr: Load-Acquire RCpc Register. Better support of release/acquire of atomics. Good for allocators and high contention code.
# Optional in v8.2, mandatory in v8.3 [9]. Supported in Graviton 2+, Azure and GCP instances. Generated from clang 15.
# rcpc: Load-Acquire RCpc Register. Better support of release/acquire of atomics. Good for allocators and high contention code.
# Optional in v8.2, mandatory in v8.3 [9]. Supported in Graviton >=2, Azure and GCP instances.
#
# [1] https://github.com/aws/aws-graviton-getting-started/blob/main/c-c%2B%2B.md
# [2] https://community.arm.com/arm-community-blogs/b/tools-software-ides-blog/posts/making-the-most-of-the-arm-architecture-in-gcc-10
# [3] https://mysqlonarm.github.io/ARM-LSE-and-MySQL/
# [4] https://dev.to/aws-builders/large-system-extensions-for-aws-graviton-processors-3eci
# [5] https://developer.arm.com/tools-and-software/open-source-software/developer-tools/llvm-toolchain/sve-support
# [6] https://developer.arm.com/documentation/100067/0612/armclang-Command-line-Options/-mcpu?lang=en
# [7] https://gcc.gnu.org/onlinedocs/gcc/ARM-Options.html
# [8] https://developer.arm.com/documentation/102651/a/What-are-dot-product-intructions-
# [9] https://developer.arm.com/documentation/dui0801/g/A64-Data-Transfer-Instructions/LDAPR?lang=en
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=armv8.2-a+simd+crypto+dotprod+ssbs -Xclang=-target-feature -Xclang=+ldapr -Wno-unused-command-line-argument")
# [1] https://github.com/aws/aws-graviton-getting-started/blob/main/c-c%2B%2B.md
# [2] https://community.arm.com/arm-community-blogs/b/tools-software-ides-blog/posts/making-the-most-of-the-arm-architecture-in-gcc-10
# [3] https://mysqlonarm.github.io/ARM-LSE-and-MySQL/
# [4] https://dev.to/aws-builders/large-system-extensions-for-aws-graviton-processors-3eci
# [5] https://developer.arm.com/tools-and-software/open-source-software/developer-tools/llvm-toolchain/sve-support
# [6] https://developer.arm.com/documentation/100067/0612/armclang-Command-line-Options/-mcpu?lang=en
# [7] https://gcc.gnu.org/onlinedocs/gcc/ARM-Options.html
# [8] https://developer.arm.com/documentation/102651/a/What-are-dot-product-intructions-
# [9] https://developer.arm.com/documentation/dui0801/g/A64-Data-Transfer-Instructions/LDAPR?lang=en
# [10] https://github.com/aws/aws-graviton-getting-started/blob/main/README.md
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=armv8.2-a+simd+crypto+dotprod+ssbs+rcpc")
endif ()
# Best-effort check: The build generates and executes intermediate binaries, e.g. protoc and llvm-tablegen. If we build on ARM for ARM
# and the build machine is too old, i.e. doesn't satisfy above modern profile, then these intermediate binaries will not run (dump
# SIGILL). Even if they could run, the build machine wouldn't be able to run the ClickHouse binary. In that case, suggest to run the
# build with the compat profile.
if (OS_LINUX AND CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)" AND NOT NO_ARMV81_OR_HIGHER)
# CPU features in /proc/cpuinfo and compiler flags don't align :( ... pick some obvious flags contained in the modern but not in the
# legacy profile (full Graviton 3 /proc/cpuinfo is "fp asimd evtstrm aes pmull sha1 sha2 crc32 atomics fphp asimdhp cpuid asimdrdm
# jscvt fcma lrcpc dcpop sha3 sm3 sm4 asimddp sha512 sve asimdfhm dit uscat ilrcpc flagm ssbs paca pacg dcpodp svei8mm svebf16 i8mm
# bf16 dgh rng")
execute_process(
COMMAND grep -P "^(?=.*atomic)(?=.*ssbs)" /proc/cpuinfo
OUTPUT_VARIABLE FLAGS)
if (NOT FLAGS)
MESSAGE(FATAL_ERROR "The build machine does not satisfy the minimum CPU requirements, try to run cmake with -DNO_ARMV81_OR_HIGHER=1")
endif()
endif()
elseif (ARCH_PPC64LE)
# By Default, build for power8 and up, allow building for power9 and up
# Note that gcc and clang have support for x86 SSE2 intrinsics when building for PowerPC
@ -102,6 +120,22 @@ elseif (ARCH_AMD64)
SET(ENABLE_AVX512_FOR_SPEC_OP 0)
endif()
# Same best-effort check for x86 as above for ARM.
if (OS_LINUX AND CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "amd64|x86_64" AND NOT NO_SSE3_OR_HIGHER)
# Test for flags in standard profile but not in NO_SSE3_OR_HIGHER profile.
# /proc/cpuid for Intel Xeon 8124: "fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse
# sse2 ss ht syscall nx pdpe1gb rdtscp lm constant_tsc arch_perfmon rep_good nopl xtopology nonstop_tsc cpuid aperfmperf
# tsc_known_freq pni pclmulqdq monitor ssse3 fma cx16 pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c
# rdrand hypervisor lahf_lm abm 3dnowprefetch invpcid_single pti fsgsbase tsc_adjust bmi1 hle avx2 smep bmi2 erms invpcid rtm mpx
# avx512f avx512dq rdseed adx smap clflushopt clwb avx512cd avx512bw avx512vl xsaveopt xsavec xgetbv1 xsaves ida arat pku ospke""
execute_process(
COMMAND grep -P "^(?=.*ssse3)(?=.*sse4_1)(?=.*sse4_2)" /proc/cpuinfo
OUTPUT_VARIABLE FLAGS)
if (NOT FLAGS)
MESSAGE(FATAL_ERROR "The build machine does not satisfy the minimum CPU requirements, try to run cmake with -DNO_SSE3_OR_HIGHER=1")
endif()
endif()
# ClickHouse can be cross-compiled (e.g. on an ARM host for x86) but it is also possible to build ClickHouse on x86 w/o AVX for x86 w/
# AVX. We only check that the compiler can emit certain SIMD instructions, we don't care if the host system is able to run the binary.
# Therefore, use check_cxx_source_compiles (= does the code compile+link?) instead of check_cxx_source_runs (= does the code

View File

@ -45,6 +45,7 @@ if (COMPILER_CLANG)
no_warning(weak-vtables)
no_warning(thread-safety-negative) # experimental flag, too many false positives
no_warning(enum-constexpr-conversion) # breaks magic-enum library in clang-16
no_warning(unsafe-buffer-usage) # too aggressive
# TODO Enable conversion, sign-conversion, double-promotion warnings.
elseif (COMPILER_GCC)
# Add compiler options only to c++ compiler

2
contrib/capnproto vendored

@ -1 +1 @@
Subproject commit e19cd661e49dd9022d3f920b69d843333b896451
Subproject commit dc8b50b999777bcb23c89bb5907c785c3f654441

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit 5022f30f3e092a54a7c101c335ce5e08769db366
Subproject commit e48aa13f67dc722511b5af33a32ba9b7748176b5

View File

@ -98,6 +98,16 @@ set(LLVM_ENABLE_BINDINGS 0 CACHE INTERNAL "")
set (LLVM_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/llvm")
set (LLVM_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/llvm-project/llvm")
# Since we always use toolchain files to generate hermatic builds, cmake will
# think it's a cross compilation, and LLVM will try to configure NATIVE LLVM
# targets with all tests enabled, which will slow down cmake configuration and
# compilation (You'll see Building native llvm-tblgen...). Let's disable the
# cross compiling indicator for now.
#
# TODO We should let cmake know whether it's indeed a cross compilation in the
# first place.
set (CMAKE_CROSSCOMPILING 0)
add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}")
set_directory_properties (PROPERTIES

View File

@ -94,6 +94,10 @@ if (ARCH_AMD64 OR ARCH_AARCH64)
add_compile_definitions(TUKLIB_FAST_UNALIGNED_ACCESS=1)
endif ()
if (ARCH_S390X)
add_compile_definitions(WORDS_BIGENDIAN)
endif ()
find_package(Threads REQUIRED)

View File

@ -30,25 +30,10 @@
# - zstd homepage : http://www.zstd.net/
# ################################################################
# Get library version based on information from input content (use regular exp)
function(GetLibraryVersion _content _outputVar1 _outputVar2 _outputVar3)
string(REGEX MATCHALL ".*define ZSTD_VERSION_MAJOR+.* ([0-9]+).*define ZSTD_VERSION_MINOR+.* ([0-9]+).*define ZSTD_VERSION_RELEASE+.* ([0-9]+)" VERSION_REGEX "${_content}")
SET(${_outputVar1} ${CMAKE_MATCH_1} PARENT_SCOPE)
SET(${_outputVar2} ${CMAKE_MATCH_2} PARENT_SCOPE)
SET(${_outputVar3} ${CMAKE_MATCH_3} PARENT_SCOPE)
endfunction()
# Define library directory, where sources and header files are located
SET(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib")
INCLUDE_DIRECTORIES(BEFORE ${LIBRARY_DIR} "${LIBRARY_DIR}/common")
# Read file content
FILE(READ "${LIBRARY_DIR}/zstd.h" HEADER_CONTENT)
# Parse version
GetLibraryVersion("${HEADER_CONTENT}" LIBVER_MAJOR LIBVER_MINOR LIBVER_RELEASE)
MESSAGE(STATUS "ZSTD VERSION ${LIBVER_MAJOR}.${LIBVER_MINOR}.${LIBVER_RELEASE}")
# cd contrib/zstd/lib
# find . -name '*.c' -or -name '*.S' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ "${LIBRARY_DIR}/"'
SET(Sources

View File

@ -43,7 +43,8 @@
"docker/test/stateful": {
"name": "clickhouse/stateful-test",
"dependent": [
"docker/test/stress"
"docker/test/stress",
"docker/test/upgrade"
]
},
"docker/test/unit": {
@ -54,6 +55,10 @@
"name": "clickhouse/stress-test",
"dependent": []
},
"docker/test/upgrade": {
"name": "clickhouse/upgrade-check",
"dependent": []
},
"docker/test/codebrowser": {
"name": "clickhouse/codebrowser",
"dependent": []

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.1.3.5"
ARG VERSION="23.2.1.2537"
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.1.3.5"
ARG VERSION="23.2.1.2537"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -1,4 +1,4 @@
FROM ubuntu:20.04
FROM ubuntu:22.04
# see https://github.com/moby/moby/issues/4032#issuecomment-192327844
ARG DEBIAN_FRONTEND=noninteractive
@ -9,19 +9,20 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
&& groupadd -r clickhouse --gid=101 \
&& useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \
&& apt-get update \
&& apt-get upgrade -yq \
&& apt-get install --yes --no-install-recommends \
apt-transport-https \
ca-certificates \
dirmngr \
gnupg \
locales \
gnupg2 \
wget \
locales \
tzdata \
&& apt-get clean
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="23.1.3.5"
ARG VERSION="23.2.1.2537"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image
@ -80,15 +81,8 @@ RUN arch=${TARGETARCH:-amd64} \
&& mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \
&& chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client
# Remove as much of Ubuntu as possible.
# ClickHouse does not need Ubuntu. It can run on top of Linux kernel without any OS distribution.
# ClickHouse does not need Docker at all. ClickHouse is above all that.
# It does not care about Ubuntu, Docker, or other cruft and you should neither.
# The fact that this Docker image is based on Ubuntu is just a misconception.
# Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all.
# ClickHouse does not care when you report false vulnerabilities by running some Docker scanners.
RUN apt-get remove --purge -y libksba8 && apt-get autoremove -y
RUN apt-get autoremove --purge -yq libksba8 && \
apt-get autoremove -yq
# we need to allow "others" access to clickhouse folder, because docker container
# can be started with arbitrary uid (openshift usecase)

View File

@ -1,5 +1,10 @@
# docker build -t clickhouse/performance-comparison .
FROM ubuntu:20.04
# Using ubuntu:22.04 over 20.04 as all other images, since:
# a) ubuntu 20.04 has too old parallel, and does not support --memsuspend
# b) anyway for perf tests it should not be important (backward compatiblity
# with older ubuntu had been checked lots of times in various tests)
FROM ubuntu:22.04
# ARG for quick switch to a given ubuntu mirror
ARG apt_archive="http://archive.ubuntu.com"

View File

@ -537,9 +537,20 @@ unset IFS
# all nodes.
numactl --show
numactl --cpunodebind=all --membind=all numactl --show
# Use less jobs to avoid OOM. Some queries can consume 8+ GB of memory.
jobs_count=$(($(grep -c ^processor /proc/cpuinfo) / 4))
numactl --cpunodebind=all --membind=all parallel --jobs $jobs_count --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log
# Notes for parallel:
#
# Some queries can consume 8+ GB of memory, so it worth to limit amount of jobs
# that can be run in parallel.
#
# --memfree:
#
# will kill jobs, which is not good (and retried until --retries exceeded)
#
# --memsuspend:
#
# If the available memory falls below 2 * size, GNU parallel will suspend some of the running jobs.
numactl --cpunodebind=all --membind=all parallel -v --joblog analyze/parallel-log.txt --memsuspend 15G --null < analyze/commands.txt 2>> analyze/errors.log
clickhouse-local --query "
-- Join the metric names back to the metric statistics we've calculated, and make

View File

@ -1,4 +1,4 @@
# rebuild in #33610
# rebuild in #47031
# docker build -t clickhouse/stateful-test .
ARG FROM_TAG=latest
FROM clickhouse/stateless-test:$FROM_TAG

View File

@ -21,10 +21,9 @@ RUN apt-get update -y \
openssl \
netcat-openbsd \
telnet \
llvm-9 \
brotli
brotli \
&& apt-get clean
COPY ./stress /stress
COPY run.sh /
ENV DATASETS="hits visits"

View File

@ -8,229 +8,13 @@ dmesg --clear
set -x
# core.COMM.PID-TID
sysctl kernel.core_pattern='core.%e.%p-%P'
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
OK="\tOK\t\\N\t"
FAIL="\tFAIL\t\\N\t"
FAILURE_CONTEXT_LINES=50
FAILURE_CONTEXT_MAX_LINE_WIDTH=400
function escaped()
{
# That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language.
# Also limit lines width just in case (too long lines are not really useful usually)
clickhouse local -S 's String' --input-format=LineAsString -q "select substr(s, 1, $FAILURE_CONTEXT_MAX_LINE_WIDTH)
from table format CustomSeparated settings format_custom_row_after_delimiter='\\\\\\\\n'"
}
function head_escaped()
{
head -n $FAILURE_CONTEXT_LINES $1 | escaped
}
function unts()
{
grep -Po "[0-9][0-9]:[0-9][0-9] \K.*"
}
function trim_server_logs()
{
head -n $FAILURE_CONTEXT_LINES "/test_output/$1" | grep -Eo " \[ [0-9]+ \] \{.*" | escaped
}
function install_packages()
{
dpkg -i $1/clickhouse-common-static_*.deb
dpkg -i $1/clickhouse-common-static-dbg_*.deb
dpkg -i $1/clickhouse-server_*.deb
dpkg -i $1/clickhouse-client_*.deb
}
function configure()
{
# install test configs
export USE_DATABASE_ORDINARY=1
export EXPORT_S3_STORAGE_POLICIES=1
/usr/share/clickhouse-test/config/install.sh
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -s /usr/share/clickhouse-test/ci/download_release_packages.py /usr/bin/download_release_packages
ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag
# avoid too slow startup
sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \
| sed "s|<snapshot_distance>100000</snapshot_distance>|<snapshot_distance>10000</snapshot_distance>|" \
> /etc/clickhouse-server/config.d/keeper_port.xml.tmp
sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml
sudo chown clickhouse /etc/clickhouse-server/config.d/keeper_port.xml
sudo chgrp clickhouse /etc/clickhouse-server/config.d/keeper_port.xml
# for clickhouse-server (via service)
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
# for clickhouse-client
export ASAN_OPTIONS='malloc_context_size=10 allocator_release_to_os_interval_ms=10000'
# since we run clickhouse from root
sudo chown root: /var/lib/clickhouse
# Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM).
echo "<clickhouse><asynchronous_metrics_update_period_s>1</asynchronous_metrics_update_period_s></clickhouse>" \
> /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml
local total_mem
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
total_mem=$(( total_mem*1024 )) # bytes
# Set maximum memory usage as half of total memory (less chance of OOM).
#
# But not via max_server_memory_usage but via max_memory_usage_for_user,
# so that we can override this setting and execute service queries, like:
# - hung check
# - show/drop database
# - ...
#
# So max_memory_usage_for_user will be a soft limit, and
# max_server_memory_usage will be hard limit, and queries that should be
# executed regardless memory limits will use max_memory_usage_for_user=0,
# instead of relying on max_untracked_memory
max_server_memory_usage_to_ram_ratio=0.5
echo "Setting max_server_memory_usage_to_ram_ratio to ${max_server_memory_usage_to_ram_ratio}"
cat > /etc/clickhouse-server/config.d/max_server_memory_usage.xml <<EOL
<clickhouse>
<max_server_memory_usage_to_ram_ratio>${max_server_memory_usage_to_ram_ratio}</max_server_memory_usage_to_ram_ratio>
</clickhouse>
EOL
local max_users_mem
max_users_mem=$((total_mem*30/100)) # 30%
echo "Setting max_memory_usage_for_user=$max_users_mem and max_memory_usage for queries to 10G"
cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml <<EOL
<clickhouse>
<profiles>
<default>
<max_memory_usage>10G</max_memory_usage>
<max_memory_usage_for_user>${max_users_mem}</max_memory_usage_for_user>
</default>
</profiles>
</clickhouse>
EOL
cat > /etc/clickhouse-server/config.d/core.xml <<EOL
<clickhouse>
<core_dump>
<!-- 100GiB -->
<size_limit>107374182400</size_limit>
</core_dump>
<!-- NOTE: no need to configure core_path,
since clickhouse is not started as daemon (via clickhouse start)
-->
<core_path>$PWD</core_path>
</clickhouse>
EOL
# Let OOM killer terminate other processes before clickhouse-server:
cat > /etc/clickhouse-server/config.d/oom_score.xml <<EOL
<clickhouse>
<oom_score>-1000</oom_score>
</clickhouse>
EOL
# Analyzer is not yet ready for testing
cat > /etc/clickhouse-server/users.d/no_analyzer.xml <<EOL
<clickhouse>
<profiles>
<default>
<constraints>
<allow_experimental_analyzer>
<readonly/>
</allow_experimental_analyzer>
</constraints>
</default>
</profiles>
</clickhouse>
EOL
}
function stop()
{
local max_tries="${1:-90}"
local pid
# Preserve the pid, since the server can hung after the PID will be deleted.
pid="$(cat /var/run/clickhouse-server/clickhouse-server.pid)"
clickhouse stop --max-tries "$max_tries" --do-not-kill && return
# We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces.
echo -e "Possible deadlock on shutdown (see gdb.log)$FAIL" >> /test_output/test_results.tsv
kill -TERM "$(pidof gdb)" ||:
sleep 5
echo "thread apply all backtrace (on stop)" >> /test_output/gdb.log
timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$pid" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log
clickhouse stop --force
}
function start()
{
counter=0
until clickhouse-client --query "SELECT 1"
do
if [ "$counter" -gt ${1:-120} ]
then
echo "Cannot start clickhouse-server"
rg --text "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt ||:
echo -e "Cannot start clickhouse-server$FAIL$(trim_server_logs application_errors.txt)" >> /test_output/test_results.tsv
cat /var/log/clickhouse-server/stdout.log
tail -n100 /var/log/clickhouse-server/stderr.log
tail -n100000 /var/log/clickhouse-server/clickhouse-server.log | rg -F -v -e '<Warning> RaftInstance:' -e '<Information> RaftInstance' | tail -n100
break
fi
# use root to match with current uid
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>>/var/log/clickhouse-server/stderr.log
sleep 0.5
counter=$((counter + 1))
done
# Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog
# and clickhouse-server can do fork-exec, for example, to run some bridge.
# Do not set nostop noprint for all signals, because some it may cause gdb to hang,
# explicitly ignore non-fatal signals that are used by server.
# Number of SIGRTMIN can be determined only in runtime.
RTMIN=$(kill -l SIGRTMIN)
echo "
set follow-fork-mode parent
handle SIGHUP nostop noprint pass
handle SIGINT nostop noprint pass
handle SIGQUIT nostop noprint pass
handle SIGPIPE nostop noprint pass
handle SIGTERM nostop noprint pass
handle SIGUSR1 nostop noprint pass
handle SIGUSR2 nostop noprint pass
handle SIG$RTMIN nostop noprint pass
info signals
continue
backtrace full
thread apply all backtrace full
info registers
disassemble /s
up
disassemble /s
up
disassemble /s
p \"done\"
detach
quit
" > script.gdb
# FIXME Hung check may work incorrectly because of attached gdb
# 1. False positives are possible
# 2. We cannot attach another gdb to get stacktraces if some queries hung
gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log &
sleep 5
# gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s)
time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||:
}
# Stress tests and upgrade check uses similar code that was placed
# in a separate bash library. See tests/ci/stress_tests.lib
source /usr/share/clickhouse-test/ci/stress_tests.lib
install_packages package_folder
@ -396,7 +180,7 @@ sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_defau
start
./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
&& echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv
@ -413,316 +197,27 @@ unset "${!THREAD_@}"
start
clickhouse-client --query "SELECT 'Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \
|| (rg --text "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt \
&& echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(trim_server_logs application_errors.txt)" \
>> /test_output/test_results.tsv)
check_server_start
stop
[ -f /var/log/clickhouse-server/clickhouse-server.log ] || echo -e "Server log does not exist\tFAIL"
[ -f /var/log/clickhouse-server/stderr.log ] || echo -e "Stderr log does not exist\tFAIL"
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.final.log
# Grep logs for sanitizer asserts, crashes and other critical errors
check_logs_for_critical_errors
# Sanitizer asserts
rg -Fa "==================" /var/log/clickhouse-server/stderr.log | rg -v "in query:" >> /test_output/tmp
rg -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
rg -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \
&& echo -e "Sanitizer assert (in stderr.log)$FAIL$(head_escaped /test_output/tmp)" >> /test_output/test_results.tsv \
|| echo -e "No sanitizer asserts$OK" >> /test_output/test_results.tsv
rm -f /test_output/tmp
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
# OOM
rg -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \
&& echo -e "Signal 9 in clickhouse-server.log$FAIL" >> /test_output/test_results.tsv \
|| echo -e "No OOM messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Logical errors
rg -Fa "Code: 49. DB::Exception: " /var/log/clickhouse-server/clickhouse-server*.log > /test_output/logical_errors.txt \
&& echo -e "Logical error thrown (see clickhouse-server.log or logical_errors.txt)$FAIL$(head_escaped /test_output/logical_errors.txt)" >> /test_output/test_results.tsv \
|| echo -e "No logical errors$OK" >> /test_output/test_results.tsv
# Remove file logical_errors.txt if it's empty
[ -s /test_output/logical_errors.txt ] || rm /test_output/logical_errors.txt
# No such key errors
rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log > /test_output/no_such_key_errors.txt \
&& echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \
|| echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv
# Remove file no_such_key_errors.txt if it's empty
[ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt
# Crash
rg -Fa "########################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \
&& echo -e "Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \
|| echo -e "Not crashed$OK" >> /test_output/test_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
rg -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server*.log > /test_output/fatal_messages.txt \
&& echo -e "Fatal message in clickhouse-server.log (see fatal_messages.txt)$FAIL$(trim_server_logs fatal_messages.txt)" >> /test_output/test_results.tsv \
|| echo -e "No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Remove file fatal_messages.txt if it's empty
[ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt
rg -Fa "########################################" /test_output/* > /dev/null \
&& echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv
function get_gdb_log_context()
{
rg -A50 -Fa " received signal " /test_output/gdb.log | head_escaped
}
rg -Fa " received signal " /test_output/gdb.log > /dev/null \
&& echo -e "Found signal in gdb.log$FAIL$(get_gdb_log_context)" >> /test_output/test_results.tsv
if [ "$DISABLE_BC_CHECK" -ne "1" ]; then
echo -e "Backward compatibility check\n"
echo "Get previous release tag"
previous_release_tag=$(clickhouse-client --version | rg -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag)
echo $previous_release_tag
echo "Clone previous release repository"
git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository
echo "Download clickhouse-server from the previous release"
mkdir previous_release_package_folder
echo $previous_release_tag | download_release_packages && echo -e "Download script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Download script failed$FAIL" >> /test_output/test_results.tsv
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log
for table in query_log trace_log
do
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
done
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
# Check if we cloned previous release repository successfully
if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
then
echo -e "Backward compatibility check: Failed to clone previous release tests$FAIL" >> /test_output/test_results.tsv
elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
then
echo -e "Backward compatibility check: Failed to download previous release packages$FAIL" >> /test_output/test_results.tsv
else
echo -e "Successfully cloned previous release tests$OK" >> /test_output/test_results.tsv
echo -e "Successfully downloaded previous release packages$OK" >> /test_output/test_results.tsv
# Uninstall current packages
dpkg --remove clickhouse-client
dpkg --remove clickhouse-server
dpkg --remove clickhouse-common-static-dbg
dpkg --remove clickhouse-common-static
rm -rf /var/lib/clickhouse/*
# Make BC check more funny by forcing Ordinary engine for system database
mkdir /var/lib/clickhouse/metadata
echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql
# Install previous release packages
install_packages previous_release_package_folder
# Start server from previous release
# Previous version may not be ready for fault injections
export ZOOKEEPER_FAULT_INJECTION=0
configure
# Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..."
rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||:
rm -f /etc/clickhouse-server/users.d/marks.xml ||:
# Remove s3 related configs to avoid "there is no disk type `cache`"
rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:
rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||:
# Turn on after 22.12
rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||:
# it uses recently introduced settings which previous versions may not have
rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||:
# Turn on after 23.1
rm -f /etc/clickhouse-server/users.d/prefetch_settings.xml ||:
start
clickhouse-client --query="SELECT 'Server version: ', version()"
# Install new package before running stress test because we should use new
# clickhouse-client and new clickhouse-test.
#
# But we should leave old binary in /usr/bin/ and debug symbols in
# /usr/lib/debug/usr/bin (if any) for gdb and internal DWARF parser, so it
# will print sane stacktraces and also to avoid possible crashes.
#
# FIXME: those files can be extracted directly from debian package, but
# actually better solution will be to use different PATH instead of playing
# games with files from packages.
mv /usr/bin/clickhouse previous_release_package_folder/
mv /usr/lib/debug/usr/bin/clickhouse.debug previous_release_package_folder/
install_packages package_folder
mv /usr/bin/clickhouse package_folder/
mv /usr/lib/debug/usr/bin/clickhouse.debug package_folder/
mv previous_release_package_folder/clickhouse /usr/bin/
mv previous_release_package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
mkdir tmp_stress_output
./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" \
--backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
&& echo -e "Backward compatibility check: Test script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: Test script failed$FAIL" >> /test_output/test_results.tsv
rm -rf tmp_stress_output
# We experienced deadlocks in this command in very rare cases. Let's debug it:
timeout 10m clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" ||
(
echo "thread apply all backtrace (on select tables count)" >> /test_output/gdb.log
timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log
clickhouse stop --force
)
# Use bigger timeout for previous version
stop 300
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log
# Start new server
mv package_folder/clickhouse /usr/bin/
mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
# Disable fault injections on start (we don't test them here, and it can lead to tons of requests in case of huge number of tables).
export ZOOKEEPER_FAULT_INJECTION=0
configure
start 500
clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \
|| (rg --text "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt \
&& echo -e "Backward compatibility check: Server failed to start$FAIL$(trim_server_logs bc_check_application_errors.txt)" >> /test_output/test_results.tsv)
clickhouse-client --query="SELECT 'Server version: ', version()"
# Let the server run for a while before checking log.
sleep 60
stop
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.dirty.log
# Error messages (we should ignore some errors)
# FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64")
# FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server.
# Let's just ignore all errors from queries ("} <Error> TCPHandler: Code:", "} <Error> executeQuery: Code:")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility
echo "Check for Error messages in server log:"
rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Code: 236. DB::Exception: Cancelled mutating parts" \
-e "REPLICA_IS_ALREADY_ACTIVE" \
-e "REPLICA_ALREADY_EXISTS" \
-e "ALL_REPLICAS_LOST" \
-e "DDLWorker: Cannot parse DDL task query" \
-e "RaftInstance: failed to accept a rpc connection due to error 125" \
-e "UNKNOWN_DATABASE" \
-e "NETWORK_ERROR" \
-e "UNKNOWN_TABLE" \
-e "ZooKeeperClient" \
-e "KEEPER_EXCEPTION" \
-e "DirectoryMonitor" \
-e "TABLE_IS_READ_ONLY" \
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \
-e "NETLINK_ERROR" \
-e "Renaming unexpected part" \
-e "PART_IS_TEMPORARILY_LOCKED" \
-e "and a merge is impossible: we didn't find" \
-e "found in queue and some source parts for it was lost" \
-e "is lost forever." \
-e "Unknown index: idx." \
-e "Cannot parse string 'Hello' as UInt64" \
-e "} <Error> TCPHandler: Code:" \
-e "} <Error> executeQuery: Code:" \
-e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \
-e "[Queue = DB::DynamicRuntimeQueue]: Code: 235. DB::Exception: Part" \
-e "The set of parts restored in place of" \
-e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
-e "Code: 269. DB::Exception: Destination table is myself" \
-e "Coordination::Exception: Connection loss" \
-e "MutateFromLogEntryTask" \
-e "No connection to ZooKeeper, cannot get shared table ID" \
-e "Session expired" \
-e "TOO_MANY_PARTS" \
-e "Container already exists" \
/var/log/clickhouse-server/clickhouse-server.backward.dirty.log | rg -Fa "<Error>" > /test_output/bc_check_error_messages.txt \
&& echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(trim_server_logs bc_check_error_messages.txt)" \
>> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: No Error messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Remove file bc_check_error_messages.txt if it's empty
[ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt
# Sanitizer asserts
rg -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
rg -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
rg -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \
&& echo -e "Backward compatibility check: Sanitizer assert (in stderr.log)$FAIL$(head_escaped /test_output/tmp)" >> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: No sanitizer asserts$OK" >> /test_output/test_results.tsv
rm -f /test_output/tmp
# OOM
rg -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
&& echo -e "Backward compatibility check: Signal 9 in clickhouse-server.log$FAIL" >> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: No OOM messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Logical errors
echo "Check for Logical errors in server log:"
rg -Fa -A20 "Code: 49. DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \
&& echo -e "Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)$FAIL$(trim_server_logs bc_check_logical_errors.txt)" \
>> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: No logical errors$OK" >> /test_output/test_results.tsv
# Remove file bc_check_logical_errors.txt if it's empty
[ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt
# Crash
rg -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \
&& echo -e "Backward compatibility check: Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: Not crashed$OK" >> /test_output/test_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
echo "Check for Fatal message in server log:"
rg -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \
&& echo -e "Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)$FAIL$(trim_server_logs bc_check_fatal_messages.txt)" \
>> /test_output/test_results.tsv \
|| echo -e "Backward compatibility check: No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Remove file bc_check_fatal_messages.txt if it's empty
[ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt
tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||:
for table in query_log trace_log
do
clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" \
| zstd --threads=0 > /test_output/$table.backward.tsv.zst ||:
done
fi
fi
dmesg -T > /test_output/dmesg.log
# OOM in dmesg -- those are real
grep -q -F -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE' /test_output/dmesg.log \
&& echo -e "OOM in dmesg$FAIL$(head_escaped /test_output/dmesg.log)" >> /test_output/test_results.tsv \
|| echo -e "No OOM in dmesg$OK" >> /test_output/test_results.tsv
collect_query_and_trace_logs
mv /var/log/clickhouse-server/stderr.log /test_output/
# Write check result into check_status.tsv
# Try to choose most specific error for the whole check status
clickhouse-local --structure "test String, res String, time Nullable(Float32), desc String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by
(test like 'Backward compatibility check%'), -- BC check goes last
(test like '%Sanitizer%') DESC,
(test like '%Killed by signal%') DESC,
(test like '%gdb.log%') DESC,
@ -732,14 +227,8 @@ clickhouse-local --structure "test String, res String, time Nullable(Float32), d
(test like '%OOM%') DESC,
(test like '%Signal 9%') DESC,
(test like '%Fatal message%') DESC,
(test like '%Error message%') DESC,
(test like '%previous release%') DESC,
rowNumberInAllBlocks()
LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv
# Core dumps
find . -type f -maxdepth 1 -name 'core.*' | while read core; do
zstd --threads=0 $core
mv $core.zst /test_output/
done
collect_core_dumps

View File

@ -0,0 +1,31 @@
# rebuild in #33610
# docker build -t clickhouse/upgrade-check .
ARG FROM_TAG=latest
FROM clickhouse/stateful-test:$FROM_TAG
RUN apt-get update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
bash \
tzdata \
fakeroot \
debhelper \
parallel \
expect \
python3 \
python3-lxml \
python3-termcolor \
python3-requests \
curl \
sudo \
openssl \
netcat-openbsd \
telnet \
brotli \
&& apt-get clean
COPY run.sh /
ENV EXPORT_S3_STORAGE_POLICIES=1
CMD ["/bin/bash", "/run.sh"]

201
docker/test/upgrade/run.sh Normal file
View File

@ -0,0 +1,201 @@
#!/bin/bash
# shellcheck disable=SC2094
# shellcheck disable=SC2086
# shellcheck disable=SC2024
# Avoid overlaps with previous runs
dmesg --clear
set -x
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -s /usr/share/clickhouse-test/ci/download_release_packages.py /usr/bin/download_release_packages
ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag
# Stress tests and upgrade check uses similar code that was placed
# in a separate bash library. See tests/ci/stress_tests.lib
source /usr/share/clickhouse-test/ci/stress_tests.lib
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
./setup_minio.sh stateless # to have a proper environment
echo "Get previous release tag"
previous_release_tag=$(dpkg --info package_folder/clickhouse-client*.deb | grep "Version: " | awk '{print $2}' | cut -f1 -d'+' | get_previous_release_tag)
echo $previous_release_tag
echo "Clone previous release repository"
git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository
echo "Download clickhouse-server from the previous release"
mkdir previous_release_package_folder
echo $previous_release_tag | download_release_packages && echo -e "Download script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Download script failed$FAIL" >> /test_output/test_results.tsv
# Check if we cloned previous release repository successfully
if ! [ "$(ls -A previous_release_repository/tests/queries)" ]
then
echo -e 'failure\tFailed to clone previous release tests' > /test_output/check_status.tsv
exit
elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
then
echo -e 'failure\tFailed to download previous release packages' > /test_output/check_status.tsv
exit
fi
echo -e "Successfully cloned previous release tests$OK" >> /test_output/test_results.tsv
echo -e "Successfully downloaded previous release packages$OK" >> /test_output/test_results.tsv
# Make upgrade check more funny by forcing Ordinary engine for system database
mkdir /var/lib/clickhouse/metadata
echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql
# Install previous release packages
install_packages previous_release_package_folder
# Start server from previous release
# Let's enable S3 storage by default
export USE_S3_STORAGE_FOR_MERGE_TREE=1
# Previous version may not be ready for fault injections
export ZOOKEEPER_FAULT_INJECTION=0
configure
# But we still need default disk because some tables loaded only into it
sudo cat /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml \
| sed "s|<main><disk>s3</disk></main>|<main><disk>s3</disk></main><default><disk>default</disk></default>|" \
> /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml.tmp mv /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml.tmp /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml
sudo chown clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml
sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml
start
clickhouse-client --query="SELECT 'Server version: ', version()"
mkdir tmp_stress_output
stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --upgrade-check --output-folder tmp_stress_output --global-time-limit=1200 \
&& echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv
rm -rf tmp_stress_output
# We experienced deadlocks in this command in very rare cases. Let's debug it:
timeout 10m clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" ||
(
echo "thread apply all backtrace (on select tables count)" >> /test_output/gdb.log
timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log
clickhouse stop --force
)
# Use bigger timeout for previous version and disable additional hang check
stop 300 false
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.stress.log
# Install and start new server
install_packages package_folder
# Disable fault injections on start (we don't test them here, and it can lead to tons of requests in case of huge number of tables).
export ZOOKEEPER_FAULT_INJECTION=0
configure
start 500
clickhouse-client --query "SELECT 'Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \
|| (rg --text "<Error>.*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt \
&& echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(trim_server_logs application_errors.txt)" \
>> /test_output/test_results.tsv)
# Remove file application_errors.txt if it's empty
[ -s /test_output/application_errors.txt ] || rm /test_output/application_errors.txt
clickhouse-client --query="SELECT 'Server version: ', version()"
# Let the server run for a while before checking log.
sleep 60
stop
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.upgrade.log
# Error messages (we should ignore some errors)
# FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64")
# FIXME Not sure if it's expected, but some tests from stress test may not be finished yet when we restarting server.
# Let's just ignore all errors from queries ("} <Error> TCPHandler: Code:", "} <Error> executeQuery: Code:")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'")
# FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility
echo "Check for Error messages in server log:"
rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Code: 236. DB::Exception: Cancelled mutating parts" \
-e "REPLICA_IS_ALREADY_ACTIVE" \
-e "REPLICA_ALREADY_EXISTS" \
-e "ALL_REPLICAS_LOST" \
-e "DDLWorker: Cannot parse DDL task query" \
-e "RaftInstance: failed to accept a rpc connection due to error 125" \
-e "UNKNOWN_DATABASE" \
-e "NETWORK_ERROR" \
-e "UNKNOWN_TABLE" \
-e "ZooKeeperClient" \
-e "KEEPER_EXCEPTION" \
-e "DirectoryMonitor" \
-e "TABLE_IS_READ_ONLY" \
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \
-e "NETLINK_ERROR" \
-e "Renaming unexpected part" \
-e "PART_IS_TEMPORARILY_LOCKED" \
-e "and a merge is impossible: we didn't find" \
-e "found in queue and some source parts for it was lost" \
-e "is lost forever." \
-e "Unknown index: idx." \
-e "Cannot parse string 'Hello' as UInt64" \
-e "} <Error> TCPHandler: Code:" \
-e "} <Error> executeQuery: Code:" \
-e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \
-e "The set of parts restored in place of" \
-e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \
-e "Code: 269. DB::Exception: Destination table is myself" \
-e "Coordination::Exception: Connection loss" \
-e "MutateFromLogEntryTask" \
-e "No connection to ZooKeeper, cannot get shared table ID" \
-e "Session expired" \
-e "TOO_MANY_PARTS" \
-e "Authentication failed" \
-e "Cannot flush" \
-e "Container already exists" \
/var/log/clickhouse-server/clickhouse-server.upgrade.log | zgrep -Fa "<Error>" > /test_output/upgrade_error_messages.txt \
&& echo -e "Error message in clickhouse-server.log (see upgrade_error_messages.txt)$FAIL$(head_escaped /test_output/bc_check_error_messages.txt)" \
>> /test_output/test_results.tsv \
|| echo -e "No Error messages after server upgrade$OK" >> /test_output/test_results.tsv
# Remove file upgrade_error_messages.txt if it's empty
[ -s /test_output/upgrade_error_messages.txt ] || rm /test_output/upgrade_error_messages.txt
# Grep logs for sanitizer asserts, crashes and other critical errors
check_logs_for_critical_errors
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
collect_query_and_trace_logs
check_oom_in_dmesg
mv /var/log/clickhouse-server/stderr.log /test_output/
# Write check result into check_status.tsv
# Try to choose most specific error for the whole check status
clickhouse-local --structure "test String, res String, time Nullable(Float32), desc String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by
(test like '%Sanitizer%') DESC,
(test like '%Killed by signal%') DESC,
(test like '%gdb.log%') DESC,
(test ilike '%possible deadlock%') DESC,
(test like '%start%') DESC,
(test like '%dmesg%') DESC,
(test like '%OOM%') DESC,
(test like '%Signal 9%') DESC,
(test like '%Fatal message%') DESC,
(test like '%Error message%') DESC,
(test like '%previous release%') DESC,
rowNumberInAllBlocks()
LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv
collect_core_dumps

View File

@ -60,12 +60,21 @@ fi
clickhouse_download_filename_prefix="clickhouse"
clickhouse="$clickhouse_download_filename_prefix"
i=0
while [ -f "$clickhouse" ]
do
clickhouse="${clickhouse_download_filename_prefix}.${i}"
i=$(($i+1))
done
if [ -f "$clickhouse" ]
then
read -p "ClickHouse binary ${clickhouse} already exists. Overwrite? [y/N] " answer
if [ "$answer" = "y" -o "$answer" = "Y" ]
then
rm -f "$clickhouse"
else
i=0
while [ -f "$clickhouse" ]
do
clickhouse="${clickhouse_download_filename_prefix}.${i}"
i=$(($i+1))
done
fi
fi
URL="https://builds.clickhouse.com/master/${DIR}/clickhouse"
echo
@ -76,9 +85,9 @@ echo
echo "Successfully downloaded the ClickHouse binary, you can run it as:
./${clickhouse}"
if [ "${OS}" = "Linux" ]
then
echo
echo "You can also install it:
sudo ./${clickhouse} install"
fi
#if [ "${OS}" = "Linux" ]
#then
#echo
#echo "You can also install it:
#sudo ./${clickhouse} install"
#fi

View File

@ -0,0 +1,37 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v22.11.6.44-stable (73ddf91298f) FIXME as compared to v22.11.5.15-stable (d763e5a9239)
#### Performance Improvement
* Backported in [#45703](https://github.com/ClickHouse/ClickHouse/issues/45703): Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)).
* Backported in [#46376](https://github.com/ClickHouse/ClickHouse/issues/46376): Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
#### Build/Testing/Packaging Improvement
* Backported in [#45977](https://github.com/ClickHouse/ClickHouse/issues/45977): Fix zookeeper downloading, update the version, and optimize the image size. [#44853](https://github.com/ClickHouse/ClickHouse/pull/44853) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#46114](https://github.com/ClickHouse/ClickHouse/issues/46114): Remove the dependency on the `adduser` tool from the packages, because we don't use it. This fixes [#44934](https://github.com/ClickHouse/ClickHouse/issues/44934). [#45011](https://github.com/ClickHouse/ClickHouse/pull/45011) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Backported in [#46483](https://github.com/ClickHouse/ClickHouse/issues/46483): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#46507](https://github.com/ClickHouse/ClickHouse/issues/46507): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Backported in [#45903](https://github.com/ClickHouse/ClickHouse/issues/45903): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Backported in [#46239](https://github.com/ClickHouse/ClickHouse/issues/46239): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set<std::string>` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)).
* Backported in [#46216](https://github.com/ClickHouse/ClickHouse/issues/46216): Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)).
* Backported in [#46444](https://github.com/ClickHouse/ClickHouse/issues/46444): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)).
* Backported in [#46676](https://github.com/ClickHouse/ClickHouse/issues/46676): Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -0,0 +1,17 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v22.3.19.6-lts (467e0a7bd77) FIXME as compared to v22.3.18.37-lts (fe512717551)
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Backported in [#46440](https://github.com/ClickHouse/ClickHouse/issues/46440): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -0,0 +1,473 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.2.1.2537-stable (52bf836e03a) FIXME as compared to v23.1.1.3077-stable (dcaac477025)
#### Backward Incompatible Change
* Extend function "toDayOfWeek()" (alias: "DAYOFWEEK") with a mode argument that encodes whether the week starts on Monday or Sunday and whether counting starts at 0 or 1. For consistency with other date time functions, the mode argument was inserted between the time and the time zone arguments. This breaks existing usage of the (previously undocumented) 2-argument syntax "toDayOfWeek(time, time_zone)". A fix is to rewrite the function into "toDayOfWeek(time, 0, time_zone)". [#45233](https://github.com/ClickHouse/ClickHouse/pull/45233) ([Robert Schulze](https://github.com/rschu1ze)).
* Rename setting `max_query_cache_size` to `filesystem_cache_max_download_size`. [#45614](https://github.com/ClickHouse/ClickHouse/pull/45614) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix applying settings for FORMAT on the client. [#46003](https://github.com/ClickHouse/ClickHouse/pull/46003) ([Azat Khuzhin](https://github.com/azat)).
* Default user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. by default, default user will not longer be able to do grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). Fix integration tests. [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Slightly improve performance of `countDigits` on realistic datasets. This closed [#44518](https://github.com/ClickHouse/ClickHouse/issues/44518). In previous versions, `countDigits(0)` returned `0`; now it returns `1`, which is more correct, and follows the existing documentation. [#46187](https://github.com/ClickHouse/ClickHouse/pull/46187) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### New Feature
* Expose ProfileEvents counters in system.part_log. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)).
* Enrichment of the existing ReplacingMergeTree engine to allow duplicates insertion. It leverages the power of both ReplacingMergeTree and CollapsingMergeTree in one mergeTree engine. Deleted data are not returned when queried, but not removed from disk neither. [#41005](https://github.com/ClickHouse/ClickHouse/pull/41005) ([youennL-cs](https://github.com/youennL-cs)).
* Add `generateULID()` function. Closes [#36536](https://github.com/ClickHouse/ClickHouse/issues/36536). [#44662](https://github.com/ClickHouse/ClickHouse/pull/44662) ([Nikolay Degterinsky](https://github.com/evillique)).
* Add `corrMatrix` Aggregatefunction, calculating each two columns. In addition, since Aggregatefunctions `covarSamp` and `covarPop` are similar to `corr`, I add `covarSampMatrix`, `covarPopMatrix` by the way. @alexey-milovidov closes [#44587](https://github.com/ClickHouse/ClickHouse/issues/44587). [#44680](https://github.com/ClickHouse/ClickHouse/pull/44680) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Rewrite aggregate functions with if expression as argument when logically equivalent. For example, avg(if(cond, col, null)) can be rewritten to avgIf(cond, col). It is helpful in performance. [#44730](https://github.com/ClickHouse/ClickHouse/pull/44730) ([李扬](https://github.com/taiyang-li)).
* Introduce arrayShuffle function for random array permutations. [#45271](https://github.com/ClickHouse/ClickHouse/pull/45271) ([Joanna Hulboj](https://github.com/jh0x)).
* Support types FIXED_SIZE_BINARY type in Arrow, FIXED_LENGTH_BYTE_ARRAY in Parquet and match them to FixedString. Add settings `output_format_parquet_fixed_string_as_fixed_byte_array/output_format_arrow_fixed_string_as_fixed_byte_array` to control default output type for FixedString. Closes [#45326](https://github.com/ClickHouse/ClickHouse/issues/45326). [#45340](https://github.com/ClickHouse/ClickHouse/pull/45340) ([Kruglov Pavel](https://github.com/Avogar)).
* Add `StorageIceberg` and table function `iceberg` to access iceberg table store on S3. [#45384](https://github.com/ClickHouse/ClickHouse/pull/45384) ([flynn](https://github.com/ucasfl)).
* Add a new column `last_exception_time` to system.replication_queue. [#45457](https://github.com/ClickHouse/ClickHouse/pull/45457) ([Frank Chen](https://github.com/FrankChen021)).
* Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* add joda format support for 'x','w','S'.Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)).
* ... Support window function `ntile`. ``` insert into test_data values(1,2), (1,3), (1,4), (2,5),(2,6); select a, b, ntile(2) over (partition by a order by b rows between unbounded preceding and unbounded following ) from test_data;. [#46256](https://github.com/ClickHouse/ClickHouse/pull/46256) ([lgbo](https://github.com/lgbo-ustc)).
* Added arrayPartialSort and arrayPartialReverseSort functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)).
* The new http parameter `client_protocol_version` allows setting a client protocol version for HTTP responses using the Native format. [#40397](https://github.com/ClickHouse/ClickHouse/issues/40397). [#46360](https://github.com/ClickHouse/ClickHouse/pull/46360) ([Geoff Genz](https://github.com/genzgd)).
* Add new function regexpExtract, like spark function REGEXP_EXTRACT. [#46469](https://github.com/ClickHouse/ClickHouse/pull/46469) ([李扬](https://github.com/taiyang-li)).
* Author: [taiyang-li](https://github.com/taiyang-li) Add new function regexpExtract, like spark function REGEXP_EXTRACT. [#46529](https://github.com/ClickHouse/ClickHouse/pull/46529) ([Alexander Gololobov](https://github.com/davenger)).
* Add new function JSONArrayLength, which returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid. [#46631](https://github.com/ClickHouse/ClickHouse/pull/46631) ([李扬](https://github.com/taiyang-li)).
#### Performance Improvement
* Improve lower/upper function performance with avx512 instructions. [#37894](https://github.com/ClickHouse/ClickHouse/pull/37894) ([yaqi-zhao](https://github.com/yaqi-zhao)).
* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#38456](https://github.com/ClickHouse/ClickHouse/pull/38456) ([Saulius Valatka](https://github.com/sauliusvl)).
* Remove the limitation that on systems with >=32 cores and SMT disabled ClickHouse uses only half of the cores. [#44973](https://github.com/ClickHouse/ClickHouse/pull/44973) ([Robert Schulze](https://github.com/rschu1ze)).
* Improve performance of function multiIf by columnar executing, speed up by 2.3x. [#45296](https://github.com/ClickHouse/ClickHouse/pull/45296) ([李扬](https://github.com/taiyang-li)).
* An option added to aggregate partitions independently if table partition key and group by key are compatible. Controlled by the setting `allow_aggregate_partitions_independently`. Disabled by default because of limited applicability (please refer to the docs). [#45364](https://github.com/ClickHouse/ClickHouse/pull/45364) ([Nikita Taranov](https://github.com/nickitat)).
* Add fastpath for function position when needle is empty. [#45382](https://github.com/ClickHouse/ClickHouse/pull/45382) ([李扬](https://github.com/taiyang-li)).
* Enable `query_plan_remove_redundant_sorting` optimization by default. Optimization implemented in [#45420](https://github.com/ClickHouse/ClickHouse/issues/45420). [#45567](https://github.com/ClickHouse/ClickHouse/pull/45567) ([Igor Nikonov](https://github.com/devcrafter)).
* Increased HTTP Transfer Encoding chunk size to improve performance of large queries using the HTTP interface. [#45593](https://github.com/ClickHouse/ClickHouse/pull/45593) ([Geoff Genz](https://github.com/genzgd)).
* Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)).
* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#45681](https://github.com/ClickHouse/ClickHouse/pull/45681) ([Anton Popov](https://github.com/CurtizJ)).
* Optimize Parquet reader by using batch reader. [#45878](https://github.com/ClickHouse/ClickHouse/pull/45878) ([LiuNeng](https://github.com/liuneng1994)).
* Improve performance of ColumnArray::filter for big int and decimal. [#45949](https://github.com/ClickHouse/ClickHouse/pull/45949) ([李扬](https://github.com/taiyang-li)).
* This change could effectively reduce the overhead of obtaining the filter from ColumnNullable(UInt8) and improve the overall query performance. To evaluate the impact of this change, we adopted TPC-H benchmark but revised the column types from non-nullable to nullable, and we measured the QPS of its queries as the performance indicator. [#45962](https://github.com/ClickHouse/ClickHouse/pull/45962) ([Zhiguo Zhou](https://github.com/ZhiguoZh)).
* Make the `_part` and `_partition_id` virtual column be `LowCardinality(String)` type. Closes [#45964](https://github.com/ClickHouse/ClickHouse/issues/45964). [#45975](https://github.com/ClickHouse/ClickHouse/pull/45975) ([flynn](https://github.com/ucasfl)).
* Improve the performance of Decimal conversion when the scale does not change. [#46095](https://github.com/ClickHouse/ClickHouse/pull/46095) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The introduced logic works if PREWHERE condition is a conjunction of multiple conditions (cond1 AND cond2 AND ... ). It groups those conditions that require reading the same columns into steps. After each step the corresponding part of the full condition is computed and the result rows might be filtered. This allows to read fewer rows in the next steps thus saving IO bandwidth and doing less computation. This logic is disabled by default for now. It will be enabled by default in one of the future releases once it is known to not have any regressions, so it is highly encouraged to be used for testing. It can be controlled by 2 settings: "enable_multiple_prewhere_read_steps" and "move_all_conditions_to_prewhere". [#46140](https://github.com/ClickHouse/ClickHouse/pull/46140) ([Alexander Gololobov](https://github.com/davenger)).
* Allow to increase prefetching for read data. [#46168](https://github.com/ClickHouse/ClickHouse/pull/46168) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Rewrite arrayExists(x -> x = 1, arr) -> has(arr, 1), which improve performance by 1.34x. [#46188](https://github.com/ClickHouse/ClickHouse/pull/46188) ([李扬](https://github.com/taiyang-li)).
* Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Update zstd to v1.5.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge/mutation is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#46280](https://github.com/ClickHouse/ClickHouse/pull/46280) ([Raúl Marín](https://github.com/Algunenano)).
* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#46282](https://github.com/ClickHouse/ClickHouse/pull/46282) ([Anton Popov](https://github.com/CurtizJ)).
* Fix performance degradation caused by [#39737](https://github.com/ClickHouse/ClickHouse/issues/39737). [#46309](https://github.com/ClickHouse/ClickHouse/pull/46309) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The `replicas_status` handle will answer quickly even in case of a large replication queue. [#46310](https://github.com/ClickHouse/ClickHouse/pull/46310) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### Improvement
* Add avx512 support for Aggregate Sum, function unary arithmetic, function comparison. [#37870](https://github.com/ClickHouse/ClickHouse/pull/37870) ([zhao zhou](https://github.com/zzachimed)).
* close issue: [#38893](https://github.com/ClickHouse/ClickHouse/issues/38893). [#38950](https://github.com/ClickHouse/ClickHouse/pull/38950) ([hexiaoting](https://github.com/hexiaoting)).
* Migration from other databases and updates/deletes are mimicked by Collapsing/Replacing. Want to use the same SELECT queries without adding FINAL to all the existing queries. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)).
* Allow configuring storage as `SETTINGS disk='<disk_name>'` (instead of `storage_policy`) and with explicit disk creation `SETTINGS disk=disk(type=s3, ...)`. [#41976](https://github.com/ClickHouse/ClickHouse/pull/41976) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add new metrics for backups: num_processed_files and processed_files_size described actual number of processed files. [#42244](https://github.com/ClickHouse/ClickHouse/pull/42244) ([Aleksandr](https://github.com/AVMusorin)).
* Added retries on interserver DNS errors. [#43179](https://github.com/ClickHouse/ClickHouse/pull/43179) ([Anton Kozlov](https://github.com/tonickkozlov)).
* Rewrote the code around marks distribution and the overall coordination of the reading in order to achieve the maximum performance improvement. This closes [#34527](https://github.com/ClickHouse/ClickHouse/issues/34527). [#43772](https://github.com/ClickHouse/ClickHouse/pull/43772) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Remove redundant DISTINCT clauses in query (subqueries). Implemented on top of query plan. It does similar optimization as `optimize_duplicate_order_by_and_distinct` regarding DISTINCT clauses. Can be enabled via `query_plan_remove_redundant_distinct` setting. Related to [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#44176](https://github.com/ClickHouse/ClickHouse/pull/44176) ([Igor Nikonov](https://github.com/devcrafter)).
* Keeper improvement: try preallocating space on the disk to avoid undefined out-of-space issues. Introduce setting `max_log_file_size` for the maximum size of Keeper's Raft log files. [#44370](https://github.com/ClickHouse/ClickHouse/pull/44370) ([Antonio Andelic](https://github.com/antonio2368)).
* ``` sumIf(123, cond) -> 123 * countIf(1, cond) sum(if(cond, 123, 0)) -> 123 * countIf(cond) sum(if(cond, 0, 123)) -> 123 * countIf(not(cond)) ```. [#44728](https://github.com/ClickHouse/ClickHouse/pull/44728) ([李扬](https://github.com/taiyang-li)).
* Optimize behavior for a replica delay api logic in case the replica is read-only. [#45148](https://github.com/ClickHouse/ClickHouse/pull/45148) ([mateng915](https://github.com/mateng0915)).
* Introduce gwp-asan implemented by llvm runtime. This closes [#27039](https://github.com/ClickHouse/ClickHouse/issues/27039). [#45226](https://github.com/ClickHouse/ClickHouse/pull/45226) ([Han Fei](https://github.com/hanfei1991)).
* ... in the case key casted from uint64 to uint32, small impact for little endian platform but key value becomes zero in big endian case. ### Documentation entry for user-facing changes. [#45375](https://github.com/ClickHouse/ClickHouse/pull/45375) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
* Mark Gorilla compression on columns of non-Float* type as suspicious. [#45376](https://github.com/ClickHouse/ClickHouse/pull/45376) ([Robert Schulze](https://github.com/rschu1ze)).
* Allow removing redundant aggregation keys with constants (e.g., simplify `GROUP BY a, a + 1` to `GROUP BY a`). [#45415](https://github.com/ClickHouse/ClickHouse/pull/45415) ([Dmitry Novik](https://github.com/novikd)).
* Show replica name that is executing a merge in the postpone_reason. [#45458](https://github.com/ClickHouse/ClickHouse/pull/45458) ([Frank Chen](https://github.com/FrankChen021)).
* Save exception stack trace in part_log. [#45459](https://github.com/ClickHouse/ClickHouse/pull/45459) ([Frank Chen](https://github.com/FrankChen021)).
* Make RegExpTreeDictionary a ua parser which is compatible with https://github.com/ua-parser/uap-core. [#45631](https://github.com/ClickHouse/ClickHouse/pull/45631) ([Han Fei](https://github.com/hanfei1991)).
* Enable ICU data support on s390x platform. [#45632](https://github.com/ClickHouse/ClickHouse/pull/45632) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
* Updated checking of SYSTEM SYNC REPLICA resolves [#45508](https://github.com/ClickHouse/ClickHouse/issues/45508) Implementation: * Updated to wait for current last entry to be processed (after pulling shared log) instead of queue size becoming 0. * Updated Subscriber to notify both queue size and removed log_entry_id. [#45648](https://github.com/ClickHouse/ClickHouse/pull/45648) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Disallow creation of new columns compressed by a combination of codecs "Delta" or "DoubleDelta" followed by codecs "Gorilla" or "FPC". This can be bypassed using setting "allow_suspicious_codecs = true". [#45652](https://github.com/ClickHouse/ClickHouse/pull/45652) ([Robert Schulze](https://github.com/rschu1ze)).
* Rename setting `replication_alter_partitions_sync` to `alter_sync`. [#45659](https://github.com/ClickHouse/ClickHouse/pull/45659) ([Antonio Andelic](https://github.com/antonio2368)).
* The `generateRandom` table function and the engine now support `LowCardinality` data types. This is useful for testing, for example you can write `INSERT INTO table SELECT * FROM generateRandom() LIMIT 1000`. This is needed to debug [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590). [#45661](https://github.com/ClickHouse/ClickHouse/pull/45661) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add ability to ignore unknown keys in JSON object for named tuples (`input_format_json_ignore_unknown_keys_in_named_tuple`). [#45678](https://github.com/ClickHouse/ClickHouse/pull/45678) ([Azat Khuzhin](https://github.com/azat)).
* - The experimental query result cache now provides more modular configuration settings. [#45679](https://github.com/ClickHouse/ClickHouse/pull/45679) ([Robert Schulze](https://github.com/rschu1ze)).
* Renamed "query result cache" to "query cache". [#45682](https://github.com/ClickHouse/ClickHouse/pull/45682) ([Robert Schulze](https://github.com/rschu1ze)).
* add **SYSTEM SYNC FILE CACHE** command. It will call sync syscall. It achieve [#8921](https://github.com/ClickHouse/ClickHouse/issues/8921). [#45685](https://github.com/ClickHouse/ClickHouse/pull/45685) ([DR](https://github.com/freedomDR)).
* Add new S3 setting `allow_head_object_request`. This PR makes usage of `GetObjectAttributes` request instead of `HeadObject` introduced in https://github.com/ClickHouse/ClickHouse/pull/45288 optional (and disabled by default). [#45701](https://github.com/ClickHouse/ClickHouse/pull/45701) ([Vitaly Baranov](https://github.com/vitlibar)).
* Add ability to override connection settings based on connection names (that said that now you can forget about storing password for each connection, you can simply put everything into `~/.clickhouse-client/config.xml` and even use different history files for them, which can be also useful). [#45715](https://github.com/ClickHouse/ClickHouse/pull/45715) ([Azat Khuzhin](https://github.com/azat)).
* Arrow format support duration type. Closes [#45669](https://github.com/ClickHouse/ClickHouse/issues/45669). [#45750](https://github.com/ClickHouse/ClickHouse/pull/45750) ([flynn](https://github.com/ucasfl)).
* Extend the logging in the Query Cache to improve investigations of the caching behavior. [#45751](https://github.com/ClickHouse/ClickHouse/pull/45751) ([Robert Schulze](https://github.com/rschu1ze)).
* The query cache's server-level settings are now reconfigurable at runtime. [#45758](https://github.com/ClickHouse/ClickHouse/pull/45758) ([Robert Schulze](https://github.com/rschu1ze)).
* Hide password in logs when a table function's arguments are specified with a named collection:. [#45774](https://github.com/ClickHouse/ClickHouse/pull/45774) ([Vitaly Baranov](https://github.com/vitlibar)).
* Improve internal S3 client to correctly deduce regions and redirections for different types of URLs. [#45783](https://github.com/ClickHouse/ClickHouse/pull/45783) ([Antonio Andelic](https://github.com/antonio2368)).
* - Add support for Map, IPv4 and IPv6 types in generateRandom. Mostly useful for testing. [#45785](https://github.com/ClickHouse/ClickHouse/pull/45785) ([Raúl Marín](https://github.com/Algunenano)).
* Support empty/notEmpty for IP types. [#45799](https://github.com/ClickHouse/ClickHouse/pull/45799) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* The column `num_processed_files` was splitted into two columns: `num_files` (for BACKUP) and `files_read` (for RESTORE). The column `processed_files_size` was splitted into two columns: `total_size` (for BACKUP) and `bytes_read` (for RESTORE). [#45800](https://github.com/ClickHouse/ClickHouse/pull/45800) ([Vitaly Baranov](https://github.com/vitlibar)).
* 1. Upgrade Intel QPL from v0.3.0 to v1.0.0 2. Build libaccel-config and link it statically to QPL library instead of dynamically. [#45809](https://github.com/ClickHouse/ClickHouse/pull/45809) ([jasperzhu](https://github.com/jinjunzh)).
* Add support for `SHOW ENGINES` query. [#45859](https://github.com/ClickHouse/ClickHouse/pull/45859) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* - Improved how the obfuscator deals with queries. [#45867](https://github.com/ClickHouse/ClickHouse/pull/45867) ([Raúl Marín](https://github.com/Algunenano)).
* Improved how memory bound merging and aggregation in order on top query plan interact. Previously we fell back to explicit sorting for AIO in some cases when it wasn't actually needed. So it is a perf issue, not a correctness one. [#45892](https://github.com/ClickHouse/ClickHouse/pull/45892) ([Nikita Taranov](https://github.com/nickitat)).
* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#45914](https://github.com/ClickHouse/ClickHouse/pull/45914) ([Joanna Hulboj](https://github.com/jh0x)).
* Add setting `check_referential_table_dependencies` to check referential dependencies on `DROP TABLE`. This PR solves [#38326](https://github.com/ClickHouse/ClickHouse/issues/38326). [#45936](https://github.com/ClickHouse/ClickHouse/pull/45936) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix `tupleElement` return `Null` when having `Null` argument. Closes [#45894](https://github.com/ClickHouse/ClickHouse/issues/45894). [#45952](https://github.com/ClickHouse/ClickHouse/pull/45952) ([flynn](https://github.com/ucasfl)).
* Throw an error on no files satisfying S3 wildcard. Closes [#45587](https://github.com/ClickHouse/ClickHouse/issues/45587). [#45957](https://github.com/ClickHouse/ClickHouse/pull/45957) ([chen](https://github.com/xiedeyantu)).
* Use cluster state data to check concurrent backup/restore. [#45982](https://github.com/ClickHouse/ClickHouse/pull/45982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Use "exact" matching for fuzzy search, which has correct case ignorance and more appropriate algorithm for matching SQL queries. [#46000](https://github.com/ClickHouse/ClickHouse/pull/46000) ([Azat Khuzhin](https://github.com/azat)).
* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#46042](https://github.com/ClickHouse/ClickHouse/pull/46042) ([Joanna Hulboj](https://github.com/jh0x)).
* Forbid wrong create View syntax `CREATE View X TO Y AS SELECT`. Closes [#4331](https://github.com/ClickHouse/ClickHouse/issues/4331). [#46043](https://github.com/ClickHouse/ClickHouse/pull/46043) ([flynn](https://github.com/ucasfl)).
* Storage Log family support settings `storage_policy`. Closes [#43421](https://github.com/ClickHouse/ClickHouse/issues/43421). [#46044](https://github.com/ClickHouse/ClickHouse/pull/46044) ([flynn](https://github.com/ucasfl)).
* Improve format `JSONColumns` when result is empty. Closes [#46024](https://github.com/ClickHouse/ClickHouse/issues/46024). [#46053](https://github.com/ClickHouse/ClickHouse/pull/46053) ([flynn](https://github.com/ucasfl)).
* - MultiVersion: replace lock_guard to atomic op. [#46057](https://github.com/ClickHouse/ClickHouse/pull/46057) ([Konstantin Morozov](https://github.com/k-morozov)).
* Add reference implementation for SipHash128. [#46065](https://github.com/ClickHouse/ClickHouse/pull/46065) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Add new metric to record allocations times and bytes using mmap. [#46068](https://github.com/ClickHouse/ClickHouse/pull/46068) ([李扬](https://github.com/taiyang-li)).
* Currently for functions like `leftPad`, `rightPad`, `leftPadUTF8`, `rightPadUTF8`, the second argument `length` must be UInt8|16|32|64|128|256. Which is too strict for clickhouse users, besides, it is not consistent with other similar functions like `arrayResize`, `substring` and so on. [#46103](https://github.com/ClickHouse/ClickHouse/pull/46103) ([李扬](https://github.com/taiyang-li)).
* Update CapnProto to v0.10.3 to avoid CVE-2022-46149 ### Documentation entry for user-facing changes. [#46139](https://github.com/ClickHouse/ClickHouse/pull/46139) ([Mallik Hassan](https://github.com/SadiHassan)).
* Fix assertion in the `welchTTest` function in debug build when the resulting statistics is NaN. Unified the behavior with other similar functions. Change the behavior of `studentTTest` to return NaN instead of throwing an exception because the previous behavior was inconvenient. This closes [#41176](https://github.com/ClickHouse/ClickHouse/issues/41176) This closes [#42162](https://github.com/ClickHouse/ClickHouse/issues/42162). [#46141](https://github.com/ClickHouse/ClickHouse/pull/46141) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* More convenient usage of big integers and ORDER BY WITH FILL. Allow using plain integers for start and end points in WITH FILL when ORDER BY big (128-bit and 256-bit) integers. Fix the wrong result for big integers with negative start or end points. This closes [#16733](https://github.com/ClickHouse/ClickHouse/issues/16733). [#46152](https://github.com/ClickHouse/ClickHouse/pull/46152) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add parts, active_parts and total_marks columns to system.tables on [issue](https://github.com/ClickHouse/ClickHouse/issues/44336). [#46161](https://github.com/ClickHouse/ClickHouse/pull/46161) ([attack204](https://github.com/attack204)).
* Functions "multi[Fuzzy]Match(Any|AnyIndex|AllIndices}" now reject regexes which will likely evaluate very slowly in vectorscan. [#46167](https://github.com/ClickHouse/ClickHouse/pull/46167) ([Robert Schulze](https://github.com/rschu1ze)).
* When `insert_null_as_default` is enabled and column doesn't have defined default value, the default of column type will be used. Also this PR fixes using default values on nulls in case of LowCardinality columns. [#46171](https://github.com/ClickHouse/ClickHouse/pull/46171) ([Kruglov Pavel](https://github.com/Avogar)).
* Prefer explicitly defined access keys for S3 clients. If `use_environment_credentials` is set to `true`, and the user has provided the access key through query or config, they will be used instead of the ones from the environment variable. [#46191](https://github.com/ClickHouse/ClickHouse/pull/46191) ([Antonio Andelic](https://github.com/antonio2368)).
* Concurrent merges are scheduled using round-robin by default to ensure fair and starvation-free operation. Previously in heavily overloaded shards, big merges could possibly be starved by smaller merges due to the use of strict priority scheduling. Added `background_merges_mutations_scheduling_policy` server config option to select scheduling algorithm (`round_robin` or `shortest_task_first`). [#46247](https://github.com/ClickHouse/ClickHouse/pull/46247) ([Sergei Trifonov](https://github.com/serxa)).
* Extend setting `input_format_null_as_default` for more formats. Fix setting `input_format_defaults_for_omitted_fields` with Native and TSKV formats. [#46284](https://github.com/ClickHouse/ClickHouse/pull/46284) ([Kruglov Pavel](https://github.com/Avogar)).
* - Add an alias "DATE_FORMAT()" for function "formatDateTime()" to improve compatibility with MySQL's SQL dialect, extend function`formatDateTime()` with substitutions "a", "b", "c", "h", "i", "k", "l" "r", "s", "W". ### Documentation entry for user-facing changes User-readable short description: `DATE_FORMAT` is an alias of `formatDateTime`. Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. (Provide link to [formatDateTime](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#formatdatetime)). [#46302](https://github.com/ClickHouse/ClickHouse/pull/46302) ([Jake Bamrah](https://github.com/JakeBamrah)).
* not for changelog - part of [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#46306](https://github.com/ClickHouse/ClickHouse/pull/46306) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Enable retries for INSERT by default in case of ZooKeeper session loss. We already use it in production. [#46308](https://github.com/ClickHouse/ClickHouse/pull/46308) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add `ProfileEvents` and `CurrentMetrics` about the callback tasks for parallel replicas (`s3Cluster` and `MergeTree` tables). [#46313](https://github.com/ClickHouse/ClickHouse/pull/46313) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add support for `DELETE` and `UPDATE` for tables using `KeeperMap` storage engine. [#46330](https://github.com/ClickHouse/ClickHouse/pull/46330) ([Antonio Andelic](https://github.com/antonio2368)).
* Update unixodbc to v2.3.11 to mitigate CVE-2011-1145 ### Documentation entry for user-facing changes. [#46363](https://github.com/ClickHouse/ClickHouse/pull/46363) ([Mallik Hassan](https://github.com/SadiHassan)).
* - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)).
* Allow writing RENAME queries with query parameters. Resolves [#45778](https://github.com/ClickHouse/ClickHouse/issues/45778). [#46407](https://github.com/ClickHouse/ClickHouse/pull/46407) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix parameterized SELECT queries with REPLACE transformer. Resolves [#33002](https://github.com/ClickHouse/ClickHouse/issues/33002). [#46420](https://github.com/ClickHouse/ClickHouse/pull/46420) ([Nikolay Degterinsky](https://github.com/evillique)).
* Exclude the internal database used for temporary/external tables from the calculation of asynchronous metric "NumberOfDatabases". This makes the behavior consistent with system table "system.databases". [#46435](https://github.com/ClickHouse/ClickHouse/pull/46435) ([Robert Schulze](https://github.com/rschu1ze)).
* Added `last_exception_time` column into distribution_queue table. [#46564](https://github.com/ClickHouse/ClickHouse/pull/46564) ([Aleksandr](https://github.com/AVMusorin)).
* Support for IN clause in parameterized views Implementation: * In case of parameterized views, the IN clause cannot be evaluated as constant expression during CREATE VIEW, added a check to ignore this step in case of parameterized view. * If parmeters are not in IN clause, we continue to evaluate it as constant expression. [#46583](https://github.com/ClickHouse/ClickHouse/pull/46583) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Do not load named collections on server startup (load them on first access instead). [#46607](https://github.com/ClickHouse/ClickHouse/pull/46607) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add separate access type `SHOW_NAMED_COLLECTIONS_SECRETS` to allow to see named collections and their keys, but making values hidden. Nevertheless, access type `SHOW NAMED COLLECTIONS` is still required. [#46667](https://github.com/ClickHouse/ClickHouse/pull/46667) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Hide arguments of custom disk merge tree setting. [#46670](https://github.com/ClickHouse/ClickHouse/pull/46670) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Ask for the password in clickhouse-client interactively in a case when the empty password is wrong. Closes [#46702](https://github.com/ClickHouse/ClickHouse/issues/46702). [#46730](https://github.com/ClickHouse/ClickHouse/pull/46730) ([Nikolay Degterinsky](https://github.com/evillique)).
* Backward compatibility for T64 codec support for IPv4. [#46747](https://github.com/ClickHouse/ClickHouse/pull/46747) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Allow to fallback from asynchronous insert to synchronous in case of large amount of data (more than `async_insert_max_data_size` bytes in single insert). [#46753](https://github.com/ClickHouse/ClickHouse/pull/46753) ([Anton Popov](https://github.com/CurtizJ)).
#### Bug Fix
* Fix wiping sensitive info in logs. [#45603](https://github.com/ClickHouse/ClickHouse/pull/45603) ([Vitaly Baranov](https://github.com/vitlibar)).
* There is a check in format "time_check() || ptr ? ptr->finished() : data->is_finished()". Operator "||" will be executed before operator "?", but expected that there should be separated time and ptr checks. Also it's unexpected to run "ptr->finished()" in case of nullptr, but with current expression it's possible. [#46054](https://github.com/ClickHouse/ClickHouse/pull/46054) ([Alexey Perevyshin](https://github.com/alexX512)).
#### Build/Testing/Packaging Improvement
* Allow to randomize merge tree settings in tests. [#38983](https://github.com/ClickHouse/ClickHouse/pull/38983) ([Anton Popov](https://github.com/CurtizJ)).
* Enable the HDFS support in PowerPC and which helps to fixes the following functional tests 02113_hdfs_assert.sh, 02244_hdfs_cluster.sql and 02368_cancel_write_into_hdfs.sh. [#44949](https://github.com/ClickHouse/ClickHouse/pull/44949) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)).
* Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* ClickHouse's fork of poco was moved from "contrib/" to "base/poco/". [#46075](https://github.com/ClickHouse/ClickHouse/pull/46075) ([Robert Schulze](https://github.com/rschu1ze)).
* Remove excessive license notices from preciseExp10.cpp. [#46163](https://github.com/ClickHouse/ClickHouse/pull/46163) ([DimasKovas](https://github.com/DimasKovas)).
* Add an option for `clickhouse-watchdog` to restart the child process. This does not make a lot of use. [#46312](https://github.com/ClickHouse/ClickHouse/pull/46312) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* If the environment variable `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` is set to 1, the Docker container will run `clickhouse-server` as a child instead of the first process, and restart it when it exited. [#46391](https://github.com/ClickHouse/ClickHouse/pull/46391) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* - Fix Systemd service file. [#46461](https://github.com/ClickHouse/ClickHouse/pull/46461) ([SuperDJY](https://github.com/cmsxbc)).
* Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Raised the minimum Clang version needed to build ClickHouse from 12 to 15. [#46710](https://github.com/ClickHouse/ClickHouse/pull/46710) ([Robert Schulze](https://github.com/rschu1ze)).
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)).
* - Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)).
* Fix functions (quantilesExactExclusive, quantilesExactInclusive) return unsorted array element. [#45379](https://github.com/ClickHouse/ClickHouse/pull/45379) ([wujunfu](https://github.com/wujunfu)).
* Fix uncaught exception in HTTPHandler when open telemetry is enabled. [#45456](https://github.com/ClickHouse/ClickHouse/pull/45456) ([Frank Chen](https://github.com/FrankChen021)).
* Don't infer Dates from 8 digit numbers. It could lead to wrong data to be read. [#45581](https://github.com/ClickHouse/ClickHouse/pull/45581) ([Kruglov Pavel](https://github.com/Avogar)).
* Fixes to correctly use `odbc_bridge_use_connection_pooling` setting. [#45591](https://github.com/ClickHouse/ClickHouse/pull/45591) ([Bharat Nallan](https://github.com/bharatnc)).
* when the callback in the cache is called, it is possible that this cache is destructed. To keep it safe, we capture members by value. It's also safe for task schedule because it will be deactivated before storage is destroyed. Resolve [#45548](https://github.com/ClickHouse/ClickHouse/issues/45548). [#45601](https://github.com/ClickHouse/ClickHouse/pull/45601) ([Han Fei](https://github.com/hanfei1991)).
* - Fix data corruption when codecs Delta or DoubleDelta are combined with codec Gorilla. [#45615](https://github.com/ClickHouse/ClickHouse/pull/45615) ([Robert Schulze](https://github.com/rschu1ze)).
* Correctly check types when using N-gram bloom filter index to avoid invalid reads. [#45617](https://github.com/ClickHouse/ClickHouse/pull/45617) ([Antonio Andelic](https://github.com/antonio2368)).
* A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set<std::string>` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)).
* Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)).
* Set compression method and level for backup Closes [#45690](https://github.com/ClickHouse/ClickHouse/issues/45690). [#45737](https://github.com/ClickHouse/ClickHouse/pull/45737) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
* Should use `select_query_typed.limitByOffset()` instead of `select_query_typed.limitOffset()`. [#45817](https://github.com/ClickHouse/ClickHouse/pull/45817) ([刘陶峰](https://github.com/taofengliu)).
* When use experimental analyzer, queries like `SELECT number FROM numbers(100) LIMIT 10 OFFSET 10;` get wrong results (empty result for this sql). That is caused by an unnecessary offset step added by planner. [#45822](https://github.com/ClickHouse/ClickHouse/pull/45822) ([刘陶峰](https://github.com/taofengliu)).
* Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Add the `query_kind` column to the `system.processes` table and the `SHOW PROCESSLIST` query. Remove duplicate code. It fixes a bug: the global configuration parameter `max_concurrent_select_queries` was not respected to queries with `INTERSECT` or `EXCEPT` chains. [#45872](https://github.com/ClickHouse/ClickHouse/pull/45872) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix crash in a function `stochasticLinearRegression`. Found by WingFuzz. [#45985](https://github.com/ClickHouse/ClickHouse/pull/45985) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)).
* * Fix read in order optimization for DESC sorting with FINAL, close [#45815](https://github.com/ClickHouse/ClickHouse/issues/45815). [#46009](https://github.com/ClickHouse/ClickHouse/pull/46009) ([Vladimir C](https://github.com/vdimir)).
* Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)).
* Fix elapsed column in system.processes (10x error). [#46047](https://github.com/ClickHouse/ClickHouse/pull/46047) ([Azat Khuzhin](https://github.com/azat)).
* Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. [#46087](https://github.com/ClickHouse/ClickHouse/pull/46087) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix environment variable substitution in the configuration when a parameter already has a value. This closes [#46131](https://github.com/ClickHouse/ClickHouse/issues/46131). This closes [#9547](https://github.com/ClickHouse/ClickHouse/issues/9547). [#46144](https://github.com/ClickHouse/ClickHouse/pull/46144) ([pufit](https://github.com/pufit)).
* Fix incorrect predicate push down with grouping sets. Closes [#45947](https://github.com/ClickHouse/ClickHouse/issues/45947). [#46151](https://github.com/ClickHouse/ClickHouse/pull/46151) ([flynn](https://github.com/ucasfl)).
* Fix possible pipeline stuck error on `fulls_sorting_join` with constant keys. [#46175](https://github.com/ClickHouse/ClickHouse/pull/46175) ([Vladimir C](https://github.com/vdimir)).
* Never rewrite tuple functions as literals during formatting to avoid incorrect results. [#46232](https://github.com/ClickHouse/ClickHouse/pull/46232) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Fix possible out of bounds error while reading LowCardinality(Nullable) in Arrow format. [#46270](https://github.com/ClickHouse/ClickHouse/pull/46270) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Fix possible crash which can be caused by an integer overflow while deserializing aggregating state of a function that stores HashTable. [#46349](https://github.com/ClickHouse/ClickHouse/pull/46349) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)).
* Fixed a LOGICAL_ERROR on an attempt to execute `ALTER ... MOVE PART ... TO TABLE`. This type of query was never actually supported. [#46359](https://github.com/ClickHouse/ClickHouse/pull/46359) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix s3Cluster schema inference in parallel distributed insert select when `parallel_distributed_insert_select` is enabled. [#46381](https://github.com/ClickHouse/ClickHouse/pull/46381) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix queries like `ALTER TABLE ... UPDATE nested.arr1 = nested.arr2 ...`, where `arr1` and `arr2` are fields of the same `Nested` column. [#46387](https://github.com/ClickHouse/ClickHouse/pull/46387) ([Anton Popov](https://github.com/CurtizJ)).
* Scheduler may fail to schedule a task. If it happens, the whole MulityPartUpload should be aborted and `UploadHelper` must wait for already scheduled tasks. [#46451](https://github.com/ClickHouse/ClickHouse/pull/46451) ([Dmitry Novik](https://github.com/novikd)).
* Fix PREWHERE for Merge with different default types (fixes some `NOT_FOUND_COLUMN_IN_BLOCK` when the default type for the column differs, also allow `PREWHERE` when the type of column is the same across tables, and prohibit it, only if it differs). [#46454](https://github.com/ClickHouse/ClickHouse/pull/46454) ([Azat Khuzhin](https://github.com/azat)).
* Fix a crash that could happen when constant values are used in `ORDER BY`. Fixes [#46466](https://github.com/ClickHouse/ClickHouse/issues/46466). [#46493](https://github.com/ClickHouse/ClickHouse/pull/46493) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Do not throw exception if `disk` setting was specified on query level, but `storage_policy` was specified in config merge tree settings section. `disk` will override setting from config. [#46533](https://github.com/ClickHouse/ClickHouse/pull/46533) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* fixes [#46557](https://github.com/ClickHouse/ClickHouse/issues/46557). [#46611](https://github.com/ClickHouse/ClickHouse/pull/46611) ([Alexander Gololobov](https://github.com/davenger)).
* Fix endless restarts of clickhouse-server systemd unit if server cannot start within 1m30sec (Disable timeout logic for starting clickhouse-server from systemd service). [#46613](https://github.com/ClickHouse/ClickHouse/pull/46613) ([Azat Khuzhin](https://github.com/azat)).
* Allocated during asynchronous inserts memory buffers were deallocated in the global context and MemoryTracker counters for corresponding user and query were not updated correctly. That led to false positive OOM exceptions. [#46622](https://github.com/ClickHouse/ClickHouse/pull/46622) ([Dmitry Novik](https://github.com/novikd)).
* Fix totals and extremes with constants in clickhouse-local. Closes [#43831](https://github.com/ClickHouse/ClickHouse/issues/43831). [#46669](https://github.com/ClickHouse/ClickHouse/pull/46669) ([Kruglov Pavel](https://github.com/Avogar)).
* Handle `input_format_null_as_default` for nested types. [#46725](https://github.com/ClickHouse/ClickHouse/pull/46725) ([Azat Khuzhin](https://github.com/azat)).
#### Bug-fix
* Updated to not clear on_expression from table_join as its used by future analyze runs resolves [#45185](https://github.com/ClickHouse/ClickHouse/issues/45185). [#46487](https://github.com/ClickHouse/ClickHouse/pull/46487) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
#### Build Improvement
* Fixed endian issue in snappy library for s390x. [#45670](https://github.com/ClickHouse/ClickHouse/pull/45670) ([Harry Lee](https://github.com/HarryLeeIBM)).
* Fixed endian issue in CityHash for s390x. [#46096](https://github.com/ClickHouse/ClickHouse/pull/46096) ([Harry Lee](https://github.com/HarryLeeIBM)).
* Fixed Functional Test 00900_long_parquet for S390x. [#46181](https://github.com/ClickHouse/ClickHouse/pull/46181) ([Sanjam Panda](https://github.com/saitama951)).
* Fixed endian issues in SQL hash functions on s390x architectures. [#46495](https://github.com/ClickHouse/ClickHouse/pull/46495) ([Harry Lee](https://github.com/HarryLeeIBM)).
#### NO CL ENTRY
* NO CL ENTRY: 'Revert "Add check for running workflows to merge_pr.py"'. [#45802](https://github.com/ClickHouse/ClickHouse/pull/45802) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* NO CL ENTRY: 'Revert "Improve behaviour of conversion into Date for boundary value 65535"'. [#46007](https://github.com/ClickHouse/ClickHouse/pull/46007) ([Antonio Andelic](https://github.com/antonio2368)).
* NO CL ENTRY: 'Revert "Allow vertical merges from compact to wide parts"'. [#46236](https://github.com/ClickHouse/ClickHouse/pull/46236) ([Anton Popov](https://github.com/CurtizJ)).
* NO CL ENTRY: 'Revert "Beter diagnostics from http in clickhouse-test"'. [#46301](https://github.com/ClickHouse/ClickHouse/pull/46301) ([Alexander Tokmakov](https://github.com/tavplubix)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Revert "Merge pull request [#38212](https://github.com/ClickHouse/ClickHouse/issues/38212) from azat/no-stress" [#38750](https://github.com/ClickHouse/ClickHouse/pull/38750) ([Azat Khuzhin](https://github.com/azat)).
* More interesting settings for Stress Tests [#41534](https://github.com/ClickHouse/ClickHouse/pull/41534) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Attempt to fix 'Local: No offset stored message' from Kafka [#42391](https://github.com/ClickHouse/ClickHouse/pull/42391) ([filimonov](https://github.com/filimonov)).
* Analyzer SETTINGS push down [#42976](https://github.com/ClickHouse/ClickHouse/pull/42976) ([Maksim Kita](https://github.com/kitaisreal)).
* Simply filesystem helpers to check is-readable/writable/executable [#43405](https://github.com/ClickHouse/ClickHouse/pull/43405) ([Azat Khuzhin](https://github.com/azat)).
* Add CPU flamegraphs for perf tests [#43529](https://github.com/ClickHouse/ClickHouse/pull/43529) ([Azat Khuzhin](https://github.com/azat)).
* More robust CI parsers [#44226](https://github.com/ClickHouse/ClickHouse/pull/44226) ([Azat Khuzhin](https://github.com/azat)).
* Fix error message for a broken distributed batches ("While sending batch") [#44907](https://github.com/ClickHouse/ClickHouse/pull/44907) ([Azat Khuzhin](https://github.com/azat)).
* Catch exceptions in BackgroundSchedulePool [#44923](https://github.com/ClickHouse/ClickHouse/pull/44923) ([Azat Khuzhin](https://github.com/azat)).
* Add encryption support to OpenSSL [#45258](https://github.com/ClickHouse/ClickHouse/pull/45258) ([Boris Kuschel](https://github.com/bkuschel)).
* Revert code in TreeRewriter for proper column order for UNION [#45282](https://github.com/ClickHouse/ClickHouse/pull/45282) ([Azat Khuzhin](https://github.com/azat)).
* Fix no shared id during drop for the fourth time [#45363](https://github.com/ClickHouse/ClickHouse/pull/45363) ([alesapin](https://github.com/alesapin)).
* HashedDictionary sharded fix nullable values [#45396](https://github.com/ClickHouse/ClickHouse/pull/45396) ([Maksim Kita](https://github.com/kitaisreal)).
* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Simplify binary locating in clickhouse-test [#45484](https://github.com/ClickHouse/ClickHouse/pull/45484) ([Azat Khuzhin](https://github.com/azat)).
* Fix race in NuRaft's asio listener [#45511](https://github.com/ClickHouse/ClickHouse/pull/45511) ([Antonio Andelic](https://github.com/antonio2368)).
* Make ColumnNode::isEqualImpl more strict [#45518](https://github.com/ClickHouse/ClickHouse/pull/45518) ([Dmitry Novik](https://github.com/novikd)).
* Fix krb5 for OpenSSL [#45519](https://github.com/ClickHouse/ClickHouse/pull/45519) ([Boris Kuschel](https://github.com/bkuschel)).
* s390x build support [#45520](https://github.com/ClickHouse/ClickHouse/pull/45520) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
* Better formatting for exception messages 2 [#45527](https://github.com/ClickHouse/ClickHouse/pull/45527) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Try to fix test `test_storage_s3/test.py::test_wrong_s3_syntax` (race in `StorageS3`) [#45529](https://github.com/ClickHouse/ClickHouse/pull/45529) ([Anton Popov](https://github.com/CurtizJ)).
* Analyzer add test for CREATE TABLE AS SELECT [#45533](https://github.com/ClickHouse/ClickHouse/pull/45533) ([Maksim Kita](https://github.com/kitaisreal)).
* LowCardinality insert fix [#45585](https://github.com/ClickHouse/ClickHouse/pull/45585) ([Maksim Kita](https://github.com/kitaisreal)).
* Update 02482_load_parts_refcounts.sh [#45604](https://github.com/ClickHouse/ClickHouse/pull/45604) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Extend assertion in buildPushingToViewsChain() to respect is_detached [#45610](https://github.com/ClickHouse/ClickHouse/pull/45610) ([Azat Khuzhin](https://github.com/azat)).
* Remove useless code [#45612](https://github.com/ClickHouse/ClickHouse/pull/45612) ([Anton Popov](https://github.com/CurtizJ)).
* Improve "at least part X is missing" error message [#45613](https://github.com/ClickHouse/ClickHouse/pull/45613) ([Azat Khuzhin](https://github.com/azat)).
* Refactoring of code near merge tree parts [#45619](https://github.com/ClickHouse/ClickHouse/pull/45619) ([Anton Popov](https://github.com/CurtizJ)).
* Update version after release [#45634](https://github.com/ClickHouse/ClickHouse/pull/45634) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update version_date.tsv and changelogs after v23.1.1.3077-stable [#45635](https://github.com/ClickHouse/ClickHouse/pull/45635) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update version_date.tsv and changelogs after v22.10.7.13-stable [#45637](https://github.com/ClickHouse/ClickHouse/pull/45637) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Improve release script [#45657](https://github.com/ClickHouse/ClickHouse/pull/45657) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Suppress TOO_MANY_PARTS in BC check [#45691](https://github.com/ClickHouse/ClickHouse/pull/45691) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix build [#45692](https://github.com/ClickHouse/ClickHouse/pull/45692) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add recordings for 23.1 and Tel Aviv [#45695](https://github.com/ClickHouse/ClickHouse/pull/45695) ([Tyler Hannan](https://github.com/tylerhannan)).
* Integrate IO scheduler with buffers for remote reads and writes [#45711](https://github.com/ClickHouse/ClickHouse/pull/45711) ([Sergei Trifonov](https://github.com/serxa)).
* Add missing SYSTEM FLUSH LOGS for clickhouse-test [#45713](https://github.com/ClickHouse/ClickHouse/pull/45713) ([Azat Khuzhin](https://github.com/azat)).
* tests: add missing allow_suspicious_codecs in 02536_delta_gorilla_corruption (fixes fasttest) [#45735](https://github.com/ClickHouse/ClickHouse/pull/45735) ([Azat Khuzhin](https://github.com/azat)).
* Improve MEMERY_LIMIT_EXCEEDED exception message [#45743](https://github.com/ClickHouse/ClickHouse/pull/45743) ([Dmitry Novik](https://github.com/novikd)).
* Fix style and typo [#45744](https://github.com/ClickHouse/ClickHouse/pull/45744) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update version_date.tsv and changelogs after v22.8.13.20-lts [#45749](https://github.com/ClickHouse/ClickHouse/pull/45749) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v22.11.5.15-stable [#45754](https://github.com/ClickHouse/ClickHouse/pull/45754) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v23.1.2.9-stable [#45755](https://github.com/ClickHouse/ClickHouse/pull/45755) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Docs: Fix formatting [#45756](https://github.com/ClickHouse/ClickHouse/pull/45756) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix typo + add boringssl comment [#45757](https://github.com/ClickHouse/ClickHouse/pull/45757) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix flaky test, @alesapin please help! [#45759](https://github.com/ClickHouse/ClickHouse/pull/45759) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove trash [#45760](https://github.com/ClickHouse/ClickHouse/pull/45760) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix Flaky Check [#45765](https://github.com/ClickHouse/ClickHouse/pull/45765) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update dictionary.md [#45775](https://github.com/ClickHouse/ClickHouse/pull/45775) ([Derek Chia](https://github.com/DerekChia)).
* Added a test for multiple ignore subqueries with nested select [#45784](https://github.com/ClickHouse/ClickHouse/pull/45784) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* Support DELETE ON CLUSTER [#45786](https://github.com/ClickHouse/ClickHouse/pull/45786) ([Alexander Gololobov](https://github.com/davenger)).
* outdated parts are loading async, need to wait them after attach [#45787](https://github.com/ClickHouse/ClickHouse/pull/45787) ([Sema Checherinda](https://github.com/CheSema)).
* Fix bug in tables drop which can lead to potential query hung [#45791](https://github.com/ClickHouse/ClickHouse/pull/45791) ([alesapin](https://github.com/alesapin)).
* Fix race condition on a part check cancellation [#45793](https://github.com/ClickHouse/ClickHouse/pull/45793) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Do not restrict count() query to 1 thread in isStorageTouchedByMutations() [#45794](https://github.com/ClickHouse/ClickHouse/pull/45794) ([Alexander Gololobov](https://github.com/davenger)).
* Fix test `test_azure_blob_storage_zero_copy_replication ` (memory leak in azure sdk) [#45796](https://github.com/ClickHouse/ClickHouse/pull/45796) ([Anton Popov](https://github.com/CurtizJ)).
* Add check for running workflows to merge_pr.py [#45801](https://github.com/ClickHouse/ClickHouse/pull/45801) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix flaky test `02531_two_level_aggregation_bug.sh` [#45806](https://github.com/ClickHouse/ClickHouse/pull/45806) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Minor doc follow-up to [#45382](https://github.com/ClickHouse/ClickHouse/issues/45382) [#45816](https://github.com/ClickHouse/ClickHouse/pull/45816) ([Robert Schulze](https://github.com/rschu1ze)).
* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Make separate DROP_PART log entry type [#45821](https://github.com/ClickHouse/ClickHouse/pull/45821) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Do not cancel created prs [#45823](https://github.com/ClickHouse/ClickHouse/pull/45823) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix ASTQualifiedAsterisk cloning [#45829](https://github.com/ClickHouse/ClickHouse/pull/45829) ([Raúl Marín](https://github.com/Algunenano)).
* Update 02540_duplicate_primary_key.sql [#45846](https://github.com/ClickHouse/ClickHouse/pull/45846) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Proper includes for ConnectionTimeoutsContext.h [#45848](https://github.com/ClickHouse/ClickHouse/pull/45848) ([Raúl Marín](https://github.com/Algunenano)).
* Fix minor mistake after refactoring [#45857](https://github.com/ClickHouse/ClickHouse/pull/45857) ([Anton Popov](https://github.com/CurtizJ)).
* Fix flaky ttl_replicated test (remove sleep) [#45858](https://github.com/ClickHouse/ClickHouse/pull/45858) ([alesapin](https://github.com/alesapin)).
* Add some context to stress test failures [#45869](https://github.com/ClickHouse/ClickHouse/pull/45869) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix clang-tidy error in Keeper `Changelog` [#45888](https://github.com/ClickHouse/ClickHouse/pull/45888) ([Antonio Andelic](https://github.com/antonio2368)).
* do not block merges when old parts are droping in drop queries [#45889](https://github.com/ClickHouse/ClickHouse/pull/45889) ([Sema Checherinda](https://github.com/CheSema)).
* do not run wal on remote disks [#45907](https://github.com/ClickHouse/ClickHouse/pull/45907) ([Sema Checherinda](https://github.com/CheSema)).
* Dashboard improvements [#45935](https://github.com/ClickHouse/ClickHouse/pull/45935) ([Kevin Zhang](https://github.com/Kinzeng)).
* Better context for stress tests failures [#45937](https://github.com/ClickHouse/ClickHouse/pull/45937) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix IO URing [#45940](https://github.com/ClickHouse/ClickHouse/pull/45940) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Docs: Remove obsolete query result cache page [#45958](https://github.com/ClickHouse/ClickHouse/pull/45958) ([Robert Schulze](https://github.com/rschu1ze)).
* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update AggregateFunctionSparkbar [#45961](https://github.com/ClickHouse/ClickHouse/pull/45961) ([Vladimir C](https://github.com/vdimir)).
* Update cherrypick_pr to get mergeable state [#45972](https://github.com/ClickHouse/ClickHouse/pull/45972) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add "final" specifier to some classes [#45973](https://github.com/ClickHouse/ClickHouse/pull/45973) ([Robert Schulze](https://github.com/rschu1ze)).
* Improve local running of cherry_pick.py [#45980](https://github.com/ClickHouse/ClickHouse/pull/45980) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Properly detect changes in Rust code and recompile Rust libraries [#45981](https://github.com/ClickHouse/ClickHouse/pull/45981) ([Azat Khuzhin](https://github.com/azat)).
* Avoid leaving symbols leftovers for query fuzzy search [#45983](https://github.com/ClickHouse/ClickHouse/pull/45983) ([Azat Khuzhin](https://github.com/azat)).
* Fix basic functionality with type `Object` and new analyzer [#45992](https://github.com/ClickHouse/ClickHouse/pull/45992) ([Anton Popov](https://github.com/CurtizJ)).
* Check dynamic columns of part before its commit [#45995](https://github.com/ClickHouse/ClickHouse/pull/45995) ([Anton Popov](https://github.com/CurtizJ)).
* Minor doc fixes for inverted index [#46004](https://github.com/ClickHouse/ClickHouse/pull/46004) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix terribly broken, fragile and potentially cyclic linking [#46006](https://github.com/ClickHouse/ClickHouse/pull/46006) ([Robert Schulze](https://github.com/rschu1ze)).
* Docs: Mention time zone randomization [#46008](https://github.com/ClickHouse/ClickHouse/pull/46008) ([Robert Schulze](https://github.com/rschu1ze)).
* Analyzer limit offset test rename [#46011](https://github.com/ClickHouse/ClickHouse/pull/46011) ([Maksim Kita](https://github.com/kitaisreal)).
* Update version_date.tsv and changelogs after v23.1.3.5-stable [#46012](https://github.com/ClickHouse/ClickHouse/pull/46012) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update sorting properties after reading in order applied [#46014](https://github.com/ClickHouse/ClickHouse/pull/46014) ([Igor Nikonov](https://github.com/devcrafter)).
* Fix disabled by mistake hung check [#46020](https://github.com/ClickHouse/ClickHouse/pull/46020) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix memory leak at creation of curl connection in azure sdk [#46025](https://github.com/ClickHouse/ClickHouse/pull/46025) ([Anton Popov](https://github.com/CurtizJ)).
* Add checks for installable packages to workflows [#46036](https://github.com/ClickHouse/ClickHouse/pull/46036) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix data race in BACKUP [#46040](https://github.com/ClickHouse/ClickHouse/pull/46040) ([Azat Khuzhin](https://github.com/azat)).
* Dump sanitizer errors in the integration tests logs [#46041](https://github.com/ClickHouse/ClickHouse/pull/46041) ([Azat Khuzhin](https://github.com/azat)).
* Temporarily disable one rabbitmq flaky test [#46052](https://github.com/ClickHouse/ClickHouse/pull/46052) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Remove unnecessary execute() while evaluating a constant expression. [#46058](https://github.com/ClickHouse/ClickHouse/pull/46058) ([Vitaly Baranov](https://github.com/vitlibar)).
* Polish S3 client [#46070](https://github.com/ClickHouse/ClickHouse/pull/46070) ([Antonio Andelic](https://github.com/antonio2368)).
* Smallish follow-up to [#46057](https://github.com/ClickHouse/ClickHouse/issues/46057) [#46072](https://github.com/ClickHouse/ClickHouse/pull/46072) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix 00002_log_and_exception_messages_formatting [#46077](https://github.com/ClickHouse/ClickHouse/pull/46077) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Disable temporarily rabbitmq tests which use channel.startConsuming() [#46078](https://github.com/ClickHouse/ClickHouse/pull/46078) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Update yarn packages for dev branches [#46079](https://github.com/ClickHouse/ClickHouse/pull/46079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Simplify code around storages s3/hudi/delta-lake [#46083](https://github.com/ClickHouse/ClickHouse/pull/46083) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix build with `-DENABLE_LIBURING=0` (or `-DENABLE_LIBRARIES=0`) [#46088](https://github.com/ClickHouse/ClickHouse/pull/46088) ([Robert Schulze](https://github.com/rschu1ze)).
* Add also last messages from stdout/stderr/debuglog in clickhouse-test [#46090](https://github.com/ClickHouse/ClickHouse/pull/46090) ([Azat Khuzhin](https://github.com/azat)).
* Sanity assertions for closing file descriptors [#46091](https://github.com/ClickHouse/ClickHouse/pull/46091) ([Azat Khuzhin](https://github.com/azat)).
* Fix flaky rabbitmq test [#46107](https://github.com/ClickHouse/ClickHouse/pull/46107) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix test_merge_tree_azure_blob_storage::test_zero_copy_replication test [#46108](https://github.com/ClickHouse/ClickHouse/pull/46108) ([Azat Khuzhin](https://github.com/azat)).
* allow_drop_detached requires an argument [#46110](https://github.com/ClickHouse/ClickHouse/pull/46110) ([Sema Checherinda](https://github.com/CheSema)).
* Fix fault injection in copier and test_cluster_copier flakiness [#46120](https://github.com/ClickHouse/ClickHouse/pull/46120) ([Azat Khuzhin](https://github.com/azat)).
* Update liburing CMakeLists.txt [#46127](https://github.com/ClickHouse/ClickHouse/pull/46127) ([Nikolay Degterinsky](https://github.com/evillique)).
* Use BAD_ARGUMENTS over LOGICAL_ERROR for schema inference error file() over fd [#46132](https://github.com/ClickHouse/ClickHouse/pull/46132) ([Azat Khuzhin](https://github.com/azat)).
* Stricter warnings + fix whitespaces in poco [#46133](https://github.com/ClickHouse/ClickHouse/pull/46133) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix dependency checks [#46138](https://github.com/ClickHouse/ClickHouse/pull/46138) ([Vitaly Baranov](https://github.com/vitlibar)).
* Interpret `cluster_name` identifier in `s3Cluster` function as literal [#46143](https://github.com/ClickHouse/ClickHouse/pull/46143) ([Nikolay Degterinsky](https://github.com/evillique)).
* Remove flaky test [#46149](https://github.com/ClickHouse/ClickHouse/pull/46149) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix spelling + duplicate includes in poco [#46155](https://github.com/ClickHouse/ClickHouse/pull/46155) ([Robert Schulze](https://github.com/rschu1ze)).
* Add 00002_log_and_exception_messages_formatting back [#46156](https://github.com/ClickHouse/ClickHouse/pull/46156) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix clickhouse/clickhouse-server description to make it in sync [#46159](https://github.com/ClickHouse/ClickHouse/pull/46159) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Complain about missing Yasm at configure time at of build time [#46162](https://github.com/ClickHouse/ClickHouse/pull/46162) ([Robert Schulze](https://github.com/rschu1ze)).
* Update Dockerfile.ubuntu [#46173](https://github.com/ClickHouse/ClickHouse/pull/46173) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Cleanup disk unittest [#46179](https://github.com/ClickHouse/ClickHouse/pull/46179) ([Sergei Trifonov](https://github.com/serxa)).
* Update 01513_optimize_aggregation_in_order_memory_long.sql [#46180](https://github.com/ClickHouse/ClickHouse/pull/46180) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Make a bug in HTTP interface less annoying [#46183](https://github.com/ClickHouse/ClickHouse/pull/46183) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* fix typo [#46207](https://github.com/ClickHouse/ClickHouse/pull/46207) ([Sergei Trifonov](https://github.com/serxa)).
* increase a time gap between insert and ttl move [#46233](https://github.com/ClickHouse/ClickHouse/pull/46233) ([Sema Checherinda](https://github.com/CheSema)).
* Make `test_replicated_merge_tree_s3_restore` less flaky [#46242](https://github.com/ClickHouse/ClickHouse/pull/46242) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix test_distributed_ddl_parallel [#46243](https://github.com/ClickHouse/ClickHouse/pull/46243) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Update 00564_versioned_collapsing_merge_tree.sql [#46245](https://github.com/ClickHouse/ClickHouse/pull/46245) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Optimize docker binary-builder [#46246](https://github.com/ClickHouse/ClickHouse/pull/46246) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update Curl to 7.87.0 [#46248](https://github.com/ClickHouse/ClickHouse/pull/46248) ([Boris Kuschel](https://github.com/bkuschel)).
* Upgrade libxml2 to address CVE-2022-40303 CVE-2022-40304 [#46249](https://github.com/ClickHouse/ClickHouse/pull/46249) ([larryluogit](https://github.com/larryluogit)).
* Run clang-format over poco [#46259](https://github.com/ClickHouse/ClickHouse/pull/46259) ([Robert Schulze](https://github.com/rschu1ze)).
* Suppress "Container already exists" in BC check [#46260](https://github.com/ClickHouse/ClickHouse/pull/46260) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix failure description for hung check [#46267](https://github.com/ClickHouse/ClickHouse/pull/46267) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add upcoming Events [#46271](https://github.com/ClickHouse/ClickHouse/pull/46271) ([Tyler Hannan](https://github.com/tylerhannan)).
* coordination: do not allow election_timeout_lower_bound_ms > election_timeout_upper_bound_ms [#46274](https://github.com/ClickHouse/ClickHouse/pull/46274) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* fix data race between check table request and background checker [#46278](https://github.com/ClickHouse/ClickHouse/pull/46278) ([Sema Checherinda](https://github.com/CheSema)).
* Try to make 02346_full_text_search less flaky [#46279](https://github.com/ClickHouse/ClickHouse/pull/46279) ([Robert Schulze](https://github.com/rschu1ze)).
* Beter diagnostics from http in clickhouse-test [#46281](https://github.com/ClickHouse/ClickHouse/pull/46281) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add more logging to RabbitMQ (to help debug tests) [#46283](https://github.com/ClickHouse/ClickHouse/pull/46283) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix window view test [#46285](https://github.com/ClickHouse/ClickHouse/pull/46285) ([Kseniia Sumarokova](https://github.com/kssenii)).
* suppressing test inaccuracy 00738_lock_for_inner_table [#46287](https://github.com/ClickHouse/ClickHouse/pull/46287) ([Sema Checherinda](https://github.com/CheSema)).
* Simplify ATTACH MergeTree table FROM S3 in tests [#46288](https://github.com/ClickHouse/ClickHouse/pull/46288) ([Azat Khuzhin](https://github.com/azat)).
* Update RabbitMQProducer.cpp [#46295](https://github.com/ClickHouse/ClickHouse/pull/46295) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix macOs compilation due to sprintf [#46298](https://github.com/ClickHouse/ClickHouse/pull/46298) ([Jordi Villar](https://github.com/jrdi)).
* Slightly improve error message for required Yasm assembler [#46328](https://github.com/ClickHouse/ClickHouse/pull/46328) ([Robert Schulze](https://github.com/rschu1ze)).
* Unifdef unused parts of poco [#46329](https://github.com/ClickHouse/ClickHouse/pull/46329) ([Robert Schulze](https://github.com/rschu1ze)).
* Trigger automerge on approved PRs [#46332](https://github.com/ClickHouse/ClickHouse/pull/46332) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Wait for background tasks in ~UploadHelper [#46334](https://github.com/ClickHouse/ClickHouse/pull/46334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix flaky test_storage_rabbitmq::test_rabbitmq_address [#46337](https://github.com/ClickHouse/ClickHouse/pull/46337) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Extract common logic for S3 [#46339](https://github.com/ClickHouse/ClickHouse/pull/46339) ([Antonio Andelic](https://github.com/antonio2368)).
* Update cluster.py [#46340](https://github.com/ClickHouse/ClickHouse/pull/46340) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Try to stabilize test 02346_full_text_search.sql [#46344](https://github.com/ClickHouse/ClickHouse/pull/46344) ([Robert Schulze](https://github.com/rschu1ze)).
* Remove an unused argument [#46346](https://github.com/ClickHouse/ClickHouse/pull/46346) ([Alexander Tokmakov](https://github.com/tavplubix)).
* fix candidate selection [#46347](https://github.com/ClickHouse/ClickHouse/pull/46347) ([Sema Checherinda](https://github.com/CheSema)).
* Do not pollute logs in clickhouse-test [#46361](https://github.com/ClickHouse/ClickHouse/pull/46361) ([Azat Khuzhin](https://github.com/azat)).
* Do not continue perf tests in case of exception in create_query/fill_query [#46362](https://github.com/ClickHouse/ClickHouse/pull/46362) ([Azat Khuzhin](https://github.com/azat)).
* Minor fix in files locating for Bugfix validate check [#46368](https://github.com/ClickHouse/ClickHouse/pull/46368) ([Vladimir C](https://github.com/vdimir)).
* Temporarily disable test_rabbitmq_overloaded_insert [#46403](https://github.com/ClickHouse/ClickHouse/pull/46403) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix test test_rabbitmq_overloaded_insert [#46404](https://github.com/ClickHouse/ClickHouse/pull/46404) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix stress test [#46405](https://github.com/ClickHouse/ClickHouse/pull/46405) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix stress tests statuses [#46406](https://github.com/ClickHouse/ClickHouse/pull/46406) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Follow-up to [#46168](https://github.com/ClickHouse/ClickHouse/issues/46168) [#46409](https://github.com/ClickHouse/ClickHouse/pull/46409) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix noisy log messages [#46410](https://github.com/ClickHouse/ClickHouse/pull/46410) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Docs: Clarify query parameters [#46419](https://github.com/ClickHouse/ClickHouse/pull/46419) ([Robert Schulze](https://github.com/rschu1ze)).
* Make tests with window view less bad [#46421](https://github.com/ClickHouse/ClickHouse/pull/46421) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Move MongoDB and PostgreSQL sources to Sources folder [#46422](https://github.com/ClickHouse/ClickHouse/pull/46422) ([Nikolay Degterinsky](https://github.com/evillique)).
* Another fix for cluster copier [#46433](https://github.com/ClickHouse/ClickHouse/pull/46433) ([Antonio Andelic](https://github.com/antonio2368)).
* Update version_date.tsv and changelogs after v22.3.18.37-lts [#46436](https://github.com/ClickHouse/ClickHouse/pull/46436) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Fix a backup test [#46449](https://github.com/ClickHouse/ClickHouse/pull/46449) ([Vitaly Baranov](https://github.com/vitlibar)).
* Do not fetch submodules in release.py [#46450](https://github.com/ClickHouse/ClickHouse/pull/46450) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* resolve race in getCSNAndAssert [#46452](https://github.com/ClickHouse/ClickHouse/pull/46452) ([Sema Checherinda](https://github.com/CheSema)).
* move database credential inputs to the center on initial load [#46455](https://github.com/ClickHouse/ClickHouse/pull/46455) ([Kevin Zhang](https://github.com/Kinzeng)).
* Improve install_check.py [#46458](https://github.com/ClickHouse/ClickHouse/pull/46458) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Change logging level of a verbose message to Trace [#46459](https://github.com/ClickHouse/ClickHouse/pull/46459) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Analyzer planner fixes before enable by default [#46471](https://github.com/ClickHouse/ClickHouse/pull/46471) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix some flaky integration tests [#46478](https://github.com/ClickHouse/ClickHouse/pull/46478) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Allow to override host for client connection credentials [#46480](https://github.com/ClickHouse/ClickHouse/pull/46480) ([Azat Khuzhin](https://github.com/azat)).
* Try fix flaky test test_parallel_distributed_insert_select_with_schema_inference [#46488](https://github.com/ClickHouse/ClickHouse/pull/46488) ([Kruglov Pavel](https://github.com/Avogar)).
* Planner filter push down optimization fix [#46494](https://github.com/ClickHouse/ClickHouse/pull/46494) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix 01161_all_system_tables test flakiness [#46499](https://github.com/ClickHouse/ClickHouse/pull/46499) ([Azat Khuzhin](https://github.com/azat)).
* Compress tar archives with zstd in intergration tests [#46516](https://github.com/ClickHouse/ClickHouse/pull/46516) ([Azat Khuzhin](https://github.com/azat)).
* chore: bump testcontainers-go to 0.18.0 [#46518](https://github.com/ClickHouse/ClickHouse/pull/46518) ([Manuel de la Peña](https://github.com/mdelapenya)).
* Rollback unnecessary sync because of checking exit code [#46520](https://github.com/ClickHouse/ClickHouse/pull/46520) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix stress test [#46521](https://github.com/ClickHouse/ClickHouse/pull/46521) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add myrrc to trusted contributors [#46526](https://github.com/ClickHouse/ClickHouse/pull/46526) ([Anton Popov](https://github.com/CurtizJ)).
* fix style [#46530](https://github.com/ClickHouse/ClickHouse/pull/46530) ([flynn](https://github.com/ucasfl)).
* Autoupdate keeper dockerfile [#46535](https://github.com/ClickHouse/ClickHouse/pull/46535) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fixes for OpenSSL and s390x [#46546](https://github.com/ClickHouse/ClickHouse/pull/46546) ([Boris Kuschel](https://github.com/bkuschel)).
* enable async-insert-max-query-number only if async_insert_deduplicate [#46549](https://github.com/ClickHouse/ClickHouse/pull/46549) ([Han Fei](https://github.com/hanfei1991)).
* Remove extra try/catch for QueryState/LocalQueryState reset [#46552](https://github.com/ClickHouse/ClickHouse/pull/46552) ([Azat Khuzhin](https://github.com/azat)).
* Whitespaces [#46553](https://github.com/ClickHouse/ClickHouse/pull/46553) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* fix build without avro [#46554](https://github.com/ClickHouse/ClickHouse/pull/46554) ([flynn](https://github.com/ucasfl)).
* Inhibit randomization in test `01551_mergetree_read_in_order_spread.sql` [#46562](https://github.com/ClickHouse/ClickHouse/pull/46562) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove PVS-Studio [#46565](https://github.com/ClickHouse/ClickHouse/pull/46565) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Inhibit settings randomization in `01304_direct_io_long.sh` [#46566](https://github.com/ClickHouse/ClickHouse/pull/46566) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix double whitespace in comment in test [#46567](https://github.com/ClickHouse/ClickHouse/pull/46567) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Rename test [#46568](https://github.com/ClickHouse/ClickHouse/pull/46568) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix ASTAsterisk::clone() [#46570](https://github.com/ClickHouse/ClickHouse/pull/46570) ([Nikolay Degterinsky](https://github.com/evillique)).
* Small update of sparkbar docs [#46579](https://github.com/ClickHouse/ClickHouse/pull/46579) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix flakiness of expect tests for clickhouse-client by avoiding history overlap [#46582](https://github.com/ClickHouse/ClickHouse/pull/46582) ([Azat Khuzhin](https://github.com/azat)).
* Always log rollback for release.py [#46586](https://github.com/ClickHouse/ClickHouse/pull/46586) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Increase table retries in `test_cluster_copier` [#46590](https://github.com/ClickHouse/ClickHouse/pull/46590) ([Antonio Andelic](https://github.com/antonio2368)).
* Update 00170_s3_cache.sql [#46593](https://github.com/ClickHouse/ClickHouse/pull/46593) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix rabbitmq test [#46595](https://github.com/ClickHouse/ClickHouse/pull/46595) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix meilisearch test flakyness [#46596](https://github.com/ClickHouse/ClickHouse/pull/46596) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update compare.sh [#46599](https://github.com/ClickHouse/ClickHouse/pull/46599) ([Alexander Tokmakov](https://github.com/tavplubix)).
* update llvm-project to fix gwp-asan [#46600](https://github.com/ClickHouse/ClickHouse/pull/46600) ([Han Fei](https://github.com/hanfei1991)).
* Temporarily disable test_rabbitmq_overloaded_insert [#46608](https://github.com/ClickHouse/ClickHouse/pull/46608) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Update 01565_reconnect_after_client_error to not expect explicit reconnect [#46619](https://github.com/ClickHouse/ClickHouse/pull/46619) ([Azat Khuzhin](https://github.com/azat)).
* Inhibit `index_granularity_bytes` randomization in some tests [#46626](https://github.com/ClickHouse/ClickHouse/pull/46626) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix coverity [#46629](https://github.com/ClickHouse/ClickHouse/pull/46629) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix 01179_insert_values_semicolon test [#46636](https://github.com/ClickHouse/ClickHouse/pull/46636) ([Azat Khuzhin](https://github.com/azat)).
* Fix typo in read prefetch [#46640](https://github.com/ClickHouse/ClickHouse/pull/46640) ([Nikita Taranov](https://github.com/nickitat)).
* Avoid OOM in perf tests [#46641](https://github.com/ClickHouse/ClickHouse/pull/46641) ([Azat Khuzhin](https://github.com/azat)).
* Fix: remove redundant sorting optimization [#46642](https://github.com/ClickHouse/ClickHouse/pull/46642) ([Igor Nikonov](https://github.com/devcrafter)).
* Fix flaky test 01710_normal_projections [#46645](https://github.com/ClickHouse/ClickHouse/pull/46645) ([Kruglov Pavel](https://github.com/Avogar)).
* Update postgres_utility.py [#46656](https://github.com/ClickHouse/ClickHouse/pull/46656) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix integration test: terminate old version without wait [#46660](https://github.com/ClickHouse/ClickHouse/pull/46660) ([alesapin](https://github.com/alesapin)).
* Break Stress tests [#46663](https://github.com/ClickHouse/ClickHouse/pull/46663) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* fix layout issues in dashboard.html [#46671](https://github.com/ClickHouse/ClickHouse/pull/46671) ([Kevin Zhang](https://github.com/Kinzeng)).
* Fix Stress tests [#46683](https://github.com/ClickHouse/ClickHouse/pull/46683) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Disable flaky test_ttl_move_memory_usage.py [#46687](https://github.com/ClickHouse/ClickHouse/pull/46687) ([Alexander Tokmakov](https://github.com/tavplubix)).
* BackgroundSchedulePool should not have any query context [#46709](https://github.com/ClickHouse/ClickHouse/pull/46709) ([Azat Khuzhin](https://github.com/azat)).
* Better exception message during Tuple JSON deserialization [#46727](https://github.com/ClickHouse/ClickHouse/pull/46727) ([Kruglov Pavel](https://github.com/Avogar)).
* Poco: POCO_HAVE_INT64 is always defined [#46728](https://github.com/ClickHouse/ClickHouse/pull/46728) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix SonarCloud Job [#46732](https://github.com/ClickHouse/ClickHouse/pull/46732) ([Julio Jimenez](https://github.com/juliojimenez)).
* Remove unused MergeTreeReadTask::remove_prewhere_column [#46744](https://github.com/ClickHouse/ClickHouse/pull/46744) ([Alexander Gololobov](https://github.com/davenger)).
* On out-of-space `at` returns error, we must terminate still [#46754](https://github.com/ClickHouse/ClickHouse/pull/46754) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* CI: don't run builds/tests when CHANGELOG.md or README.md were modified [#46773](https://github.com/ClickHouse/ClickHouse/pull/46773) ([Robert Schulze](https://github.com/rschu1ze)).
* Cosmetics in hashing code [#46780](https://github.com/ClickHouse/ClickHouse/pull/46780) ([Robert Schulze](https://github.com/rschu1ze)).
#### Testing Improvement
* Fixed functional test 00304_http_external_data for s390x. [#45807](https://github.com/ClickHouse/ClickHouse/pull/45807) ([Harry Lee](https://github.com/HarryLeeIBM)).

View File

@ -39,12 +39,59 @@ To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` director
Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables.
### Restricting test runs
A test can have zero or more _test tags_ specifying restrictions for test runs.
For `.sh` tests tags are written as a comment on the second line:
```bash
#!/usr/bin/env bash
# Tags: no-fasttest
```
For `.sql` tests tags are placed in the first line as a SQL comment:
```sql
-- Tags: no-fasttest
SELECT 1
```
|Tag name | What it does | Usage example |
|---|---|---|
| `disabled`| Test is not run ||
| `long` | Test's execution time is extended from 1 to 10 minutes ||
| `deadlock` | Test is run in a loop for a long time ||
| `race` | Same as `deadlock`. Prefer `deadlock` ||
| `shard` | Server is required to listen to `127.0.0.*` ||
| `distributed` | Same as `shard`. Prefer `shard` ||
| `global` | Same as `shard`. Prefer `shard` ||
| `zookeeper` | Test requires Zookeeper or ClickHouse Keeper to run | Test uses `ReplicatedMergeTree` |
| `replica` | Same as `zookeeper`. Prefer `zookeeper` ||
| `no-fasttest`| Test is not run under [Fast test](continuous-integration#fast-test) | Test uses `MySQL` table engine which is disabled in Fast test|
| `no-[asan, tsan, msan, ubsan]` | Disables tests in build with [sanitizers](#sanitizers) | Test is run under QEMU which doesn't work with sanitizers |
| `no-replicated-database` |||
| `no-ordinary-database` |||
| `no-parallel` | Disables running other tests in parallel with this one | Test reads from `system` tables and invariants may be broken|
| `no-parallel-replicas` |||
| `no-debug` |||
| `no-stress` |||
| `no-polymorphic-parts` |||
| `no-random-settings` |||
| `no-random-merge-tree-settings` |||
| `no-backward-compatibility-check` |||
| `no-cpu-x86_64` |||
| `no-cpu-aarch64` |||
| `no-cpu-ppc64le` |||
| `no-s3-storage` |||
In addition to the above settings, you can use `USE_*` flags from `system.build_options` to define usage of particular ClickHouse features.
For example, if your test uses a MySQL table, you should add a tag `use-mysql`.
### Choosing the Test Name
The name of the test starts with a five-digit prefix followed by a descriptive name, such as `00422_hash_function_constexpr.sql`. To choose the prefix, find the largest prefix already present in the directory, and increment it by one. In the meantime, some other tests might be added with the same numeric prefix, but this is OK and does not lead to any problems, you don't have to change it later.
Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper. `shard` is for tests that requires server to listen `127.0.0.*`; `distributed` or `global` have the same meaning. `long` is for tests that run slightly longer that one second. You can disable these groups of tests using `--no-zookeeper`, `--no-shard` and `--no-long` options, respectively. Make sure to add a proper prefix to your test name if it needs ZooKeeper or distributed queries.
### Checking for an Error that Must Occur
Sometimes you want to test that a server error occurs for an incorrect query. We support special annotations for this in SQL tests, in the following form:

View File

@ -19,7 +19,9 @@ CREATE TABLE deltalake
**Engine parameters**
- `url` — Bucket url with path to the existing Delta Lake table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
**Example**
@ -27,7 +29,24 @@ CREATE TABLE deltalake
CREATE TABLE deltalake ENGINE=DeltaLake('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123')
```
Using named collections:
``` xml
<clickhouse>
<named_collections>
<deltalake_conf>
<url>http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/</url>
<access_key_id>ABC123<access_key_id>
<secret_access_key>Abc+123</secret_access_key>
</deltalake_conf>
</named_collections>
</clickhouse>
```
```sql
CREATE TABLE deltalake ENGINE=DeltaLake(deltalake_conf, filename = 'test_table')
```
## See also
- [deltaLake table function](../../../sql-reference/table-functions/deltalake.md)

View File

@ -19,7 +19,9 @@ CREATE TABLE hudi_table
**Engine parameters**
- `url` — Bucket url with the path to an existing Hudi table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
**Example**
@ -27,7 +29,24 @@ CREATE TABLE hudi_table
CREATE TABLE hudi_table ENGINE=Hudi('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123')
```
Using named collections:
``` xml
<clickhouse>
<named_collections>
<hudi_conf>
<url>http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/</url>
<access_key_id>ABC123<access_key_id>
<secret_access_key>Abc+123</secret_access_key>
</hudi_conf>
</named_collections>
</clickhouse>
```
```sql
CREATE TABLE hudi_table ENGINE=Hudi(hudi_conf, filename = 'test_table')
```
## See also
- [hudi table function](/docs/en/sql-reference/table-functions/hudi.md)

View File

@ -0,0 +1,52 @@
---
slug: /en/engines/table-engines/integrations/iceberg
sidebar_label: Iceberg
---
# Iceberg Table Engine
This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3.
## Create Table
Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table.
``` sql
CREATE TABLE iceberg_table
ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,])
```
**Engine parameters**
- `url` — url with the path to an existing Iceberg table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
**Example**
```sql
CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test')
```
Using named collections:
``` xml
<clickhouse>
<named_collections>
<iceberg_conf>
<url>http://test.s3.amazonaws.com/clickhouse-bucket/</url>
<access_key_id>test<access_key_id>
<secret_access_key>test</secret_access_key>
</iceberg_conf>
</named_collections>
</clickhouse>
```
```sql
CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table')
```
## See also
- [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md)

View File

@ -162,22 +162,59 @@ If you want to change the target table by using `ALTER`, we recommend disabling
## Configuration {#configuration}
Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (`kafka`) and topic-level (`kafka_*`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists).
Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (below `<kafka>`) and topic-level (below `<kafka><kafka_topic>`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists).
``` xml
<!-- Global configuration options for all tables of Kafka engine type -->
<kafka>
<!-- Global configuration options for all tables of Kafka engine type -->
<debug>cgrp</debug>
<auto_offset_reset>smallest</auto_offset_reset>
<!-- Configuration specific to topics "logs" and "stats" -->
<kafka_topic>
<name>logs</name>
<retry_backoff_ms>250</retry_backoff_ms>
<fetch_min_bytes>100000</fetch_min_bytes>
</kafka_topic>
<kafka_topic>
<name>stats</name>
<retry_backoff_ms>400</retry_backoff_ms>
<fetch_min_bytes>50000</fetch_min_bytes>
</kafka_topic>
</kafka>
```
<details markdown="1">
<summary>Example in deprecated syntax</summary>
``` xml
<kafka>
<!-- Global configuration options for all tables of Kafka engine type -->
<debug>cgrp</debug>
<auto_offset_reset>smallest</auto_offset_reset>
</kafka>
<!-- Configuration specific for topic "logs" -->
<!-- Configuration specific to topics "logs" and "stats" -->
<!-- Does NOT support periods in topic names, e.g. "logs.security"> -->
<kafka_logs>
<retry_backoff_ms>250</retry_backoff_ms>
<fetch_min_bytes>100000</fetch_min_bytes>
</kafka_logs>
<kafka_stats>
<retry_backoff_ms>400</retry_backoff_ms>
<fetch_min_bytes>50000</fetch_min_bytes>
</kafka_stats>
```
</details>
For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `<check_crcs>true</check_crcs>`.
### Kerberos support {#kafka-kerberos-support}

View File

@ -1,6 +1,6 @@
# Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex}
The main task that indexes achieve is to quickly find nearest neighbors for multidimensional data. An example of such a problem can be finding similar pictures (texts) for a given picture (text). That problem can be reduced to finding the nearest [embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning). They can be created from data using [UDF](../../../sql-reference/functions/index.md#executable-user-defined-functions).
The main task that indexes achieve is to quickly find nearest neighbors for multidimensional data. An example of such a problem can be finding similar pictures (texts) for a given picture (text). That problem can be reduced to finding the nearest [embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning). They can be created from data using [UDF](/docs/en/sql-reference/functions/index.md/#executable-user-defined-functions).
The next queries find the closest neighbors in N-dimensional space using the L2 (Euclidean) distance:
``` sql
@ -39,7 +39,7 @@ Approximate Nearest Neighbor Search Indexes (`ANNIndexes`) are similar to skip i
LIMIT N
```
In these queries, `DistanceFunction` is selected from [distance functions](../../../sql-reference/functions/distance-functions). `Point` is a known vector (something like `(0.1, 0.1, ... )`). To avoid writing large vectors, use [client parameters](../../../interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters). `Value` - a float value that will bound the neighbourhood.
In these queries, `DistanceFunction` is selected from [distance functions](/docs/en/sql-reference/functions/distance-functions.md). `Point` is a known vector (something like `(0.1, 0.1, ... )`). To avoid writing large vectors, use [client parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters). `Value` - a float value that will bound the neighbourhood.
:::note
ANN index can't speed up query that satisfies both types (`where + order by`, only one of them). All queries must have the limit, as algorithms are used to find nearest neighbors and need a specific number of them.
@ -85,13 +85,13 @@ As the indexes are built only during insertions into table, `INSERT` and `OPTIMI
You can create your table with index which uses certain algorithm. Now only indices based on the following algorithms are supported:
# Index list
- [Annoy](../../../engines/table-engines/mergetree-family/annindexes.md#annoy-annoy)
- [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy)
# Annoy {#annoy}
Implementation of the algorithm was taken from [this repository](https://github.com/spotify/annoy).
Short description of the algorithm:
The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D e.t.c.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest.
The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest.
To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points.
__Examples__:
@ -118,7 +118,7 @@ ORDER BY id;
```
:::note
Table with array field will work faster, but all arrays **must** have same length. Use [CONSTRAINT](../../../sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(data) = 256`.
Table with array field will work faster, but all arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(data) = 256`.
:::
Parameter `NumTrees` is the number of trees which the algorithm will create. The bigger it is, the slower (approximately linear) it works (in both `CREATE` and `SELECT` requests), but the better accuracy you get (adjusted for randomness). By default it is set to `100`. Parameter `DistanceName` is name of distance function. By default it is set to `L2Distance`. It can be set without changing first parameter, for example

View File

@ -5,6 +5,10 @@ description: Install ClickHouse
slug: /en/install
---
import Tabs from '@theme/Tabs';
import TabItem from '@theme/TabItem';
import CodeBlock from '@theme/CodeBlock';
# Install ClickHouse
You have three options for getting up and running with ClickHouse:
@ -19,17 +23,27 @@ The quickest and easiest way to get up and running with ClickHouse is to create
## Self-Managed Install
:::tip
For production installs of a specific release version see the [installation options](#available-installation-options) down below.
:::
<Tabs>
<TabItem value="linux" label="Linux" default>
1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable:
```bash
curl https://clickhouse.com/ | sh
```
1. Run the `install` command, which defines a collection of useful symlinks along with the files and folders used by ClickHouse - all of which you can see in the output of the install script:
```bash
sudo ./clickhouse install
```
1. At the end of the install script, you are prompted for a password for the `default` user. Feel free to enter a password, or you can optionally leave it blank:
```response
Creating log directory /var/log/clickhouse-server.
Creating data directory /var/lib/clickhouse.
@ -40,6 +54,7 @@ The quickest and easiest way to get up and running with ClickHouse is to create
Enter password for default user:
```
You should see the following output:
```response
ClickHouse has been successfully installed.
@ -51,10 +66,45 @@ The quickest and easiest way to get up and running with ClickHouse is to create
```
1. Run the following command to start the ClickHouse server:
```bash
sudo clickhouse start
```
</TabItem>
<TabItem value="macos" label="macOS">
1. The simplest way to download ClickHouse locally is to run the following command. If your operating system is supported, an appropriate ClickHouse binary will be downloaded and made runnable:
```bash
sudo clickhouse start
curl https://clickhouse.com/ | sh
```
1. Run the ClickHouse server:
```bash
./clickhouse server
```
1. Open a new terminal and use the **clickhouse-client** to connect to your service:
```bash
./clickhouse client
```
```response
./clickhouse client
ClickHouse client version 23.2.1.1501 (official build).
Connecting to localhost:9000 as user default.
Connected to ClickHouse server version 23.2.1 revision 54461.
local-host :)
```
You are ready to start sending DDL and SQL commands to ClickHouse!
</TabItem>
</Tabs>
:::tip
The [Quick Start](/docs/en/quick-start.mdx/#step-1-get-clickhouse) walks through the steps to download and run ClickHouse, connect to it, and insert data.
:::

View File

@ -1971,7 +1971,8 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t
- [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`.
- [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`.
- [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`.
- [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`.
- [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`.
- [output_format_parquet_version](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_version) - The version of Parquet format used in output format. Default value - `2.latest`.
## Arrow {#data-format-arrow}

View File

@ -229,7 +229,7 @@ To prevent inferring the same schema every time ClickHouse read the data from th
There are special settings that control this cache:
- `schema_inference_cache_max_elements_for_{file/s3/hdfs/url}` - the maximum number of cached schemas for the corresponding table function. The default value is `4096`. These settings should be set in the server config.
- `use_cache_for_{file,s3,hdfs,url}_schema_inference` - allows turning on/off using cache for schema inference. These settings can be used in queries.
- `schema_inference_use_cache_for_{file,s3,hdfs,url}` - allows turning on/off using cache for schema inference. These settings can be used in queries.
The schema of the file can be changed by modifying the data or by changing format settings.
For this reason, the schema inference cache identifies the schema by file source, format name, used format settings, and the last modification time of the file.
@ -1177,8 +1177,7 @@ This setting can be used to specify the types of columns that could not be deter
**Example**
```sql
DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}'
SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)'
DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1
```
```response
┌─name────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐

View File

@ -467,7 +467,7 @@ Port for exchanging data between ClickHouse servers.
The hostname that can be used by other servers to access this server.
If omitted, it is defined in the same way as the `hostname-f` command.
If omitted, it is defined in the same way as the `hostname -f` command.
Useful for breaking away from a specific network interface.

View File

@ -50,7 +50,7 @@ If there are multiple profiles active for a user, then constraints are merged. M
Read-only mode is enabled by `readonly` setting (not to confuse with `readonly` constraint type):
- `readonly=0`: No read-only restrictions.
- `readonly=1`: Only read queries are allowed and settings cannot be changes unless `changeable_in_readonly` is set.
- `readonly=1`: Only read queries are allowed and settings cannot be changed unless `changeable_in_readonly` is set.
- `readonly=2`: Only read queries are allowed, but settings can be changed, except for `readonly` setting itself.

View File

@ -15,11 +15,12 @@ When writing data, ClickHouse throws an exception if input data contain columns
Supported formats:
- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow)
- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) (and other JSON formats)
- [BSONEachRow](../../interfaces/formats.md/#bsoneachrow) (and other JSON formats)
- [TSKV](../../interfaces/formats.md/#tskv)
- All formats with suffixes WithNames/WithNamesAndTypes
- [JSONColumns](../../interfaces/formats.md/#jsoncolumns)
- [MySQLDump](../../interfaces/formats.md/#mysqldump)
- [Native](../../interfaces/formats.md/#native)
Possible values:
@ -78,7 +79,7 @@ Default value: 1.
## input_format_defaults_for_omitted_fields {#input_format_defaults_for_omitted_fields}
When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes.
When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) (and other JSON formats), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated), [TSKV](../../interfaces/formats.md/#tskv), [Parquet](../../interfaces/formats.md/#parquet), [Arrow](../../interfaces/formats.md/#arrow), [Avro](../../interfaces/formats.md/#avro), [ORC](../../interfaces/formats.md/#orc), [Native](../../interfaces/formats.md/#native) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes.
:::note
When this option is enabled, extended table metadata are sent from server to client. It consumes additional computing resources on the server and can reduce performance.
@ -96,7 +97,9 @@ Default value: 1.
Enables or disables the initialization of [NULL](../../sql-reference/syntax.md/#null-literal) fields with [default values](../../sql-reference/statements/create/table.md/#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable).
If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting.
This setting is applicable to [INSERT ... VALUES](../../sql-reference/statements/insert-into.md) queries for text input formats.
This setting is applicable for most input formats.
For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too.
Possible values:
@ -139,6 +142,10 @@ y Nullable(String)
z IPv4
```
:::warning
If the `schema_inference_hints` is not formated properly, or if there is a typo or a wrong datatype, etc... the whole schema_inference_hints will be ignored.
:::
## schema_inference_make_columns_nullable {#schema_inference_make_columns_nullable}
Controls making inferred types `Nullable` in schema inference for formats without information about nullability.
@ -504,7 +511,7 @@ Enabled by default.
Ignore unknown keys in json object for named tuples.
Disabled by default.
Enabled by default.
## input_format_json_defaults_for_missing_elements_in_named_tuple {#input_format_json_defaults_for_missing_elements_in_named_tuple}
@ -1099,6 +1106,12 @@ Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedStrin
Enabled by default.
### output_format_parquet_version {#output_format_parquet_version}
The version of Parquet format used in output format. Supported versions: `1.0`, `2.4`, `2.6` and `2.latest`.
Default value: `2.latest`.
## Hive format settings {#hive-format-settings}
### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter}

View File

@ -851,6 +851,15 @@ Result:
└─────────────┴───────────┘
```
## log_processors_profiles {#settings-log_processors_profiles}
Write time that processor spent during execution/waiting for data to `system.processors_profile_log` table.
See also:
- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#system-processors_profile_log)
- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)
## max_insert_block_size {#settings-max_insert_block_size}
The size of blocks (in a count of rows) to form for insertion into a table.
@ -3939,3 +3948,71 @@ Default value: `0`.
:::note
Use this setting only for backward compatibility if your use cases depend on old syntax.
:::
## final {#final}
Automatically applies [FINAL](../../sql-reference/statements/select/from/#final-modifier) modifier to all tables in a query, to tables where [FINAL](../../sql-reference/statements/select/from/#final-modifier) is applicable, including joined tables and tables in sub-queries, and
distributed tables.
Possible values:
- 0 - disabled
- 1 - enabled
Default value: `0`.
Example:
```sql
CREATE TABLE test
(
key Int64,
some String
)
ENGINE = ReplacingMergeTree
ORDER BY key;
INSERT INTO test FORMAT Values (1, 'first');
INSERT INTO test FORMAT Values (1, 'second');
SELECT * FROM test;
┌─key─┬─some───┐
│ 1 │ second │
└─────┴────────┘
┌─key─┬─some──┐
│ 1 │ first │
└─────┴───────┘
SELECT * FROM test SETTINGS final = 1;
┌─key─┬─some───┐
│ 1 │ second │
└─────┴────────┘
SET final = 1;
SELECT * FROM test;
┌─key─┬─some───┐
│ 1 │ second │
└─────┴────────┘
```
## asterisk_include_materialized_columns {#asterisk_include_materialized_columns}
Include [MATERIALIZED](../../sql-reference/statements/create/table/#materialized) columns for wildcard query (`SELECT *`).
Possible values:
- 0 - disabled
- 1 - enabled
Default value: `0`.
## asterisk_include_alias_columns {#asterisk_include_alias_columns}
Include [ALIAS](../../sql-reference/statements/create/table/#alias) columns for wildcard query (`SELECT *`).
Possible values:
- 0 - disabled
- 1 - enabled
Default value: `0`.

View File

@ -0,0 +1,76 @@
# system.processors_profile_log {#system-processors_profile_log}
This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)).
Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened.
- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened.
- `id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of processor
- `parent_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Parent processors IDs
- `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query
- `name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the processor.
- `elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was executed.
- `input_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting for data (from other processor).
- `output_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting because output port was full.
- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step.
- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result.
- `input_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows consumed by processor.
- `input_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes consumed by processor.
- `output_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows generated by processor.
- `output_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes generated by processor.
**Example**
Query:
``` sql
EXPLAIN PIPELINE
SELECT sleep(1)
┌─explain─────────────────────────┐
│ (Expression) │
│ ExpressionTransform │
│ (SettingQuotaAndLimits) │
│ (ReadFromStorage) │
│ SourceFromSingleChunk 0 → 1 │
└─────────────────────────────────┘
SELECT sleep(1)
SETTINGS log_processors_profiles = 1
Query id: feb5ed16-1c24-4227-aa54-78c02b3b27d4
┌─sleep(1)─┐
│ 0 │
└──────────┘
1 rows in set. Elapsed: 1.018 sec.
SELECT
name,
elapsed_us,
input_wait_elapsed_us,
output_wait_elapsed_us
FROM system.processors_profile_log
WHERE query_id = 'feb5ed16-1c24-4227-aa54-78c02b3b27d4'
ORDER BY name ASC
```
Result:
``` text
┌─name────────────────────┬─elapsed_us─┬─input_wait_elapsed_us─┬─output_wait_elapsed_us─┐
│ ExpressionTransform │ 1000497 │ 2823 │ 197 │
│ LazyOutputFormat │ 36 │ 1002188 │ 0 │
│ LimitsCheckingTransform │ 10 │ 1002994 │ 106 │
│ NullSource │ 5 │ 1002074 │ 0 │
│ NullSource │ 1 │ 1002084 │ 0 │
│ SourceFromSingleChunk │ 45 │ 4736 │ 1000819 │
└─────────────────────────┴────────────┴───────────────────────┴────────────────────────┘
```
Here you can see:
- `ExpressionTransform` was executing `sleep(1)` function, so it `work` will takes 1e6, and so `elapsed_us` > 1e6.
- `SourceFromSingleChunk` need to wait, because `ExpressionTransform` does not accept any data during execution of `sleep(1)`, so it will be in `PortFull` state for 1e6 us, and so `output_wait_elapsed_us` > 1e6.
- `LimitsCheckingTransform`/`NullSource`/`LazyOutputFormat` need to wait until `ExpressionTransform` will execute `sleep(1)` to process the result, so `input_wait_elapsed_us` > 1e6.
**See Also**
- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)

View File

@ -0,0 +1,52 @@
---
slug: /en/operations/system-tables/server_settings
---
# server_settings
Contains information about global settings for the server, which were specified in `config.xml`.
Currently, the table shows only settings from the first layer of `config.xml` and doesn't support nested configs (e.g. [logger](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-logger)).
Columns:
- `name` ([String](../../sql-reference/data-types/string.md)) — Server setting name.
- `value` ([String](../../sql-reference/data-types/string.md)) — Server setting value.
- `default` ([String](../../sql-reference/data-types/string.md)) — Server setting default value.
- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml`
- `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description.
- `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type.
**Example**
The following example shows how to get information about server settings which name contains `thread_pool`.
``` sql
SELECT *
FROM system.server_settings
WHERE name LIKE '%thread_pool%'
```
``` text
┌─name─────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┐
│ max_thread_pool_size │ 5000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │
│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │
│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │
│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │
│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │
│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │
└──────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┘
```
Using of `WHERE changed` can be useful, for example, when you want to check
whether settings in configuration files are loaded correctly and are in use.
<!-- -->
``` sql
SELECT * FROM system.server_settings WHERE changed AND name='max_thread_pool_size'
```
**See also**
- [Settings](../../operations/system-tables/settings.md)
- [Configuration Files](../../operations/configuration-files.md)
- [Server Settings](../../operations/server-configuration-parameters/settings.md)

View File

@ -16,6 +16,7 @@ Columns:
- `readonly` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting:
- `0` — Current user can change the setting.
- `1` — Current user cant change the setting.
- `default` ([String](../../sql-reference/data-types/string.md)) — Setting default value.
**Example**

View File

@ -28,6 +28,7 @@ Columns:
- `revision` (UInt32) — ClickHouse revision.
- `source_file` (LowCardinality(String)) — Source file from which the logging was done.
- `source_line` (UInt64) — Source line from which the logging was done.
- `message_format_string` (LowCardinality(String)) — A format string that was used to format the message.
**Example**
@ -51,4 +52,5 @@ message: Update period 15 seconds
revision: 54440
source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start()
source_line: 45
message_format_string: Update period {} seconds
```

View File

@ -7,7 +7,7 @@ sidebar_position: 37
Calculates the value of `Σ((x - x̅)(y - y̅)) / (n - 1)`.
Returns Float64. When `n <= 1`, returns +∞.
Returns Float64. When `n <= 1`, returns `nan`.
:::note
This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `covarSampStable` function. It works slower but provides a lower computational error.

View File

@ -48,7 +48,35 @@ When dividing by zero you get inf, -inf, or nan.
## intDiv(a, b)
Calculates the quotient of the numbers. Divides into integers, rounding down (by the absolute value).
An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one.
Returns an integer of the type of the dividend (the first parameter).
An exception is thrown when dividing by zero, when the quotient does not fit in the range of the dividend, or when dividing a minimal negative number by minus one.
**Example**
Query:
```sql
SELECT
intDiv(toFloat64(1), 0.001) AS res,
toTypeName(res)
```
```response
┌──res─┬─toTypeName(intDiv(toFloat64(1), 0.001))─┐
│ 1000 │ Int64 │
└──────┴─────────────────────────────────────────┘
```
```sql
SELECT
intDiv(1, 0.001) AS res,
toTypeName(res)
```
```response
Received exception from server (version 23.2.1):
Code: 153. DB::Exception: Received from localhost:9000. DB::Exception: Cannot perform integer division, because it will produce infinite or too large number: While processing intDiv(1, 0.001) AS res, toTypeName(res). (ILLEGAL_DIVISION)
```
## intDivOrZero(a, b)

View File

@ -579,3 +579,33 @@ Result:
│ 3628800 │
└───────────────┘
```
## width_bucket(operand, low, high, count)
Returns the number of the bucket in which `operand` falls in a histogram having `count` equal-width buckets spanning the range `low` to `high`. Returns `0` if `operand < low`, and returns `count+1` if `operand >= high`.
`operand`, `low`, `high` can be any native number type. `count` can only be unsigned native integer and its value cannot be zero.
**Syntax**
```sql
widthBucket(operand, low, high, count)
```
There is also a case insensitive alias called `WIDTH_BUCKET` to provide compatibility with other databases.
**Example**
Query:
``` sql
SELECT widthBucket(10.15, -8.6, 23, 18);
```
Result:
``` text
┌─widthBucket(10.15, -8.6, 23, 18)─┐
│ 11 │
└──────────────────────────────────┘
```

View File

@ -226,6 +226,17 @@ SELECT splitByNonAlpha(' 1! a, b. ');
Concatenates string representations of values listed in the array with the separator. `separator` is an optional parameter: a constant string, set to an empty string by default.
Returns the string.
**Example**
``` sql
SELECT arrayStringConcat(['12/05/2021', '12:50:00'], ' ') AS DateString;
```
```text
┌─DateString──────────┐
│ 12/05/2021 12:50:00 │
└─────────────────────┘
```
## alphaTokens(s[, max_substrings]), splitByAlpha(s[, max_substrings])
Selects substrings of consecutive bytes from the ranges a-z and A-Z.Returns an array of substrings.

View File

@ -1,7 +1,7 @@
---
slug: /en/sql-reference/functions/string-replace-functions
sidebar_position: 42
sidebar_label: For Replacing in Strings
sidebar_label: Replacing in Strings
---
# Functions for Searching and Replacing in Strings

View File

@ -1,7 +1,7 @@
---
slug: /en/sql-reference/functions/string-search-functions
sidebar_position: 41
sidebar_label: For Searching in Strings
sidebar_label: Searching in Strings
---
# Functions for Searching in Strings
@ -382,12 +382,12 @@ Checks whether string `haystack` matches the regular expression `pattern`. The p
Returns 1 in case of a match, and 0 otherwise.
Matching is based on UTF-8, e.g. `.` matches the Unicode code point `¥` which is represented in UTF-8 using two bytes. The regular expression must not contain null bytes.
If the haystack or pattern contain a sequence of bytes that are not valid UTF-8, the behavior is undefined.
No automatic Unicode normalization is performed, if you need it you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
If the haystack or the pattern are not valid UTF-8, then the behavior is undefined.
No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
Unlike re2's default behavior, `.` matches line breaks. To disable this, prepend the pattern with `(?-s)`.
For patterns to search for substrings in a string, it is better to use LIKE or position, since they work much faster.
For patterns to search for substrings in a string, it is better to use functions [like](#like) or [position](#position) since they work much faster.
## multiMatchAny(haystack, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\])
@ -529,21 +529,25 @@ Result:
## like(haystack, pattern), haystack LIKE pattern operator
Checks whether a string matches a simple regular expression.
The regular expression can contain the metasymbols `%` and `_`.
Checks whether a string matches a LIKE expression.
A LIKE expression contains a mix of normal characters and the following metasymbols:
`%` indicates any quantity of any bytes (including zero characters).
- `%` indicates an arbitrary number of arbitrary characters (including zero characters).
`_` indicates any one byte.
- `_` indicates a single arbitrary character.
Use the backslash (`\`) for escaping metasymbols. See the note on escaping in the description of the match function.
- `\` is for escaping literals `%`, `_` and `\`.
Matching is based on UTF-8, e.g. `_` matches the Unicode code point `¥` which is represented in UTF-8 using two bytes.
If the haystack or pattern contain a sequence of bytes that are not valid UTF-8, then the behavior is undefined.
No automatic Unicode normalization is performed, if you need it you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
If the haystack or the pattern are not valid UTF-8, then the behavior is undefined.
No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
For regular expressions like `%needle%`, the code is more optimal and works as fast as the `position` function.
For other regular expressions, the code is the same as for the match function.
To match against literals `%`, `_` and `/` (which are LIKE metacharacters), prepend them with a backslash, i.e. `\%`, `\_` and `\\`.
The backslash loses its special meaning, i.e. is interpreted literally, if it prepends a character different than `%`, `_` or `\`.
Note that ClickHouse requires backslashes in strings [to be quoted as well](../syntax.md#string), so you would actually need to write `\\%`, `\\_` and `\\\\`.
For patterns of the form `%needle%`, the function is as fast as the `position` function.
Other LIKE expressions are internally converted to a regular expression and executed with a performance similar to function `match`.
## notLike(haystack, pattern), haystack NOT LIKE pattern operator

View File

@ -1,8 +1,8 @@
---
slug: /en/sql-reference/functions/tuple-map-functions
sidebar_position: 46
sidebar_label: Working with maps
title: "Functions for maps"
sidebar_label: Maps
title: "Functions for Maps"
---
## map
@ -440,7 +440,7 @@ mapApply(func, map)
**Parameters**
- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function).
- `func` - [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function).
- `map` — [Map](../../sql-reference/data-types/map.md).
**Returned value**
@ -480,7 +480,7 @@ mapFilter(func, map)
**Parameters**
- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function).
- `func` - [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function).
- `map` — [Map](../../sql-reference/data-types/map.md).
**Returned value**

View File

@ -6,22 +6,23 @@ sidebar_label: Type Conversion
# Type Conversion Functions
## Common Issues of Numeric Conversions
## Common Issues with Data Conversion
When you convert a value from one to another data type, you should remember that if you try to fit a value from a larger data type to a smaller one (for example Int64 to Int32), or convert from one data type to another (for example `String` to `Int`), you could have data loss. Test beforehand.
Be aware of potential data loss if values of a datatype are converted to a smaller datatype (for example from `Int64` to `Int32`) or between
incompatible datatypes (for example from `String` to `Int`). Make sure to check carefully if the result is as expected.
ClickHouse has the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion).
ClickHouse generally uses the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion).
## toInt(8\|16\|32\|64\|128\|256)
Converts an input value to the [Int](/docs/en/sql-reference/data-types/int-uint.md) data type. This function family includes:
Converts an input value to a value the [Int](/docs/en/sql-reference/data-types/int-uint.md) data type. This function family includes:
- `toInt8(expr)`Results in the `Int8` data type.
- `toInt16(expr)`Results in the `Int16` data type.
- `toInt32(expr)`Results in the `Int32` data type.
- `toInt64(expr)`Results in the `Int64` data type.
- `toInt128(expr)`Results in the `Int128` data type.
- `toInt256(expr)`Results in the `Int256` data type.
- `toInt8(expr)`Converts to a value of data type `Int8`.
- `toInt16(expr)`Converts to a value of data type `Int16`.
- `toInt32(expr)`Converts to a value of data type `Int32`.
- `toInt64(expr)`Converts to a value of data type `Int64`.
- `toInt128(expr)`Converts to a value of data type `Int128`.
- `toInt256(expr)`Converts to a value of data type `Int256`.
**Arguments**
@ -53,7 +54,7 @@ Result:
## toInt(8\|16\|32\|64\|128\|256)OrZero
It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If failed, returns 0.
Takes an argument of type [String](/docs/en/sql-reference/data-types/string.md) and tries to parse it into an Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `0`.
**Example**
@ -73,7 +74,7 @@ Result:
## toInt(8\|16\|32\|64\|128\|256)OrNull
It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If failed, returns NULL.
It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns `NULL`.
**Example**
@ -93,7 +94,7 @@ Result:
## toInt(8\|16\|32\|64\|128\|256)OrDefault
It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If failed, returns the default type value.
It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If unsuccessful, returns the default type value.
**Example**
@ -116,11 +117,11 @@ Result:
Converts an input value to the [UInt](/docs/en/sql-reference/data-types/int-uint.md) data type. This function family includes:
- `toUInt8(expr)`Results in the `UInt8` data type.
- `toUInt16(expr)`Results in the `UInt16` data type.
- `toUInt32(expr)`Results in the `UInt32` data type.
- `toUInt64(expr)`Results in the `UInt64` data type.
- `toUInt256(expr)`Results in the `UInt256` data type.
- `toUInt8(expr)`Converts to a value of data type `UInt8`.
- `toUInt16(expr)`Converts to a value of data type `UInt16`.
- `toUInt32(expr)`Converts to a value of data type `UInt32`.
- `toUInt64(expr)`Converts to a value of data type `UInt64`.
- `toUInt256(expr)`Converts to a value of data type `UInt256`.
**Arguments**
@ -128,7 +129,7 @@ Converts an input value to the [UInt](/docs/en/sql-reference/data-types/int-uint
**Returned value**
Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type.
- Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type.
Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers.
@ -166,26 +167,30 @@ Result:
## toDate
Converts the argument to `Date` data type.
Converts the argument to [Date](/docs/en/sql-reference/data-types/date.md) data type.
If the argument is [DateTime](/docs/en/sql-reference/data-types/datetime.md) or [DateTime64](/docs/en/sql-reference/data-types/datetime64.md), it truncates it and leaves the date component of the DateTime:
If the argument is `DateTime` or `DateTime64`, it truncates it, leaving the date component of the DateTime:
```sql
SELECT
now() AS x,
toDate(x)
```
```response
┌───────────────────x─┬─toDate(now())─┐
│ 2022-12-30 13:44:17 │ 2022-12-30 │
└─────────────────────┴───────────────┘
```
If the argument is a string, it is parsed as Date or DateTime. If it was parsed as DateTime, the date component is being used:
If the argument is a [String](/docs/en/sql-reference/data-types/string.md), it is parsed as [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). If it was parsed as [DateTime](/docs/en/sql-reference/data-types/datetime.md), the date component is being used:
```sql
SELECT
toDate('2022-12-30') AS x,
toTypeName(x)
```
```response
┌──────────x─┬─toTypeName(toDate('2022-12-30'))─┐
│ 2022-12-30 │ Date │
@ -193,18 +198,20 @@ SELECT
1 row in set. Elapsed: 0.001 sec.
```
```sql
SELECT
toDate('2022-12-30 01:02:03') AS x,
toTypeName(x)
```
```response
┌──────────x─┬─toTypeName(toDate('2022-12-30 01:02:03'))─┐
│ 2022-12-30 │ Date │
└────────────┴───────────────────────────────────────────┘
```
If the argument is a number and it looks like a UNIX timestamp (is greater than 65535), it is interpreted as a DateTime, then truncated to Date in the current timezone. The timezone argument can be specified as a second argument of the function. The truncation to Date depends on the timezone:
If the argument is a number and looks like a UNIX timestamp (is greater than 65535), it is interpreted as a [DateTime](/docs/en/sql-reference/data-types/datetime.md), then truncated to [Date](/docs/en/sql-reference/data-types/date.md) in the current timezone. The timezone argument can be specified as a second argument of the function. The truncation to [Date](/docs/en/sql-reference/data-types/date.md) depends on the timezone:
```sql
SELECT
@ -217,6 +224,7 @@ SELECT
toDate(ts) AS date_Amsterdam_2,
toDate(ts, 'Pacific/Apia') AS date_Samoa_2
```
```response
Row 1:
──────
@ -232,7 +240,7 @@ date_Samoa_2: 2022-12-31
The example above demonstrates how the same UNIX timestamp can be interpreted as different dates in different time zones.
If the argument is a number and it is smaller than 65536, it is interpreted as the number of days since 1970-01-01 (a UNIX day) and converted to Date. It corresponds to the internal numeric representation of the `Date` data type. Example:
If the argument is a number and it is smaller than 65536, it is interpreted as the number of days since 1970-01-01 (the first UNIX day) and converted to [Date](/docs/en/sql-reference/data-types/date.md). It corresponds to the internal numeric representation of the `Date` data type. Example:
```sql
SELECT toDate(12345)
@ -270,8 +278,6 @@ SELECT
└─────────────────────┴───────────────┴─────────────┴─────────────────────┘
```
Have a nice day working with dates and times.
## toDateOrZero
## toDateOrNull
@ -288,7 +294,7 @@ Have a nice day working with dates and times.
## toDate32
Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32` returns the border values supported by `Date32`. If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, borders of `Date` are taken into account.
Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32` returns the border values supported by [Date32](/docs/en/sql-reference/data-types/date32.md). If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, it's borders are taken into account.
**Syntax**
@ -302,9 +308,7 @@ toDate32(expr)
**Returned value**
- A calendar date.
Type: [Date32](/docs/en/sql-reference/data-types/date32.md).
- A calendar date. Type [Date32](/docs/en/sql-reference/data-types/date32.md).
**Example**
@ -332,7 +336,7 @@ SELECT toDate32('1899-01-01') AS value, toTypeName(value);
└────────────┴────────────────────────────────────┘
```
3. With `Date`-type argument:
3. With [Date](/docs/en/sql-reference/data-types/date.md) argument:
``` sql
SELECT toDate32(toDate('1899-01-01')) AS value, toTypeName(value);
@ -386,7 +390,7 @@ Result:
## toDate32OrDefault
Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32OrDefault` returns the lower border value supported by `Date32`. If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if an invalid argument is received.
Converts the argument to the [Date32](/docs/en/sql-reference/data-types/date32.md) data type. If the value is outside the range, `toDate32OrDefault` returns the lower border value supported by [Date32](/docs/en/sql-reference/data-types/date32.md). If the argument has [Date](/docs/en/sql-reference/data-types/date.md) type, it's borders are taken into account. Returns default value if an invalid argument is received.
**Example**
@ -666,7 +670,7 @@ YYYY-MM-DD
YYYY-MM-DD hh:mm:ss
```
As an exception, if converting from UInt32, Int32, UInt64, or Int64 numeric types to Date, and if the number is greater than or equal to 65536, the number is interpreted as a Unix timestamp (and not as the number of days) and is rounded to the date. This allows support for the common occurrence of writing toDate(unix_timestamp), which otherwise would be an error and would require writing the more cumbersome toDate(toDateTime(unix_timestamp)).
As an exception, if converting from UInt32, Int32, UInt64, or Int64 numeric types to Date, and if the number is greater than or equal to 65536, the number is interpreted as a Unix timestamp (and not as the number of days) and is rounded to the date. This allows support for the common occurrence of writing `toDate(unix_timestamp)`, which otherwise would be an error and would require writing the more cumbersome `toDate(toDateTime(unix_timestamp))`.
Conversion between a date and a date with time is performed the natural way: by adding a null time or dropping the time.
@ -696,7 +700,7 @@ Also see the `toUnixTimestamp` function.
## toFixedString(s, N)
Converts a String type argument to a FixedString(N) type (a string with fixed length N). N must be a constant.
Converts a [String](/docs/en/sql-reference/data-types/string.md) type argument to a [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) type (a string of fixed length N).
If the string has fewer bytes than N, it is padded with null bytes to the right. If the string has more bytes than N, an exception is thrown.
## toStringCutToZero(s)
@ -914,7 +918,7 @@ Result:
└─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘
```
Conversion to FixedString(N) only works for arguments of type [String](/docs/en/sql-reference/data-types/string.md) or [FixedString](/docs/en/sql-reference/data-types/fixedstring.md).
Conversion to [FixedString (N)](/docs/en/sql-reference/data-types/fixedstring.md) only works for arguments of type [String](/docs/en/sql-reference/data-types/string.md) or [FixedString](/docs/en/sql-reference/data-types/fixedstring.md).
Type conversion to [Nullable](/docs/en/sql-reference/data-types/nullable.md) and back is supported.
@ -1174,7 +1178,7 @@ For all of the formats with separator the function parses months names expressed
**Returned value**
- `time_string` converted to the `DateTime` data type.
- `time_string` converted to the [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type.
**Examples**
@ -1254,10 +1258,10 @@ Result:
**See Also**
- [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/)
- [RFC 1123](https://tools.ietf.org/html/rfc1123)
- [toDate](#todate)
- [toDateTime](#todatetime)
- [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/)
## parseDateTimeBestEffortUS

View File

@ -19,8 +19,15 @@ CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_n
## Example
Create a user:
```sql
CREATE USER robin IDENTIFIED BY 'password';
```
Create the `max_memory_usage_profile` settings profile with value and constraints for the `max_memory_usage` setting and assign it to user `robin`:
``` sql
CREATE SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin
CREATE
SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000
TO robin
```

View File

@ -17,10 +17,11 @@ By default, tables are created only on the current server. Distributed DDL queri
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [compression_codec] [TTL expr1],
name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [compression_codec] [TTL expr2],
name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [compression_codec] [TTL expr1] [COMMENT 'comment for column'],
name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [compression_codec] [TTL expr2] [COMMENT 'comment for column'],
...
) ENGINE = engine
COMMENT 'comment for table'
```
Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine.
@ -32,6 +33,8 @@ Expressions can also be defined for default values (see below).
If necessary, primary key can be specified, with one or more key expressions.
Comments can be added for columns and for the table.
### With a Schema Similar to Other Table
``` sql
@ -127,6 +130,26 @@ Default expressions may be defined as an arbitrary expression from table constan
Normal default value. If the INSERT query does not specify the corresponding column, it will be filled in by computing the corresponding expression.
Example:
```sql
CREATE OR REPLACE TABLE test
(
id UInt64,
updated_at DateTime DEFAULT now(),
updated_at_date Date DEFAULT toDate(updated_at)
)
ENGINE = MergeTree
ORDER BY id;
INSERT INTO test (id) Values (1);
SELECT * FROM test;
┌─id─┬──────────updated_at─┬─updated_at_date─┐
│ 1 │ 2023-02-24 17:06:46 │ 2023-02-24 │
└────┴─────────────────────┴─────────────────┘
```
### MATERIALIZED
`MATERIALIZED expr`
@ -135,6 +158,36 @@ Materialized expression. Such a column cant be specified for INSERT, because
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.
Example:
```sql
CREATE OR REPLACE TABLE test
(
id UInt64,
updated_at DateTime MATERIALIZED now(),
updated_at_date Date MATERIALIZED toDate(updated_at)
)
ENGINE = MergeTree
ORDER BY id;
INSERT INTO test Values (1);
SELECT * FROM test;
┌─id─┐
│ 1 │
└────┘
SELECT id, updated_at, updated_at_date FROM test;
┌─id─┬──────────updated_at─┬─updated_at_date─┐
│ 1 │ 2023-02-24 17:08:08 │ 2023-02-24 │
└────┴─────────────────────┴─────────────────┘
SELECT * FROM test SETTINGS asterisk_include_materialized_columns=1;
┌─id─┬──────────updated_at─┬─updated_at_date─┐
│ 1 │ 2023-02-24 17:08:08 │ 2023-02-24 │
└────┴─────────────────────┴─────────────────┘
```
### EPHEMERAL
`EPHEMERAL [expr]`
@ -142,6 +195,34 @@ In addition, this column is not substituted when using an asterisk in a SELECT q
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.
Example:
```sql
CREATE OR REPLACE TABLE test
(
id UInt64,
unhexed String EPHEMERAL,
hexed FixedString(4) DEFAULT unhex(unhexed)
)
ENGINE = MergeTree
ORDER BY id
INSERT INTO test (id, unhexed) Values (1, '5a90b714');
SELECT
id,
hexed,
hex(hexed)
FROM test
FORMAT Vertical;
Row 1:
──────
id: 1
hexed: Z<><5A>
hex(hexed): 5A90B714
```
### ALIAS
`ALIAS expr`
@ -156,6 +237,29 @@ If you add a new column to a table but later change its default expression, the
It is not possible to set default values for elements in nested data structures.
```sql
CREATE OR REPLACE TABLE test
(
id UInt64,
size_bytes Int64,
size String Alias formatReadableSize(size_bytes)
)
ENGINE = MergeTree
ORDER BY id;
INSERT INTO test Values (1, 4678899);
SELECT id, size_bytes, size FROM test;
┌─id─┬─size_bytes─┬─size─────┐
│ 1 │ 4678899 │ 4.46 MiB │
└────┴────────────┴──────────┘
SELECT * FROM test SETTINGS asterisk_include_alias_columns=1;
┌─id─┬─size_bytes─┬─size─────┐
│ 1 │ 4678899 │ 4.46 MiB │
└────┴────────────┴──────────┘
```
## Primary Key
You can define a [primary key](../../../engines/table-engines/mergetree-family/mergetree.md#primary-keys-and-indexes-in-queries) when creating a table. Primary key can be specified in two ways:
@ -166,7 +270,7 @@ You can define a [primary key](../../../engines/table-engines/mergetree-family/m
CREATE TABLE db.table_name
(
name1 type1, name2 type2, ...,
PRIMARY KEY(expr1[, expr2,...])]
PRIMARY KEY(expr1[, expr2,...])
)
ENGINE = engine;
```

View File

@ -54,6 +54,10 @@ SELECT * FROM view(column1=value1, column2=value2 ...)
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
```
:::tip
Here is a step by step guide on using [Materialized views](docs/en/guides/developer/cascading-materialized-views.md).
:::
Materialized views store data transformed by the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query.
When creating a materialized view without `TO [db].[table]`, you must specify `ENGINE` the table engine for storing data.

View File

@ -95,7 +95,7 @@ You can insert data separately from the query by using the command-line client o
If table has [constraints](../../sql-reference/statements/create/table.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped.
## Inserting the Results of `SELECT`
## Inserting the Results of SELECT
**Syntax**

View File

@ -36,6 +36,8 @@ Queries that use `FINAL` are executed slightly slower than similar queries that
**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine havet happened yet and deal with it by applying aggregation (for example, to discard duplicates).
`FINAL` can be applied automatically using [FINAL](../../../operations/settings/settings.md#final) setting to all tables in a query using a session or a user profile.
## Implementation Details
If the `FROM` clause is omitted, data will be read from the `system.one` table.

View File

@ -83,9 +83,13 @@ Examples: `1`, `10_000_000`, `0xffff_ffff`, `18446744073709551615`, `0xDEADBEEF`
### String
Only string literals in single quotes are supported. The enclosed characters can be backslash-escaped. The following escape sequences have a corresponding special value: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. In all other cases, escape sequences in the format `\c`, where `c` is any character, are converted to `c`. It means that you can use the sequences `\'`and`\\`. The value will have the [String](../sql-reference/data-types/string.md) type.
String literals must be enclosed in single quotes, double quotes are not supported.
Escaping works either
In string literals, you need to escape at least `'` and `\`. Single quotes can be escaped with the single quote, literals `'It\'s'` and `'It''s'` are equal.
- using a preceding single quote where the single-quote character `'` (and only this character) can be escaped as `''`, or
- using a preceding backslash with the following supported escape sequences: `\\`, `\'`, `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. The backslash loses its special meaning, i.e. will be interpreted literally, if it precedes characters different than the listed ones.
In string literals, you need to escape at least `'` and `\` using escape codes `\'` (or: `''`) and `\\`.
### Compound

View File

@ -6,25 +6,28 @@ sidebar_label: format
# format
Extracts table structure from data and parses it according to specified input format.
Parses data from arguments according to specified input format. If structure argument is not specified, it's extracted from the data.
**Syntax**
``` sql
format(format_name, data)
format(format_name, [structure], data)
```
**Parameters**
- `format_name` — The [format](../../interfaces/formats.md#formats) of the data.
- `structure` - Structure of the table. Optional. Format 'column1_name column1_type, column2_name column2_type, ...'.
- `data` — String literal or constant expression that returns a string containing data in specified format
**Returned value**
A table with data parsed from `data` argument according specified format and extracted schema.
A table with data parsed from `data` argument according to specified format and specified or extracted structure.
**Examples**
Without `structure` argument:
**Query:**
``` sql
SELECT * FROM format(JSONEachRow,
@ -67,6 +70,29 @@ $$)
└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
With `structure` argument:
**Query:**
```sql
SELECT * FROM format(JSONEachRow, 'a String, b UInt32',
$$
{"a": "Hello", "b": 111}
{"a": "World", "b": 123}
{"a": "Hello", "b": 112}
{"a": "World", "b": 124}
$$)
```
**Result:**
```response
┌─a─────┬───b─┐
│ Hello │ 111 │
│ World │ 123 │
│ Hello │ 112 │
│ World │ 124 │
└───────┴─────┘
```
**See Also**
- [Formats](../../interfaces/formats.md)

Some files were not shown because too many files have changed in this diff Show More