mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 17:44:23 +00:00
56 KiB
56 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2022 |
2022 Changelog
ClickHouse release v22.12.1.1752-stable (688e488e93
) FIXME as compared to v22.11.1.1360-stable (0d211ed198
)
Backward Incompatible Change
- Fixed backward incompatibility in (de)serialization of states of
min
,max
,any*
,argMin
,argMax
aggregate functions withString
argument. The incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/41431 and affects 22.9, 22.10 and 22.11 branches (fixed since 22.9.6, 22.10.4 and 22.11.2 correspondingly). Some minor releases of 22.3, 22.7 and 22.8 branches are also affected: 22.3.13...22.3.14 (fixed since 22.3.15), 22.8.6...22.8.9 (fixed since 22.8.10), 22.7.6 and newer (will not be fixed in 22.7, we recommend to upgrade from 22.7.* to 22.8.10 or newer). This release note does not concern users that have never used affected versions. Incompatible versions append extra'\0'
to strings when reading states of the aggregate functions mentioned above. For example, if an older version saved state ofanyState('foobar')
tostate_column
then incompatible version will print'foobar\0'
onanyMerge(state_column)
. Also incompatible versions write states of the aggregate functions without trailing'\0'
. Newer versions (that have the fix) can correctly read data written by all versions including incompatible versions, except one corner case. If an incompatible version saved a state with a string that actually ends with null character, then newer version will trim trailing'\0'
when reading state of affected aggregate function. For example, if an incompatible version saved state ofanyState('abrac\0dabra\0')
tostate_column
then newer versions will print'abrac\0dabra'
onanyMerge(state_column)
. The issue also affects distributed queries when an incompatible version works in a cluster together with older or newer versions. #43038 (Raúl Marín).
New Feature
- Add "grace_hash" join_algorithm. #38191 (BigRedEye).
- Merging on initiator now uses the same memory bound approach as merging of local aggregation results if
enable_memory_bound_merging_of_aggregation_results
is set. #40879 (Nikita Taranov). - Add BSONEachRow input/output format. In this format, ClickHouse formats/parses each row as a separated BSON Document and each column is formatted/parsed as a single BSON field with column name as a key. #42033 (mark-polokhov).
- close: #37631. #42265 (刘陶峰).
- Added
multiplyDecimal
anddivideDecimal
functions for decimal operations with fixed precision. #42438 (Andrey Zvonov). - Added
system.moves
table with list of currently moving parts. #42660 (Sergei Trifonov). - Keeper feature: add support for embedded Prometheus endpoint. #43087 (Antonio Andelic).
- Added age function to calculate difference between two dates or dates with time values expressed as number of full units. Close #41115. #43123 (Roman Vasin).
- Add settings
max_streams_for_merge_tree_reading
andallow_asynchronous_read_from_io_pool_for_merge_tree
. Settingmax_streams_for_merge_tree_reading
limits the number of reading streams for MergeTree tables. Settingallow_asynchronous_read_from_io_pool_for_merge_tree
enables background I/O pool to read fromMergeTree
tables. This may increase performance for I/O bound queries if used together withmax_streams_to_max_threads_ratio
ormax_streams_for_merge_tree_reading
. #43260 (Nikolai Kochetov). - Add the expression of the index on
data_skipping_indices
system table. #43308 (Guillaume Tassery). - New hash function xxh3 added. Also performance of
xxHash32
andxxHash64
improved on arm thanks to library update. #43411 (Nikita Taranov). -
- Temporary data (for external sorting, aggregation, and JOINs) can share storage with the filesystem cache for remote disks and evict it, close #42158. #43457 (Vladimir C).
- Add column
engine_full
to system tabledatabases
so that users can access whole engine definition of database via system tables. #43468 (凌涛). - Add password complexity rules and checks for creating a new user. #43719 (Nikolay Degterinsky).
- Add function concatWithSeparator , like concat_ws in spark. #43749 (李扬).
- Added constraints for merge tree settings. #43903 (Sergei Trifonov).
- Support numeric literals with _ as separator. #43925 (jh0x).
- Add a new setting
input_format_json_read_objects_as_strings
that allows to parse nested JSON objects into Strings in all JSON input formats. This setting is disable by default. #44052 (Kruglov Pavel).
Performance Improvement
- Optimisation is getting skipped now if
max_size_to_preallocate_for_aggregation
has too small value. Default value of this setting increased to10^8
. #43945 (Nikita Taranov).
Improvement
- Support numeric literals with underscores. closes #28967. #39129 (unbyte).
- Add
FROM table SELECT column
syntax. #41095 (Nikolay Degterinsky). - This PR changes how followed queries delete parts: truncate table, alter table drop part, alter table drop partition. Now these queries make empty parts which cover old parts. This makes truncate query works without exclusive lock which means concurrent reads aren't locked. Also achieved durability in all those queries. If request is succeeded then no resurrected pars appear later. Note that atomicity is achieved only with transaction scope. #41145 (Sema Checherinda).
SET param_x
query no longer requires manual string serialization for the value of the parameter. For example, querySET param_a = '[\'a\', \'b\']'
can now be written likeSET param_a = ['a', 'b']
. #41874 (Nikolay Degterinsky).filesystemAvailable
and related functions support one optional argument with disk name, and changefilesystemFree
tofilesystemUnreserved
. Closes #35076. #42064 (flynn).- Increased the default value of search_limit to 256, and added LDAP server config option to change that to an arbitrary value. Closes: #42276. #42461 (Vasily Nemkov).
- Add cosine distance for annoy. #42778 (Filatenkov Artur).
- Allow to remove sensitive information from the exception messages also. Resolves #41418. #42940 (filimonov).
- Keeper improvement: Add 4lw command
rqld
which can manually assign a node as leader. #43026 (JackyWoo). - Apply connection timeouts settings for Distributed async INSERT from the query. #43156 (Azat Khuzhin).
- unhex function support FixedString arguments. issue42369. #43207 (DR).
- Priority is given to deleting completely expired Parts,related #42869. #43222 (zhongyuankai).
- Follow-up to https://github.com/ClickHouse/ClickHouse/pull/42484. Mask sensitive information in logs better; mask secret parts in the output of queries
SHOW CREATE TABLE
andSELECT FROM system.tables
. Also resolves #41418. #43227 (Vitaly Baranov). - Enable compress marks and primary key. #43288 (SmitaRKulkarni).
- resolve issue #38075 . Right now async insert doesn't support deduplication, because multiple small inserts will coexist in one part, which corespond multiple
block id
s. This solution is straitfoward: The change involves: 1. mark offsets for every inserts in every chunk 2. calculate multipleblock_id
s when sinker receive a chunk 3. get block number lock by theseblock_id
s 3.1. if fails, remove the dup insert(s) and dupblock_id
(s) from block and recalculateoffsets
agian. 3.2. if succeeds, commitblock_id
's and other items into keeper a. if fails, do 3.1 b. if succeeds, everything succeeds. #43304 (Han Fei). - More precise and reactive CPU load indication on client. #43307 (Sergei Trifonov).
- Restrict default access to named collections for user defined in config. It must have explicit
show_named_collections=1
to be able to see them. #43325 (Kseniia Sumarokova). - Support reading of subcolumns of nested types from storage
S3
and table functions3
with formatsParquet
,Arrow
andORC
. #43329 (chen). -
- Systemd integration now correctly notifies systemd that service is really started and is ready to server requests. #43400 (Коренберг Марк).
- Add table_uuid to system.parts. #43404 (Azat Khuzhin).
- Added client option to display the number of locally processed rows in non-interactive mode (--print-num-processed-rows). #43407 (jh0x).
- Show read rows while reading from stdin from client. Closes #43423. #43442 (Kseniia Sumarokova).
- Keeper improvement: try syncing logs to disk in parallel with replication. #43450 (Antonio Andelic).
- Show progress bar while reading from s3 table function / engine. #43454 (Kseniia Sumarokova).
- Progress bar will show both read and written rows. #43496 (Ilya Yatsishin).
- Implement
aggregation-in-order
optimization on top of query plan. It is enabled by default (but works only together withoptimize_aggregation_in_order
, which is disabled by default). Setquery_plan_aggregation_in_order = 0
to use previous AST-based version. #43592 (Nikolai Kochetov). - Allow to send profile events with
trace_type = 'ProfileEvent'
tosystem.trace_log
on each increment with current stack, profile event name and value of increment. It can be enabled by settingtrace_profile_events
and used to debug performance of queries. #43639 (Anton Popov). - Keeper improvement: requests are batched more often. The batching can be controlled with the new setting
max_requests_quick_batch_size
. #43686 (Antonio Andelic). - Added possibility to use array as a second parameter for cutURLParameter function. Close #6827. #43788 (Roman Vasin).
- Implement referential dependencies and use them to create tables in the correct order while restoring from a backup. #43834 (Vitaly Baranov).
- Add a new setting
input_format_max_binary_string_size
to limit string size in RowBinary format. #43842 (Kruglov Pavel). -
- Fix some incorrect logic in ast level optimization related. #43873 (Duc Canh Le).
- Support query like
SHOW FULL TABLES ...
. #43910 (Filatenkov Artur). - When ClickHouse requests a remote HTTP server, and it returns an error, the numeric HTTP code was not displayed correctly in the exception message. Closes #43919. #43920 (Alexey Milovidov).
- Settings
merge_tree_min_rows_for_concurrent_read_for_remote_filesystem/merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem
did not respect adaptive granularity. Fat rows did not decrease the number of read rows (as it is was done formerge_tree_min_rows_for_concurrent_read/merge_tree_min_bytes_for_concurrent_read
, which could lead to high memory usage. #43965 (Nikolai Kochetov). - Support
optimize_if_transform_strings_to_enum
in new analyzer. #43999 (Antonio Andelic). - This is to upgrade the new "DeflateQpl" compression codec which has been implemented on previous PR (details: https://github.com/ClickHouse/ClickHouse/pull/39494). This patch improves codec on below aspects: 1. QPL v0.2.0 to QPL v0.3.0 Intel® Query Processing Library (QPL) 2. Improve CMake file for fixing QPL build issues for QPL v0.3.0。 3. Link the QPL library with libaccel-config at build time instead of runtime loading on QPL v0.2.0 (dlopen) 4. Fixed log print issue in CompressionCodecDeflateQpl.cpp. #44024 (jasperzhu).
- Follow-up to https://github.com/ClickHouse/ClickHouse/pull/43834 Fix review issues; dependencies from
Distributed
table engine and fromcluster()
function are also considered now; as well as dependencies of a dictionary defined without host & port specified. #44158 (Vitaly Baranov).
Bug Fix
- Fix mutations not making progress when checksums do not match between replicas (e.g. caused by a change in data format on an upgrade). #36877 (nvartolomei).
- fix skip_unavailable_shards does not work using hdfsCluster table function. #43236 (chen).
- fix s3 support question mark wildcard. Closes #42731. #43253 (chen).
-
- Fix functions arrayFirstOrNull and arrayLastOrNull or null when array is Nullable. #43274 (Duc Canh Le).
-
- we create a new zk path called "async_blocks" for replicated tables in #43304 . However, for tables created in older versions, this path does not exist and will cause error when doing partition operations. This PR will create this node when initializing replicated tree. - This PR created a flag
async_insert_deduplicate
withfalse
default value to control whether to use this function. As mentioned in #38075 , this function is not yet fully finished. I would turn off it by default. #44223 (Han Fei).
- we create a new zk path called "async_blocks" for replicated tables in #43304 . However, for tables created in older versions, this path does not exist and will cause error when doing partition operations. This PR will create this node when initializing replicated tree. - This PR created a flag
Build/Testing/Packaging Improvement
- Add support for FreeBSD/powerpc64le. #40422 (pkubaj).
- Bump Testcontainers for Go to v0.15.0. #43278 (Manuel de la Peña).
- ... Enable base64 on s390x > Information about CI checks: https://clickhouse.com/docs/en/development/continuous-integration/. #43352 (Suzy Wang).
- Shutdown will be much faster if do not call clearOldPartsFromFilesystem. Especially this is right for tests with zero-copy due to single thread deletion parts. clearOldPartsFromFilesystem is unnecessary after https://github.com/ClickHouse/ClickHouse/pull/41145. #43760 (Sema Checherinda).
- Integrate skim into the client/local. #43922 (Azat Khuzhin).
- Allow clickhouse to use openssl as a dynamic library and in-tree for development purposes. #43991 (Boris Kuschel).
- Closes #43912. #43992 (Nikolay Degterinsky).
- Bring sha512 sums back to the building step. #44017 (Mikhail f. Shiryaev).
- Kill stress tests after 2.5h in case of hanging process. #44214 (Mikhail f. Shiryaev).
Bug Fix (user-visible misbehavior in official stable or prestable release)
- Fixed unable to log in (because of failure to create session_log entry) in rare case of messed up setting profiles. ... #42641 (Vasily Nemkov).
- Fix incorrect UserTimeMicroseconds/SystemTimeMicroseconds accounting. #42791 (Azat Khuzhin).
- Do not suppress exceptions in web disk. Fix retries for web disk. #42800 (Azat Khuzhin).
- Fixed race condition between inserts and dropping MVs. #43161 (AlfVII).
- Fixed bug which could lead to deadlock while using asynchronous inserts. #43233 (Anton Popov).
- Additional check on zero uncompressed size is added to
CompressionCodecDelta
. #43255 (Nikita Taranov). - An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. #43297 (Arthur Passos).
- Fix bad cast from LowCardinality column when using short circuit function execution. Proper fix of https://github.com/ClickHouse/ClickHouse/pull/42937. #43311 (Kruglov Pavel).
- Fixed queries with
SAMPLE BY
with prewhere optimization on tables usingMerge
engine. #43315 (Antonio Andelic). - Fix
DESCRIBE
fordeltaLake
andhudi
table functions. #43323 (Antonio Andelic). - Check and compare the content of
format_version
file inMergeTreeData
so tables can be loaded even if the storage policy was changed. #43328 (Antonio Andelic). - Fix possible (very unlikely) "No column to rollback" logical error during INSERT into Buffer. #43336 (Azat Khuzhin).
- Fix a bug that allowed FucntionParser to parse an unlimited amount of round brackets into one function if
allow_function_parameters
is set. #43350 (Nikolay Degterinsky). - MaterializeMySQL support ddl: drop table t1,t2 and Compatible with most of MySQL drop ddl. #43366 (zzsmdfj).
- Fix possible
Cannot create non-empty column with type Nothing
in functions if/multiIf. Closes #43356. #43368 (Kruglov Pavel). - Fix a bug when row level filter uses default value of column. #43387 (Alexander Gololobov).
- Query with DISTINCT + LIMIT BY + LIMIT can return fewer rows than expected. Fixes #43377. #43410 (Igor Nikonov).
- Fix sumMap() for Nullable(Decimal()). #43414 (Azat Khuzhin).
- Fix date_diff() for hour/minute on macOS. Close #42742. #43466 (zzsmdfj).
- Fix incorrect memory accounting because of merges/mutations. #43516 (Azat Khuzhin).
- Substitute UDFs in
CREATE
query to avoid failures during loading at the startup. Additionally, UDFs can now be used asDEFAULT
expressions for columns. #43539 (Antonio Andelic). - Correctly report errors in queries even when multiple JOINs optimization is taking place. #43583 (Salvatore).
- Fixed primary key analysis with conditions involving
toString(enum)
. #43596 (Nikita Taranov). - During recovering of the lost replica there could a situation where we need to atomically swap two table names (use EXCHANGE), but instead previously we tried to use two RENAME queries. Which was obviously failed and moreover failed the whole recovery process of the database replica. #43628 (Nikita Mikhaylov).
- fix s3Cluster function returns NOT_FOUND_COLUMN_IN_BLOCK error. Closes #43534. #43629 (chen).
- Optimized number of List requests to ZooKeeper when selecting a part to merge. Previously it could produce thousands of requests in some cases. Fixes #43647. #43675 (Alexander Tokmakov).
- Fix posssible logical error 'Array sizes mismatched' while parsing JSON object with arrays with same key names but with different nesting level. Closes #43569. #43693 (Kruglov Pavel).
- Fixed possible exception in case of distributed group by with an alias column among aggregation keys. #43709 (Nikita Taranov).
- Fix bug which can lead to broken projections if zero-copy replication is enabled and used. #43764 (alesapin).
- Fixed
ALTER ... RESET SETTING
withON CLUSTER
. It could be applied to one replica only. Fixes #43843. #43848 (Elena Torró). -
- Fix logical error in right storage join with using. #43963 (Vladimir C).
- Keeper fix: throw if interserver port for Raft is already in use. Fix segfault in Prometheus when Raft server failed to initialize. #43984 (Antonio Andelic).
- Fix order by positional arg in case unneeded columns pruning. Closes #43964. #43987 (Kseniia Sumarokova).
-
- Fix bug with wrong order of keys in Storage Join. #44012 (Vladimir C).
- Fixed exception when subquery contains having but doesn't contain actual aggregation. #44051 (Nikita Taranov).
- Fix race in s3 multipart upload. This race could cause the error
Part number must be an integer between 1 and 10000, inclusive. (S3_ERROR)
while restoring from a backup. #44065 (Vitaly Baranov). - Fix undefined behavior in the
quantiles
function, which might lead to uninitialized memory. Found by fuzzer. This closes #44066. #44067 (Alexey Milovidov). - Prevent dropping nested column if it creates empty part. #44159 (Antonio Andelic).
- Fix
LOGICAL_ERROR
in case when fetch of part was stopped while fetching projection to the disk with enabled zero-copy replication. #44173 (Anton Popov). - Fix possible Bad cast from type DB::IAST const* to DB::ASTLiteral const*. Closes #44191. #44192 (Kruglov Pavel).
- Prevent
ReadonlyReplica
metric from having negative values. #44220 (Antonio Andelic).
Build Improvement
- Fixed Endian issues in hex string conversion on s390x (which is not supported by ClickHouse). #41245 (Harry Lee).
- ... toDateTime64 conversion generates wrong time on z build, add bit_cast swap fix to support toDateTime64 on s390x platform. #42847 (Suzy Wang).
- ... s390x support for ip coding functions. #43078 (Suzy Wang).
- Fix byte order issue of wide integers for s390x. #43228 (Harry Lee).
- Fixed endian issue in bloom filter serialization for s390x. #43642 (Harry Lee).
- Fixed setting TCP_KEEPIDLE of client connection for s390x. #43850 (Harry Lee).
- Fix endian issue in StringHashTable for s390x. #44049 (Harry Lee).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Revert "S3 request per second rate throttling""'. #43335 (Sergei Trifonov).
- NO CL ENTRY: 'Update version after release'. #43348 (Mikhail f. Shiryaev).
- NO CL ENTRY: 'Revert "Add table_uuid to system.parts"'. #43571 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "Fix endian issue in integer hex string conversion"'. #43613 (Vladimir C).
- NO CL ENTRY: 'Update replication.md'. #43643 (Peignon Melvyn).
- NO CL ENTRY: 'Revert "Temporary files evict fs cache"'. #43883 (Vladimir C).
- NO CL ENTRY: 'Update html interface doc'. #44064 (San).
- NO CL ENTRY: 'Revert "Add function 'age'"'. #44203 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Builtin skim"'. #44227 (Azat Khuzhin).
- NO CL ENTRY: 'Revert "Add information about written rows in progress indicator"'. #44255 (Alexey Milovidov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Build libcxx and libcxxabi from llvm-project #42730 (Robert Schulze).
- Allow release only from ready commits #43019 (Mikhail f. Shiryaev).
- Add global flags to base/ libraries #43082 (Raúl Marín).
- Enable strict typing check in tests/ci #43132 (Mikhail f. Shiryaev).
- Add server UUID for disks access checks (read/read-by-offset/write/delete) to avoid possible races #43143 (Azat Khuzhin).
- Do not include libcxx library for C #43166 (Azat Khuzhin).
- Followup fixes for FuseFunctionsPass #43217 (Vladimir C).
- Fix bug in replication queue which can lead to premature mutation finish #43231 (alesapin).
- Support
CREATE / ALTER / DROP NAMED COLLECTION
queries under according access types #43252 (Kseniia Sumarokova). - Fix race in
IColumn::dumpStructure
#43269 (Anton Popov). - Sanitize thirdparty libraries for public flags #43275 (Azat Khuzhin).
- stress: increase timeout for server waiting after TERM #43277 (Azat Khuzhin).
- Fix cloning of ASTIdentifier #43282 (Nikolay Degterinsky).
- Fix race on write in
ReplicatedMergeTree
#43289 (Antonio Andelic). - Cancel lambda api url #43295 (Mikhail f. Shiryaev).
- Fixed: Typo #43312 (Raevsky Rudolf).
- Analyzer small fixes #43321 (Maksim Kita).
- Fix: make test_read_only_table more stable #43326 (Igor Nikonov).
- Make insertRangeFrom() more exception safe #43338 (Azat Khuzhin).
- Analyzer added indexes support #43341 (Maksim Kita).
- Allow to "drop tables" from s3_plain disk (so as from web disk) #43343 (Azat Khuzhin).
- Add --max-consecutive-errors for clickhouse-benchmark #43344 (Azat Khuzhin).
- Add #43072 #43345 (Nikita Taranov).
- Suggest users installation troubleshooting #43346 (Mikhail f. Shiryaev).
- Update version_date.tsv and changelogs after v22.11.1.1360-stable #43349 (robot-clickhouse).
- Provide full stacktrace in case of uncaught exception during server startup #43364 (Azat Khuzhin).
- Update SECURITY.md on new stable tags #43365 (Mikhail f. Shiryaev).
- Splitting checks in CI more #43373 (alesapin).
- Update version_date.tsv and changelogs after v22.8.9.24-lts #43393 (robot-clickhouse).
- Fix mess with signed sizes in SingleValueDataString #43401 (Alexander Tokmakov).
- Add a comment #43403 (Alexey Milovidov).
- Avoid race condition for updating system.distribution_queue values #43406 (Azat Khuzhin).
- Fix flaky 01926_order_by_desc_limit #43408 (Azat Khuzhin).
- Fix possible heap-use-after-free in local if history file cannot be created #43409 (Azat Khuzhin).
- Fix flaky test #43435 (Alexey Milovidov).
- Fix backward compatibility check #43436 (Alexey Milovidov).
- Fix typo #43446 (Alexey Milovidov).
- Remove noise from logs about NetLink in Docker #43447 (Alexey Milovidov).
- Modify test slightly #43448 (Alexey Milovidov).
- Set run_passes to 1 by default #43451 (Dmitry Novik).
- Do not reuse jemalloc memory in test_global_overcommit #43453 (Dmitry Novik).
- Fix createTableSharedID again #43458 (Alexander Tokmakov).
- Use smaller buffer for small files #43460 (Alexander Gololobov).
- Merging #42064 #43461 (Anton Popov).
- Use all parameters with prefixes from ssm #43467 (Mikhail f. Shiryaev).
- Avoid possible DROP hung due to attached web disk #43489 (Azat Khuzhin).
- Improve fuzzy search in clickhouse-client/clickhouse-local #43498 (Azat Khuzhin).
- check ast limits for create_parser_fuzzer #43504 (Sema Checherinda).
- Add another test for SingleDataValueString #43514 (Alexander Tokmakov).
- Move password reset message from client to server #43517 (Alexey Milovidov).
- Sync everything to persistent storage to avoid writeback affects perf tests #43530 (Azat Khuzhin).
- bump lib for diag #43538 (Dale McDiarmid).
- Temporarily disable
test_hive_query
#43542 (Alexander Tokmakov). - Analyzer SumIfToCountIfPass fix #43543 (Maksim Kita).
- Analyzer UniqInjectiveFunctionsEliminationPass #43547 (Maksim Kita).
- Disable broken 00176_bson_parallel_parsing #43550 (Alexander Tokmakov).
- Add benchmark for query interpretation with JOINs #43556 (Raúl Marín).
- Analyzer table functions untuple fix #43572 (Maksim Kita).
- Prepare CI for universal runners preallocated pool #43579 (Mikhail f. Shiryaev).
- Iterate list without index-based access #43584 (Alexander Gololobov).
- Remove code that I do not understand #43593 (Alexander Tokmakov).
- Add table_uuid to system.parts (resubmit) #43595 (Azat Khuzhin).
- Move perf tests for Aarch64 from PRs to master #43623 (Alexander Tokmakov).
- Fix flaky 01175_distributed_ddl_output_mode_long #43626 (Alexander Tokmakov).
- Speedup backup config loading #43627 (Alexander Gololobov).
- Fix #43478 #43636 (Nikolai Kochetov).
- Do not checkout submodules recursively #43637 (Mikhail f. Shiryaev).
- Optimize binary-builder size #43654 (Mikhail f. Shiryaev).
- Fix flaky
KeeperMap
integration tests #43658 (Antonio Andelic). - Fix data race in
Keeper
snapshot #43663 (Antonio Andelic). - Use docker images cache from merged PRs in master and release branches #43664 (Mikhail f. Shiryaev).
- Update AsynchronousReadIndirectBufferFromRemoteFS.cpp #43667 (Kseniia Sumarokova).
- Fix pagination issue in GITHUB_JOB_ID() #43681 (Mikhail f. Shiryaev).
- Try fix flaky test 00176_bson_parallel_parsing #43696 (Kruglov Pavel).
- Fix log messages in clickhouse-copier #43707 (Alexey Milovidov).
- try to remove clickhouse if already exists #43728 (Yakov Olkhovskiy).
- Fix 43622 #43731 (Amos Bird).
- Fix example of colored prompt in client #43738 (Azat Khuzhin).
- Minor fixes in annoy index documentation #43743 (Robert Schulze).
- Terminate lost runners #43756 (Mikhail f. Shiryaev).
- Update README.md #43759 (Tyler Hannan).
- Fix included_elements calculation in AggregateFunctionNullVariadic #43763 (Dmitry Novik).
- Migrate runner_token_rotation_lambda to zip-package deployment #43766 (Mikhail f. Shiryaev).
- Analyzer compound expression crash fix #43768 (Maksim Kita).
- Migrate termination lambda to zip-package #43769 (Mikhail f. Shiryaev).
- Fix flaky
test_store_cleanup
#43770 (Alexander Tokmakov). - Attempt to fix StyleCheck condition #43773 (Mikhail f. Shiryaev).
- Rerun PullRequestCI on changed description body #43777 (Mikhail f. Shiryaev).
- Yet another fix for AggregateFunctionMinMaxAny #43778 (Alexander Tokmakov).
- Add google benchmark to contrib #43779 (Nikita Taranov).
- Fix EN doc as in #43765 #43780 (Alexander Gololobov).
- Detach threads from thread group #43781 (Alexander Tokmakov).
- Try making
test_keeper_zookeeper_converter
less flaky #43789 (Antonio Andelic). - Polish UDF substitution visitor #43790 (Antonio Andelic).
- Analyzer ConstantNode refactoring #43793 (Maksim Kita).
- Update Poco #43802 (Alexander Gololobov).
- Add another BC check suppression #43810 (Alexander Tokmakov).
- tests: fix 01676_long_clickhouse_client_autocomplete flakiness #43819 (Azat Khuzhin).
- Use disk operation to serialize and deserialize meta files of StorageFilelog #43826 (flynn).
- Add constexpr #43827 (zhanglistar).
- Do not postpone removal of in-memory tables #43833 (Alexander Tokmakov).
- Increase some logging level for keeper client. #43835 (Nikolai Kochetov).
- FuseFunctionsPass small fix #43837 (Maksim Kita).
- Followup fixes for XML helpers #43845 (Alexander Gololobov).
- Hold ProcessListEntry a bit longer in case of exception from Interpreter #43847 (Alexander Tokmakov).
- A little improve performance of PODArray #43860 (zhanglistar).
- Change email for robot-clickhouse to immutable one #43861 (Mikhail f. Shiryaev).
- Rerun DocsCheck on edited PR description #43862 (Mikhail f. Shiryaev).
- Temporarily disable misc-* slow clang-tidy checks #43863 (Robert Schulze).
- do not leave tmp part on disk, do not go to the keeper for remove it #43866 (Sema Checherinda).
- do not read part status just for logging #43868 (Sema Checherinda).
- Analyzer Context refactoring #43884 (Maksim Kita).
- Analyzer CTE resolution fix #43893 (Maksim Kita).
- Improve release script #43894 (Mikhail f. Shiryaev).
- Use only PRs to our repository in pr_info on push #43895 (Mikhail f. Shiryaev).
- Join engine works with analyzer #43897 (Vladimir C).
- Fix reports #43904 (Mikhail f. Shiryaev).
- Fix vim settings (and make it compatible with neovim) #43909 (Azat Khuzhin).
- Fix clang tidy errors introduced in #43834 #43911 (Nikita Taranov).
- Fix BACKUP TO S3 for Google Cloud Storage #43940 (Azat Khuzhin).
- Fix tags workflow #43942 (Mikhail f. Shiryaev).
- Generate missed changelogs for latest releases #43944 (Mikhail f. Shiryaev).
- Fix typo in tests/ci/bugfix_validate_check.py #43973 (Vladimir C).
- Remove test logging of signal "EINTR" #44001 (Kruglov Pavel).
- Some cleanup of isDeterministic(InScopeOfQuery)() #44011 (Robert Schulze).
- Try to keep runners alive for longer #44015 (Mikhail f. Shiryaev).
- Fix relaxed "too many parts" threshold #44021 (Sergei Trifonov).
- Correct CompressionCodecGorilla exception message #44023 (Duc Canh Le).
- Fix exception message #44034 (Nikolay Degterinsky).
- Update version_date.tsv and changelogs after v22.8.11.15-lts #44035 (robot-clickhouse).
- do not hardlink serialization.json in new part #44036 (Sema Checherinda).
- Fix tracing of profile events #44045 (Anton Popov).
- Slightly better clickhouse disks and remove DiskMemory #44050 (alesapin).
- Assign release PRs #44055 (Mikhail f. Shiryaev).
- Merging #36877 #44059 (Alexey Milovidov).
- add changelogs #44061 (Dan Roscigno).
- Fix the CACHE_PATH creation for default value #44079 (Mikhail f. Shiryaev).
- Fix aspell #44090 (Mikhail f. Shiryaev).
- Fix another bug in AggregateFunctionMinMaxAny #44091 (Alexander Tokmakov).
- Analyzer aggregate function lambda crash fix #44098 (Maksim Kita).
- Fix -Wshorten-64-to-32 on FreeBSD and enable -Werror #44121 (Azat Khuzhin).
- Fix flaky test
02497_trace_events_stress_long
#44124 (Anton Popov). - Minor file renaming #44125 (Robert Schulze).
- Fix typo #44127 (Alexey Milovidov).
- Better descriptions of signals #44129 (Alexey Milovidov).
- make calls to be sure that parts are deleted #44156 (Sema Checherinda).
- Ignore "session expired" errors after BC check #44157 (Alexander Tokmakov).
- Fix incorrect assertion #44160 (Alexander Tokmakov).
- Close GRPC channels in tests #44184 (Antonio Andelic).
- Remove misleading message from logs #44190 (Alexander Tokmakov).
- Minor clang-tidy fixes in fromUnixTimestamp64() #44194 (Igor Nikonov).
- Hotfix for "check_status.tsv doesn't exists" in stress tests #44197 (Alexander Tokmakov).
- Fix documentation after #42438 #44200 (Alexey Milovidov).
- Fix an assertion in transactions #44202 (Alexander Tokmakov).
- Add log message #44237 (Alexander Tokmakov).