mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
63 KiB
63 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2022 |
2022 Changelog
ClickHouse release v21.6.1.6891-prestable FIXME as compared to v21.5.1.6601-prestable
New Feature
- Add projection support for MergeTree* tables. #20202 (Amos Bird).
- Add back
indexHint
function. This is for #21238 . This reverts https://github.com/ClickHouse/ClickHouse/pull/9542 . This fixes #9540 . #21304 (Amos Bird). -
- Add aggregate function sumCount. This function returns a tuple of two fields: sum and count. #21337 (hexiaoting).
- Added less secure IMDS credentials provider for S3 which works under docker correctly. #21852 (Vladimir Chebotarev).
-
- New aggregate function
deltaSumTimestamp
for summing the difference between consecutive rows while maintaining ordering during merge by storing timestamps. #21888 (Russ Frank).
- New aggregate function
-
- LDAP: implemented user DN detection functionality to use when mapping Active Directory groups to ClickHouse roles. #22228 (Denis Glazachev).
- Introduce a new function: arrayProduct which accept an array as the parameter, and return the product of all the elements in array. Close issue: #21613. #22242 (hexiaoting).
- Add setting
indexes
(boolean, disabled by default) toEXPLAIN PIPELINE
query. When enabled, shows used indexes, number of filtered parts and granules for every index applied. Supported forMergeTree*
tables. #22352 (Nikolai Kochetov). - Add setting
json
(boolean, 0 by default) forEXPLAIN PLAN
query. When enabled, query output will be a singleJSON
row. It is recommended to useTSVRaw
format to avoid unnecessary escaping. #23082 (Nikolai Kochetov). - Added
SYSTEM QUERY RELOAD MODEL
,SYSTEM QUERY RELOAD MODELS
. Closes #18722. #23182 (Maksim Kita). - Made progress bar for LocalServer and united it for Client and Local. #23196 (Egor Savin).
- Support DDL dictionaries for DatabaseMemory. Closes #22354. Added support for
DETACH DICTIONARY PERMANENTLY
. Added support forEXCHANGE DICTIONARIES
for Atomic database engine. Added support for moving dictionaries between databases usingRENAME DICTIONARY
. #23436 (Maksim Kita). - Allow globs
{...}
, which act like shards, and failover options with separator|
for URL table function. Closes #17181. #23446 (Kseniia Sumarokova). - If
insert_null_as_default
= 1, insert default values instead of NULL inINSERT ... SELECT
andINSERT ... SELECT ... UNION ALL ...
queries. Closes #22832. #23524 (Kseniia Sumarokova). - Implement table comments. closes #23225. #23548 (flynn).
- Introduce a new function: arrayProduct which accept an array as the parameter, and return the product of all the elements in array. Closes #21613. #23782 (Maksim Kita).
- Add postgres-like cast operator (
::
). E.g.:[1, 2]::Array(UInt8)
,0.1::Decimal(4, 4)
,number::UInt16
. #23871 (Anton Popov). - ... #23910 (Xiang Zhou).
- Add function splitByRegexp. #24077 (abel-cheng).
- Add
thread_name
column insystem.stack_trace
. This closes #23256. #24124 (abel-cheng).
Performance Improvement
- Enable
compile_expressions
setting by default. When this setting enabled, compositions of simple functions and operators will be compiled to native code with LLVM at runtime. #8482 (Alexey Milovidov). - ORC input format reading by stripe instead of reading entire table into memory by once which is cost memory when file size is huge. #23102 (Chao Ma).
- Update
re2
library. Performance of regular expressions matching is improved. Also this PR adds compatibility with gcc-11. #24196 (Raúl Marín).
Improvement
- Support Array data type for inserting and selecting data in Arrow, Parquet and ORC formats. #21770 (taylor12805).
- Add settings
external_storage_max_read_rows
andexternal_storage_max_read_rows
for MySQL table engine, dictionary source and MaterializeMySQL minor data fetches. #22697 (TCeason). - Retries on HTTP connection drops in S3. #22988 (Vladimir Chebotarev).
- Fix the case when a progress bar in interactive mode in clickhouse-client that appear in the middle of the data may rewrite some parts of visible data in terminal. This closes #19283. #23050 (Alexey Milovidov).
- Added possibility to restore MergeTree parts to 'detached' directory for DiskS3. #23112 (Pavel Kovalenko).
- Skip unavaiable replicas when writing to distributed tables. #23152 (Amos Bird).
- Support LowCardinality nullability with
join_use_nulls
, close #15101. #23237 (Vladimir C). - Disable settings
use_hedged_requests
andasync_socket_for_remote
because there is an evidence that it may cause issues. #23261 (Alexey Milovidov). - Fixed
quantile(s)TDigest
. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. #23314 (Vladimir Chebotarev). - Allow user to specify empty string instead of database name for
MySQL
storage. Default database will be used for queries. In previous versions it was working for SELECT queries and not support for INSERT was also added. This closes #19281. This can be useful working withSphinx
or other MySQL-compatible foreign databases. #23319 (Alexey Milovidov). - Disable min_bytes_to_use_mmap_io by default. #23322 (Azat Khuzhin).
- If user applied a misconfiguration by mistakenly setting
max_distributed_connections
to value zero, every query to aDistributed
table will throw exception with a message containing "logical error". But it's really an expected behaviour, not a logical error, so the exception message was slightly incorrect. It also triggered checks in our CI enviroment that ensures that no logical errors ever happen. Instead we will treatmax_distributed_connections
misconfigured to zero as the minimum possible value (one). #23348 (Azat Khuzhin). - Keep default timezone on DateTime operations if it was not provided explicitly. For example, if you add one second to a value of
DateTime
type without timezone it will remainDateTime
without timezone. In previous versions the value of default timezone was placed to the returned data type explicitly so it becomes DateTime('something'). This closes #4854. #23392 (Alexey Milovidov). - Previously, MySQL 5.7.9 was not supported due to SQL incompatibility. Now leave MySQL parameter verification to the MaterializeMySQL. #23413 (TCeason).
- Possibility to change S3 disk settings in runtime via new
SYSTEM RESTART DISK
SQL command. #23429 (Pavel Kovalenko). - Respect lock_acquire_timeout_for_background_operations for OPTIMIZE. #23623 (Azat Khuzhin).
- Make big integers production ready. Add support for
UInt128
data type. Fix known issues with theDecimal256
data type. Support big integers in dictionaries. Supportgcd
/lcm
functions for big integers. Support big integers in array search and conditional functions. SupportLowCardinality(UUID)
. Support big integers ingenerateRandom
table function andclickhouse-obfuscator
. Fix error with returningUUID
from scalar subqueries. This fixes #7834. This fixes #23936. This fixes #4176. This fixes #24018. This fixes #17828. Backward incompatible change: values ofUUID
type cannot be compared with integer. For example, instead of writinguuid != 0
typeuuid != '00000000-0000-0000-0000-000000000000'
. #23631 (Alexey Milovidov). - Add
_partition_value
virtual column to MergeTree table family. It can be used to prune partition in a deterministic way. It's needed to implement partition matcher for mutations. #23673 (Amos Bird). - Enable
async_socket_for_remote
by default. #23683 (Nikolai Kochetov). - When there is some
ReplicatedMergeTree
tables whose zookeeper is expired, it will throw the error below when we select the meta data of some table fromsystem.tables
withselect_sequential_consistency
is enabled:Session expired (Session expired): While executing Tables
. #23793 (Fuwang Hu). - Added
region
parameter for S3 storage and disk. #23846 (Vladimir Chebotarev). - Allow configuring different log levels for different logging channels. Closes #19569. #23857 (filimonov).
- Add
broken_data_files
/broken_data_compressed_bytes
intosystem.distribution_queue
. Add metric for number of files for asynchronous insertion into Distributed tables that has been marked as broken (BrokenDistributedFilesToInsert
). #23885 (Azat Khuzhin). - Allow to add specific queue settings via table settng
rabbitmq_queue_settings_list
. (Closes #23737 and #23918). Allow user to control all RabbitMQ setup: if table settingrabbitmq_queue_consume
is set to1
- RabbitMQ table engine will only connect to specified queue and will not perform any RabbitMQ consumer-side setup like declaring exchange, queues, bindings. (Closes #21757). Add proper cleanup when RabbitMQ table is dropped - delete queues, which the table has declared and all bound exchanges - if they were created by the table. #23887 (Kseniia Sumarokova). - Measure found rate (the percentage for which the value was found) for dictionaries (see
found_rate
insystem.dictionaries
). #23916 (Azat Khuzhin). - Add hints for Enum names. Closes #17112. #23919 (flynn).
- Add support for HTTP compression (determined by
Content-Encoding
HTTP header) inhttp
dictionary source. This fixes #8912. #23946 (Filatenkov Artur). - Preallocate support for hashed/sparse_hashed dictionaries. #23979 (Azat Khuzhin).
- Support specifying table schema for postgresql dictionary source. Closes #23958. #23980 (Kseniia Sumarokova).
- Log information about OS name, kernel version and CPU architecture on server startup. #23988 (Azat Khuzhin).
- enable DateTime64 to be a version column in ReplacingMergeTree. #23992 (kevin wan).
- Add support for
ORDER BY WITH FILL
withDateTime64
. #24016 (kevin wan). - Now
prefer_column_name_to_alias = 1
will also favor column names forgroup by
,having
andorder by
. This fixes #23882. #24022 (Amos Bird). - Do not acquire lock for total_bytes/total_rows for Buffer engine. #24066 (Azat Khuzhin).
- Flush Buffer tables before shutting down tables (within one database), to avoid discarding blocks due to underlying table had been already detached (and
Destination table default.a_data_01870 doesn't exist. Block of data is discarded
error in the log). #24067 (Azat Khuzhin). - Preserve dictionaries until storage shutdown (this will avoid possible
external dictionary 'DICT' not found
errors at server shutdown during final Buffer flush). #24068 (Azat Khuzhin). - Update zstd to v1.5.0. #24135 (Raúl Marín).
- Fix crash when memory allocation fails in simdjson. https://github.com/simdjson/simdjson/pull/1567 . Mark as improvement because it's a rare bug. #24147 (Amos Bird).
Bug Fix
- This PR fixes a crash on shutdown which happened because of currentConnections() could return zero while some connections were still alive. #23154 (Vitaly Baranov).
- QueryAliasVisitor to prefer alias for ASTWithAlias if subquery was optimized to constant. Fixes #22924. Fixes #10401. #23191 (Maksim Kita).
- Fixed
Not found column
error when selecting fromMaterializeMySQL
with condition on key column. Fixes #22432. #23200 (Alexander Tokmakov). - Fixed the behavior when disabling
input_format_with_names_use_header
setting discards all the input with CSVWithNames format. This fixes #22406. #23202 (Nikita Mikhaylov). - Add type conversion for optimize_skip_unused_shards_rewrite_in (fixes
use-of-uninitialized-value
withoptimize_skip_unused_shards_rewrite_in
). #23219 (Azat Khuzhin). - Fixed simple key dictionary from DDL creation if primary key is not first attribute. Fixes #23236. #23262 (Maksim Kita).
- Fixed very rare (distributed) race condition between creation and removal of ReplicatedMergeTree tables. It might cause exceptions like
node doesn't exist
on attempt to create replicated table. Fixes #21419. #23294 (Alexander Tokmakov). - Fixed very rare race condition on background cleanup of old blocks. It might cause a block not to be deduplicated if it's too close to the end of deduplication window. #23301 (Alexander Tokmakov).
- Fix possible crash in case if
unknown packet
was received form remote query (withasync_socket_for_remote
enabled). Maybe fixes #21167. #23309 (Nikolai Kochetov). - Don't relax NOT conditions during partition pruning. This fixes #23305 and #21539. #23310 (Amos Bird).
-
- Fix bug in dict join with join_algorithm = 'auto'. Close #23002. #23312 (Vladimir C).
- Fix possible
Block structure mismatch
error for queries withUNION
which could possibly happen after filter-push-down optimization. Fixes #23029. #23359 (Nikolai Kochetov). - Fix incompatible constant expression generation during partition pruning based on virtual columns. This fixes https://github.com/ClickHouse/ClickHouse/pull/21401#discussion_r611888913. #23366 (Amos Bird).
ORDER BY
withCOLLATE
was not working correctly if the column is in primary key (or is a monotonic function of it) and the settingoptimize_read_in_order
is not turned off. This closes #22379. Workaround for older versions: turn the settingoptimize_read_in_order
off. #23375 (Alexey Milovidov).- Remove support for
argMin
andargMax
for singleTuple
argument. The code was not memory-safe. The feature was added by mistake and it is confusing for people. These functions can be reintroduced under different names later. This fixes #22384 and reverts #17359. #23393 (Alexey Milovidov). - Allow to move more conditions to
PREWHERE
as it was before version 21.1. Insufficient number of moved condtions could lead to worse performance. #23397 (Anton Popov). - Kafka storage may support parquet format messages. #23412 (Chao Ma).
- Kafka storage may support
arrow
andarrowstream
format messages. #23415 (Chao Ma). - Fixed
Cannot unlink file
error on unsuccessful creation of ReplicatedMergeTree table with multidisk configuration. This closes #21755. #23433 (Alexander Tokmakov). -
- Bug fix for
deltaSum
aggregate function in counter reset case ... #23437 (Russ Frank).
- Bug fix for
- Fix bug that does not allow cast from empty array literal, to array with dimensions greater than 1. Closes #14476. #23456 (Maksim Kita).
- Fix corner cases in vertical merges with
ReplacingMergeTree
. In rare cases they could lead to fails of merges with exceptions likeIncomplete granules are not allowed while blocks are granules size
. #23459 (Anton Popov). - When modify column's default value without datatype, and this column is used as ReplacingMergeTree's parameter like column
b
in the below example, then the server will core dump:CREATE TABLE alter_test (a Int32, b DateTime) ENGINE = ReplacingMergeTree(b) ORDER BY a; ALTER TABLE alter_test MODIFY COLUMN `b` DEFAULT now();
the sever throw error:2021.04.22 09:48:00.685317 [ 2607 ] {} <Trace> BaseDaemon: Received signal 11 2021.04.22 09:48:00.686110 [ 2705 ] {} <Fatal> BaseDaemon: ######################################## 2021.04.22 09:48:00.686336 [ 2705 ] {} <Fatal> BaseDaemon: (version 21.6.1.1, build id: 6459E84DFCF8E778546C5AD2FFE91B3AD71E1B1B) (from thread 2619) (no query) Received signal Segmentation fault (11) 2021.04.22 09:48:00.686572 [ 2705 ] {} <Fatal> BaseDaemon: Address: NULL pointer. Access: read. Address not mapped to object. 2021.04.22 09:48:00.686686 [ 2705 ] {} <Fatal> BaseDaemon: Stack trace: 0x1c2585d7 0x1c254f66 0x1bb7e403 0x1bb58923 0x1bb56a85 0x1c6840ef 0x1c691148 0x2061a05c 0x2061a8e4 0x20775a03 0x207722bd 0x20771048 0x7f6e5c25be25 0x7f6e5bd81bad 2021.04.22 09:48:02.283045 [ 2705 ] {} <Fatal> BaseDaemon: 4. /mnt/disk4/hewenting/ClickHouse/src/src/Storages/MergeTree/MergeTreeData.cpp:1449: DB::(anonymous namespace)::checkVersionColumnTypesConversion(DB::IDataType const*, DB::IDataType const*, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >) @ 0x1c2585d7 in /mnt/disk4/hewenting/ClickHouse/build-dbgsrc-clang-dev-nested/programs/clickhouse-server 2021.04.22 09:48:03.714451 [ 2705 ] {} <Fatal> BaseDaemon: 5. /mnt/disk4/hewenting/ClickHouse/src/src/Storages/MergeTree/MergeTreeData.cpp:1582: DB::MergeTreeData::checkAlterIsPossible(DB::AlterCommands const&, std::__1::shared_ptr<DB::Context>) const @ 0x1c254f66 in /mnt/disk4/hewenting/ClickHouse/build-dbgsrc-clang-dev-nested/programs/clickhouse-server 2021.04.22 09:48:04.692949 [ 2705 ] {} <Fatal> BaseDaemon: 6. /mnt/disk4/hewenting/ClickHouse/src/src/Interpreters/InterpreterAlterQuery.cpp:144: DB::InterpreterAlterQuery::execute() @ 0x1bb7e403 in /mnt/disk4/hewenting/ClickHouse/build-dbgsrc-clang-dev-nested/programs/clickhouse-server
. #23483 (hexiaoting). - Fix
columns
function when multiple joins in select query. Closes #22736. #23501 (Maksim Kita). -
- Fix bug with
Join
andWITH TOTALS
, close #17718. #23549 (Vladimir C).
- Fix bug with
- Fix restart / stop command hanging. Closes #20214. #23552 (filimonov).
- Fix misinterpretation of some
LIKE
expressions with escape sequences. #23610 (Alexey Milovidov). - Fixed server fault when inserting data through HTTP caused an exception. This fixes #23512. #23643 (Nikita Mikhaylov).
- Added an exception in case of completely the same values in both samples in aggregate function
mannWhitneyUTest
. This fixes #23646. #23654 (Nikita Mikhaylov). - Fixed a bug in recovery of staled
ReplicatedMergeTree
replica. Some metadata updates could be ignored by staled replica ifALTER
query was executed during downtime of the replica. #23742 (Alexander Tokmakov). - Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. #23744 (Azat Khuzhin).
- Fix
heap_use_after_free
when reading from hdfs ifValues
format is used. #23761 (Kseniia Sumarokova). - Fix crash when
PREWHERE
and row policy filter are both in effect with empty result. #23763 (Amos Bird). - Fixed remote JDBC bridge timeout connection issue. Closes #9609. #23771 (Maksim Kita).
- Fix
CLEAR COLUMN
does not work when it is referenced by materialized view. Close #23764. #23781 (flynn). - Fix error
Can't initialize pipeline with empty pipe
for queries withGLOBAL IN/JOIN
anduse_hedged_requests
. Fixes #23431. #23805 (Nikolai Kochetov). - Better handling of URI's in
PocoHTTPClient
. Fixed bug with URLs containing+
symbol, data with such keys could not be read previously. #23822 (Vladimir Chebotarev). - HashedDictionary complex key update field initial load fix. Closes #23800. #23824 (Maksim Kita).
- Better handling of HTTP errors in
PocoHTTPClient
. Response bodies of HTTP errors were being ignored earlier. #23844 (Vladimir Chebotarev). - Fix
distributed_group_by_no_merge=2
withGROUP BY
and aggregate function wrapped into regular function (had been broken in #23546). Throw exception in case of someone trying to usedistributed_group_by_no_merge=2
with window functions. Disableoptimize_distributed_group_by_sharding_key
for queries with window functions. #23906 (Azat Khuzhin). - Fix implementation of connection pool of PostgreSQL engine. Closes #23897. #23909 (Kseniia Sumarokova).
- Fix keys metrics accounting for CACHE() dictionary with duplicates in the source (leads to
DictCacheKeysRequestedMiss
overflows). #23929 (Azat Khuzhin). - Fix SIGSEGV for external GROUP BY and overflow row (i.e. queries like
SELECT FROM GROUP BY WITH TOTALS SETTINGS max_bytes_before_external_group_by>0, max_rows_to_group_by>0, group_by_overflow_mode='any', totals_mode='before_having'
). #23962 (Azat Khuzhin). - Some
ALTER PARTITION
queries might causePart A intersects previous part B
andUnexpected merged part C intersecting drop range D
errors in replication queue. It's fixed. Fixes #23296. #23997 (Alexander Tokmakov). - Fix crash in MergeJoin, close #24010. #24013 (Vladimir C).
- now64() supports optional timezone argument ... #24091 (Vasily Nemkov).
- Fixed using const
DateTime
value vsDateTime64
column in WHERE. ... #24100 (Vasily Nemkov). - Bug: explain pipeline with
select xxx final
shows wrong pipeline: ``` dell123 :) explain pipeline select z from prewhere_move_select_final final;. #24116 (hexiaoting). - Fix a rare bug that could lead to a partially initialized table that can serve write requests (insert/alter/so on). Now such tables will be in readonly mode. #24122 (alesapin).
- Fix race condition which could happen in RBAC under a heavy load. This PR fixes #24090, #24134,. #24176 (Vitaly Baranov).
- Update nested column with const condition will make server crash. ``` CREATE TABLE test_wide_nested (
id
Int,info.id
Array(Int),info.name
Array(String),info.age
Array(Int) ) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; set mutations_sync = 1;. #24183 (hexiaoting). - Fix abnormal server termination due to hdfs becoming not accessible during query execution. Closes #24117. #24191 (Kseniia Sumarokova).
- Fix wrong typo at StorageMemory, this bug was introduced at #15127, now fixed, Closes #24192. #24193 (张中南).
Build/Testing/Packaging Improvement
- Adding Map type tests in TestFlows. #21087 (vzakaznikov).
- Testflows tests for DateTime64 Extended Range. #22729 (Andrey Zvonov).
- CMake will be failed with settings as bellow
-DENABLE_CASSANDRA=OFF -DENABLE_AMQPCPP=ON
... #22984 (Ben). - Add simple tool for benchmarking [Zoo]Keeper. #23038 (alesapin).
- Remove a source of nondeterminism from build. Now builds at different point of time will produce byte-identical binaries. Partially addressed #22113. #23559 (Alexey Milovidov).
- Avoid possible build dependency on locale and filesystem order. This allows reproducible builds. #23600 (Alexey Milovidov).
- Always enable asynchronous-unwind-tables explicitly. It may fix query profiler on AArch64. #23602 (Alexey Milovidov).
- Fix Memory Sanitizer report in GRPC library. This closes #19234. #23615 (Alexey Milovidov).
- Window functions tests in TestFlows. #23704 (vzakaznikov).
- Adds support for building on Solaris-derived operating systems. #23746 (bnaecker).
- Update librdkafka 1.6.0-RC3 to 1.6.1. #23874 (filimonov).
- Enabling running of all TestFlows modules in parallel. #23942 (vzakaznikov).
- Fixing window functions distributed tests by moving to a deterministic sharding key. #23975 (vzakaznikov).
- Add more benchmarks for hash tables, including the Swiss Table from Google (that appeared to be slower than ClickHouse hash map in our specific usage scenario). #24111 (Maksim Kita).
- Support building on Illumos. #24144 (bnaecker).
Other
- Automated backporting now looks at the label 'release' of PRs to consider the release branch. #23363 (Ivan).
- Add test cases for arrayElement. related issue: #22765. #23484 (hexiaoting).
- Rename uniqThetaSketch (https://github.com/ClickHouse/ClickHouse/issues/14893) to uniqTheta. #24019 (Kruglov Pavel).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Function
arrayFold
for folding over array with accumulator"'. #23248 (Alexey Milovidov). - NO CL ENTRY: 'Revert "[RFC] Fix memory tracking with min_bytes_to_use_mmap_io"'. #23276 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "add uniqThetaSketch"'. #23334 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Fix CI build for gcc-10"'. #23772 (Alexey Milovidov).
- NO CL ENTRY: 'Update syntax.md'. #24267 (lulichao).
NOT FOR CHANGELOG / INSIGNIFICANT
- Apply idle_connnection_timeout/poll_interval after each query #21938 (Azat Khuzhin).
- Do not silently catch errors for writing to S3 #22208 (Azat Khuzhin).
- Add dockerhub-proxy to runner #23138 (Ilya Yatsishin).
- merging sumCount fusion PR #21337 #23159 (Alexander Kuzmenkov).
- Minor fixes in ATTACH query #23189 (Alexander Tokmakov).
- Merging #22503 #23195 (Anton Popov).
- Fix logical error in stress tests #23197 (Nikita Mikhaylov).
- more stable formatting for negate() #23201 (Alexander Kuzmenkov).
- Use double quote identifier in odbc as default in case of error #23217 (Kseniia Sumarokova).
- Add -Wundef for gcc builds #23258 (Azat Khuzhin).
- Report an error if jemalloc.background_thread was requested #23259 (Azat Khuzhin).
- Add trace_log into stateless/stress test artifacts #23264 (Azat Khuzhin).
- Zlib use attribute constructor for functable initialization #23266 (Maksim Kita).
- Fix integration tests for Hedged requests #23275 (Alexey Milovidov).
- Function toDateTime decimal overflow ubsan fix #23278 (Maksim Kita).
- Link keeper-bench to clickhouse_common_zookeeper #23302 (Raúl Marín).
- Print errors on db creation in clickhouse-test #23304 (Alexander Tokmakov).
- fix broken perf test #23308 (Alexander Kuzmenkov).
- Fix bad test 01602_max_distributed_connections #23317 (Alexey Milovidov).
- Fix hdfs reading from files with spaces #23318 (Kseniia Sumarokova).
- add check that p.pruning works #23321 (Denny Crane).
- Add test for #7815 #23332 (Alexey Milovidov).
- Fix flaky test 01666_merge_tree_max_query_limit.sh #23335 (Alexey Milovidov).
- Add test for #12077 #23352 (Kseniia Sumarokova).
- Disable clickhouse-odbc-bridge build when ODBC is disabled #23357 (Denis Glazachev).
- Fix AppleClang build #23358 (Denis Glazachev).
- Use Atomic database for development environment #23377 (Alexey Milovidov).
- Add test for #2582 #23378 (Alexey Milovidov).
- Add test for #1647 #23379 (Alexey Milovidov).
- Allow rabbitmq vhost in table settings #23452 (Kseniia Sumarokova).
- System dictionaries virtual key column #23458 (Maksim Kita).
- ISSUES-23310 Try fix MySQL 8.0 address already in use #23462 (Winter Zhang).
- Fix error in perf test #23469 (Alexey Milovidov).
- clickhouse-test: print database name on failures #23486 (Azat Khuzhin).
- upload cpu model to perf test db #23514 (Alexander Kuzmenkov).
- Fix function tests flaps #23517 (Azat Khuzhin).
- fix pvs warnings #23520 (Alexander Kuzmenkov).
- ignore empty input chunks generated by joins #23542 (Alexander Kuzmenkov).
- fix window functions for Distributed tables #23546 (Alexander Kuzmenkov).
- add more info to perf test report #23550 (Alexander Kuzmenkov).
- export trace log from stateless tests in flamegraph-friendly format #23553 (Alexander Kuzmenkov).
- add numactl info to perf test run attributes #23554 (Alexander Kuzmenkov).
- Less chance of OOM in stress tests #23561 (Alexey Milovidov).
- Fix inconsistent formatting for tupleElement (for fuzzer) #23595 (Azat Khuzhin).
- Remove unneeded code from CMakeLists #23597 (Alexey Milovidov).
- Somewhat better handling of paths in CMake (incomplete) #23598 (Alexey Milovidov).
- Remove old trash (a little) #23599 (Alexey Milovidov).
- Remove some garbage from RocksDB CMakeLists #23601 (Alexey Milovidov).
- Add warning about gcc usage #23603 (Alexey Milovidov).
- Remove garbage from CMakeLists (2) #23604 (Alexey Milovidov).
- Remove rotten parts of release script #23605 (Alexey Milovidov).
- Remove useless file #23606 (Alexey Milovidov).
- Skip CatBoost tests under MSan #23614 (Alexey Milovidov).
- Fix segfault in TSan on _exit #23616 (Alexey Milovidov).
- Fix bad test 01641_memory_tracking_insert_optimize_long #23617 (Alexey Milovidov).
- Remove pbuilder #23618 (Alexey Milovidov).
- Aggregator remove unused code #23635 (Maksim Kita).
- Merging #22984 #23637 (Alexander Tokmakov).
- Move non gtest unit tests to /examples folder #23644 (Nikita Mikhaylov).
- Kerberized HDFS test is fluky - retries and extra output if failure #23650 (Ilya Golshtein).
- Fix documentation for DETACH ON CLUSTER PERMANENTLY #23653 (Azat Khuzhin).
- Skip integration test for library bridge under MSan #23662 (Alexey Milovidov).
- Fix a few PVS-Studio warnings #23663 (Alexey Milovidov).
- Enable use-after-destruction detection in MSan #23664 (Azat Khuzhin).
- Fix cyrus-sasl msan warning #23672 (Ilya Yatsishin).
- A little bit faster merge of aggregating states. #23681 (Nikolai Kochetov).
- stable query indexes in perf test #23707 (Alexander Kuzmenkov).
- Minor changes in code #14254 #23709 (Alexey Milovidov).
- Fix illiterate error message #23700 #23710 (Alexey Milovidov).
- Remove obsolete compilers #23711 (Alexey Milovidov).
- Follow-up for #23644 #23712 (Alexey Milovidov).
- Disable test_odbc_interaction for MSan build #23722 (Vladimir C).
- fix a typo in query formatting check in fuzzer #23726 (Alexander Kuzmenkov).
- ExternalLoaderRepository fix arcadia #23732 (Maksim Kita).
- Ignore cmake-in-clickhouse #23740 (Nikolai Kochetov).
- Refactor join step #23743 (Nikolai Kochetov).
- clickhouse-test: send TERM to all childs (to avoid hung check triggering) #23750 (Azat Khuzhin).
- Improve test_insert_into_distributed #23751 (Azat Khuzhin).
- Fix CI build for gcc-10 #23760 (Maksim Kita).
- Update array-functions.md #23762 (fancno).
- Remove unused compilers (fixed for the troublesome "unbundled" build) #23766 (Maksim Kita).
- Slightly better hardening for intersecting parts #23767 (Alexander Tokmakov).
- Workaround for PVS-Studio #23776 (Alexey Milovidov).
- Add examples folder filter to ya.make.in #23786 (Maksim Kita).
- Function default implementation for nulls small optimization #23799 (Maksim Kita).
- autodetect arch of gosu in server dockerfile #23802 (filimonov).
- Add test for #18170 #23813 (Nikita Mikhaylov).
- clickhouse-test: add missing whitespace before printing database on error #23820 (Azat Khuzhin).
- Improve 00840_long_concurrent_select_and_drop_deadlock #23823 (Azat Khuzhin).
- Flat, Hashed dictionary include update field bytes into bytes_allocated #23825 (Maksim Kita).
- XDBCBridgeHelper use global context #23836 (Maksim Kita).
- gcc-10 installation no-install-reccomends option fix #23840 (Maksim Kita).
- replxx readline compatibility #23855 (Azat Khuzhin).
- Add file paths into logs on failed distributed async sends #23856 (Azat Khuzhin).
- Reduce the amount of logs that StorageMergeTree::selectPartsToMutate outputs in busy systems. #23863 (Raúl Marín).
- Add DiskRestartProxy.cpp to ya.make #23868 (Vladimir C).
- Fix some warnings by PVS-Studio #23877 (Alexey Milovidov).
- Add Read/WriteBufferFromFileDecorator.cpp to ya.make #23879 (Vladimir C).
- @CurtizJ convinced me that this test has to be deleted #23883 (Alexey Milovidov).
- bash completion improvements #23884 (Azat Khuzhin).
- Remove obsolete code #23886 (Alexey Milovidov).
- Minor code simplification (implemented TODO) #23896 (Alexey Milovidov).
- Add run options with default config path according to issues/23875 #23898 (ice1x).
- Fix test_insert_into_distributed flaps #23903 (Azat Khuzhin).
- Fix bad test about compression codecs #23908 (Alexey Milovidov).
- Cleanup IDatabase.h from extra headers #23912 (Azat Khuzhin).
- Minor simplification #23923 (Alexey Milovidov).
- Drop unnecessary ports where applicable #23928 (Ernest Zaslavsky).
- Check MIN/MAX attributes in the list of dictionary attributes #23948 (Azat Khuzhin).
- typo: fix a typo in Compression/CodecT64 #23952 (mwish).
- Don't try GLIBC_COMPATIBILITY for i686 Linux #23959 (divanorama).
- Function arrayDifference decimal math overflow #23961 (Maksim Kita).
- Use 0 over nan for hit_rate in case of 0 queries to the cache dictionary #23963 (Azat Khuzhin).
- Round floats in Aggregator log messages #23965 (Azat Khuzhin).
- support longer query ids in trace log for perf tests #23969 (Alexander Kuzmenkov).
- PVS-Studio fixes, part 6 #23970 (Alexey Milovidov).
- Functional stateless tests fix numbers #23974 (Maksim Kita).
- use LowCardinality for AsynchronousMetricLog name column #23981 (flynn).
- Function dictGetOrNull handle empty rows execute #23990 (Maksim Kita).
- CompileDAG fix Sip hash #24004 (Maksim Kita).
- Fix bad code #24007 (Alexey Milovidov).
- Better error codes in Keeper when no leader alive #24017 (alesapin).
- Fix ArenaWithFreeLists test #24021 (Maksim Kita).
- Run check_*_compiler_flag earlier #24037 (Amos Bird).
- Performance tests disable compile expressions #24043 (Maksim Kita).
- for trivial INSERT SELECT, adjust block size in bytes as well #24048 (Alexander Kuzmenkov).
- Enable thread_local in Arcadia build #24051 (Yuriy Chernyshov).
- lower two-level aggregation threshold for uniq test to avoid jitter #24058 (Alexander Kuzmenkov).
- Fix empty key projection query analysis #24062 (Amos Bird).
- Generate ya.make for missing UUID.cpp #24064 (Alexander Gololobov).
- bash-completion: complete available formats #24065 (Azat Khuzhin).
- Fix concurrent snapshot read/write #24073 (alesapin).
- Also retry database creation in
clickhouse-test
#24088 (alesapin). - Calculate header from ActionsDAG #24108 (Nikolai Kochetov).
- Fix arcadia #24133 (Nikita Mikhaylov).
- TestReadAfterAIO: Use the current path instead of /tmp for temporal files #24139 (Raúl Marín).
- Fix a few trailing whitespaces in output #24150 (Azat Khuzhin).
- IFunction refactoring #24155 (Maksim Kita).
- Fix add projection to replicated mergetree #24162 (Amos Bird).
- Fix distributed processing when using projection #24168 (Amos Bird).
- fix tiny code style #24169 (flynn).
- fix reinterpretAsFixedString for UUID #24177 (flynn).
- Function move file #24178 (Maksim Kita).
- Updated LRUHashMap benchmarks #24182 (Maksim Kita).
- Remove temporary files #24186 (Kruglov Pavel).
- Try mute grpc msan #24197 (Nikolai Kochetov).
- Update slack link #24200 (Ilya Yatsishin).
- Simplier isLocalAddress #24203 (alesapin).
- Increase timeout for server restart in integration tests. #24205 (Nikolai Kochetov).
- Addition to #23997 #24208 (Alexander Tokmakov).
- Small code simplification #24210 (Maksim Kita).
- Speedup test [test_jbod_balancer/test.py] #24247 (alesapin).
- remove useless code #24248 (flynn).
- One more attempt to fix retries in clickhouse-test #24249 (alesapin).
- Log exception in Allocator::free #24256 (Nikolai Kochetov).
- Fix broken HTML markup on website #24265 (Alexey Milovidov).
- Fix arcadia #24282 (Nikita Mikhaylov).
- Fixed Arcadia after IFunctionOverloadResolver interface refactoring #24284 (Maksim Kita).
- Fix isLocalAddress() (ifa_addr maybe NULL) #24308 (Azat Khuzhin).
- Fix bad test for Enum hints #24313 (Alexey Milovidov).
New Feature #14893
-
- Add uniqThetaSketch to support Theta Sketch in ClickHouse. #22609 (Ping Yu).
-
- Add uniqThetaSketch to support Theta Sketch in ClickHouse. #23894 (Ping Yu).