- A fix for a feature that previously had unwanted behaviour. Do not allow direct select for Kafka/RabbitMQ/FileLog. Can be enabled by setting
stream_like_engine_allow_direct_select
. Direct select will be not allowed even if enabled by setting, in case there is an attached materialized view. For Kafka and RabbitMQ direct selectm if allowed, will not commit massages by default. To enable commits with direct select, user must use storage level settingkafka{rabbitmq}_commit_on_select=1
(default0
). #31053 (Kseniia Sumarokova). - A slight change in behaviour of a new function. Return unquoted string in JSON_VALUE. Closes #27965. #31008 (Kseniia Sumarokova).
- Setting rename. Add custom null representation support for TSV/CSV input formats. Fix deserialing Nullable(String) in TSV/CSV/JSONCompactStringsEachRow/JSONStringsEachRow input formats. Rename
output_format_csv_null_representation
andoutput_format_tsv_null_representation
toformat_csv_null_representation
andformat_tsv_null_representation
accordingly. #30497 (Kruglov Pavel). - Further deprecation of already unused code. This is relevant only for users of ClickHouse versions older than 20.6. A "leader election" mechanism is removed from
ReplicatedMergeTree
, because multiple leaders are supported since 20.6. If you are upgrading from an older version and some replica with an old version is a leader, then server will fail to start after upgrade. Stop replicas with old version to make new version start. After that it will not be possible to downgrade to version older than 20.6. #32140 (tavplubix).
- Implemented more of the ZooKeeper Four Letter Words commands in clickhouse-keeper: https://zookeeper.apache.org/doc/r3.4.8/zookeeperAdmin.html#sc_zkCommands. #28981 (JackyWoo). Now
clickhouse-keeper
is feature complete. - Support for
Bool
data type. #31072 (kevin wan). - Support for
PARTITION BY
in File, URL, HDFS storages and withINSERT INTO
table function. Closes #30273. #30690 (Kseniia Sumarokova). - Added
CONSTRAINT ... ASSUME ...
(without checking duringINSERT
). Added query transformation to CNF (ClickHouse#11749) for more convenient optimization. Added simple query rewriting using constraints (only simple matching now, will be improved to support <,=,>... later). Added ability to replace heavy columns with light columns if it's possible. #18787 (Nikita Vasilev). - Basic access authentication for http/url functions. #31648 (michael1589).
- Support
INTERVAL
type inSTEP
clause forWITH FILL
modifier. #30927 (Anton Popov). - Add support for parallel reading from multiple files and support globs in
FROM INFILE
clause. #30135 (Filatenkov Artur). - Add support for
Identifier
table and database query parameters. Closes #27226. #28668 (Nikolay Degterinsky). - TLDR: Major improvements of completeness and consistency of text formats. Refactor formats
TSV
,TSVRaw
,CSV
andJSONCompactEachRow
,JSONCompactStringsEachRow
, remove code duplication, add base interface for formats with-WithNames
and-WithNamesAndTypes
suffixes. Add formatsCSVWithNamesAndTypes
,TSVRawWithNames
,TSVRawWithNamesAndTypes
,JSONCompactEachRowWIthNames
,JSONCompactStringsEachRowWIthNames
,RowBinaryWithNames
. Support parallel parsing for formatsTSVWithNamesAndTypes
,TSVRaw(WithNames/WIthNamesAndTypes)
,CSVWithNamesAndTypes
,JSONCompactEachRow(WithNames/WIthNamesAndTypes)
,JSONCompactStringsEachRow(WithNames/WIthNamesAndTypes)
. Support columns mapping and types checking forRowBinaryWithNamesAndTypes
format. Add settinginput_format_with_types_use_header
which specify if we should check that types written in <format_name>WIthNamesAndTypes
format matches with table structure. Add settinginput_format_csv_empty_as_default
and use it in CSV format instead ofinput_format_defaults_for_omitted_fields
(because this setting should not controlcsv_empty_as_default
). Fix usage of settinginput_format_defaults_for_omitted_fields
(it was used only ascsv_empty_as_default
, but it should control calculation of default expressions for omitted fields). Fix Nullable input/output inTSVRaw
format, make this format fully compatible with inserting into TSV. Fix inserting NULLs inLowCardinality(Nullable)
wheninput_format_null_as_default
is enabled (previously default values was inserted instead of actual NULLs). Fix strings deserialization inJSONStringsEachRow
/JSONCompactStringsEachRow
formats (strings were parsed just until first '\n' or '\t'). Add ability to useRaw
escaping rule in Template input format. Add diagnostic info for JSONCompactEachRow(WithNames/WIthNamesAndTypes) input format. Fix bug with parallel parsing of-WithNames
formats in case when settingmin_chunk_bytes_for_parallel_parsing
is less than bytes in a single row. #30178 (Kruglov Pavel). Allow to print/parse names and types of colums inCustomSeparated
input/output format. Add formatsCustomSeparatedWithNames/WithNamesAndTypes
similar toTSVWithNames/WithNamesAndTypes
. #31434 (Kruglov Pavel). - Aliyun OSS Storage support. #31286 (cfcz48).
- Exposes all settings of the global thread pool in the configuration file. #31285 (Tomáš Hromada).
- Introduced window functions
exponentialTimeDecayedSum
,exponentialTimeDecayedMax
,exponentialTimeDecayedCount
andexponentialTimeDecayedAvg
which are more effective thanexponentialMovingAverage
for bigger windows. Also more use-cases were covered. #29799 (Vladimir Chebotarev). - Add option to compress logs before writing them to a file using LZ4. Closes #23860. #29219 (Nikolay Degterinsky).
- Support
JOIN ON 1 = 1
that have CROSS JOIN semantic. This closes #25578. #25894 (Vladimir C). - Add Map combinator for
Map
type. - Rename oldsum-, min-, max- Map
for mapped arrays tosum-, min-, max- MappedArrays
. #24539 (Ildus Kurbangaliev). - Make reading from HTTP retriable. Closes #29696. #29894 (Kseniia Sumarokova).
WINDOW VIEW
to enable stream processing in ClickHouse. #8331 (vxider).- Drop support for using Ordinary databases with
MaterializedMySQL
. #31292 (Stig Bakken). - Implement the commands BACKUP and RESTORE for the Log family. This feature is under development. #30688 (Vitaly Baranov).
- Reduce memory usage when reading with
s3
/url
/hdfs
formatsParquet
,ORC
,Arrow
(controlled by settinginput_format_allow_seeks
, enabled by default). Also add settingremote_read_min_bytes_for_seek
to control seeks. Closes #10461. Closes #16857. #30936 (Kseniia Sumarokova). - Add optimizations for constant conditions in JOIN ON, ref #26928. #27021 (Vladimir C).
- Support parallel formatting for all text formats, except
JSONEachRowWithProgress
andPrettyCompactMonoBlock
. #31489 (Kruglov Pavel). - Speed up count over nullable columns. #31806 (Raúl Marín).
- Speed up
avg
andsumCount
aggregate functions. #31694 (Raúl Marín). - Improve performance of JSON and XML output formats. #31673 (alexey-milovidov).
- Improve performance of syncing data to block device. This closes #31181. #31229 (zhanglistar).
- Fixing query performance issue in
LiveView
tables. Fixes #30831. #31006 (vzakaznikov). - Speed up query parsing. #31949 (Raúl Marín).
- Allow to split
GraphiteMergeTree
rollup rules for plain/tagged metrics (optionalrule_type
field). #25122 (Michail Safronov). - Remove excessive
DESC TABLE
requests forremote()
(in case ofremote('127.1', system.one)
(i.e. identifier as the db.table instead of string) there was excessiveDESC TABLE
request). #32019 (Azat Khuzhin). - Optimize function
tupleElement
to reading of subcolumn with enabled settingoptimize_functions_to_subcolumns
. #31261 (Anton Popov). - Optimize function
mapContains
to reading of subcolumnkey
with enabled settingsoptimize_functions_to_subcolumns
. #31218 (Anton Popov). - Add settings
merge_tree_min_rows_for_concurrent_read_for_remote_filesystem
andmerge_tree_min_bytes_for_concurrent_read_for_remote_filesystem
. #30970 (Kseniia Sumarokova). - Skipping mutations of different partitions in
StorageMergeTree
. #21326 (Vladimir Chebotarev).
- Do not allow to drop a table or dictionary if some tables or dictionaries depend on it. #30977 (tavplubix).
- Allow versioning of aggregate function states. Now we can introduce backward compatible changes in serialization format of aggregate function states. Closes #12552. #24820 (Kseniia Sumarokova).
- Support PostgreSQL style
ALTER MODIFY COLUMN
syntax. #32003 (SuperDJY). - Added
update_field
support forRangeHashedDictionary
,ComplexKeyRangeHashedDictionary
. #32185 (Maksim Kita). - The
murmurHash3_128
andsipHash128
functions now accept an arbitrary number of arguments. This closes #28774. #28965 (小路). - Support default expression for
HDFS
storage and optimize fetching when source is column oriented. #32256 (李扬). - Improve the operation name of an opentelemetry span. #32234 (Frank Chen).
- Use
Content-Type: application/x-ndjson
(http://ndjson.org/) for output formatJSONEachRow
. #32223 (Dmitriy Dorofeev). - Improve skipping unknown fields with quoted escaping rule in Template/CustomSeparated formats. Previously you could skip only quoted strings, now you can skip values with any type. #32204 (Kruglov Pavel).
- Now
clickhouse-keeper
refuses to start or apply configuration changes when they contain duplicated IDs or endpoints. Fixes #31339. #32121 (alesapin). - Set Content-Type in HTTP packets issued from URL engine. #32113 (Frank Chen).
- Return Content-Type as 'application/json' for
JSONEachRow
format ifoutput_format_json_array_of_rows
is enabled. #32112 (Frank Chen). - Allow to parse
+
beforeFloat32
/Float64
values. #32079 (Kruglov Pavel). - Allow a user configured
hdfs_replication
parameter forDiskHDFS
andStorageHDFS
. Closes #32039. #32049 (leosunli). - Added ClickHouse
exception
andexception_code
fields to opentelemetry span log. #32040 (Frank Chen). - Improve opentelemetry span log duration - it was is zero at the query level if there is a query exception. #32038 (Frank Chen).
- Fix the issue that
LowCardinality
ofInt256
cannot be created. #31832 (alexey-milovidov). - Recreate
system.*_log
tables in case of different engine/partition_by. #31824 (Azat Khuzhin). MaterializedMySQL
: Fix issue with table named 'table'. #31781 (Håvard Kvålen).- ClickHouse dictionary source: support predefined connections. Closes #31705. #31749 (Kseniia Sumarokova).
- Allow to use predefined connections configuration for Kafka and RabbitMQ engines (the same way as for other integration table engines). #31691 (Kseniia Sumarokova).
- Always re-render prompt while navigating history in clickhouse-client. This will improve usability of manipulating very long queries that don't fit on screen. #31675 (alexey-milovidov) (author: Amos Bird).
- Add key bindings for navigating through history (instead of lines/history). #31641 (Azat Khuzhin).
- Improve the
max_execution_time
checks. Fixed some cases when timeout checks do not happen and query could run too long. #31636 (Raúl Marín). - Better exception message when
users.xml
cannot be loaded due to bad password hash. This closes #24126. #31557 (Vitaly Baranov). - Use shard and replica name from
Replicated
database arguments when expanding macros inReplicatedMergeTree
arguments if these macros are not defined in config. Closes #31471. #31488 (tavplubix). - Better analysis for
min/max/count
projection. Now, with enabledallow_experimental_projection_optimization
, virtualmin/max/count
projection can be used together with columns from partition key. #31474 (Amos Bird). - Add
--pager
support forclickhouse-local
. #31457 (Azat Khuzhin). - Fix waiting of the editor during interactive query edition (
waitpid()
returns -1 onSIGWINCH
andEDITOR
andclickhouse-local
/clickhouse-client
works concurrently). #31456 (Azat Khuzhin). - Throw an exception if there is some garbage after field in
JSONCompactStrings(EachRow)
format. #31455 (Kruglov Pavel). - Default value of
http_send_timeout
andhttp_receive_timeout
settings changed from 1800 (30 minutes) to 180 (3 minutes). #31450 (tavplubix). MaterializedMySQL
now handlesCREATE TABLE ... LIKE ...
DDL queries. #31410 (Stig Bakken).- Return artificial create query when executing
show create table
on system's tables. #31391 (SuperDJY). - Previously progress was shown only for
numbers
table function. Now fornumbers_mt
it is also shown. #31318 (Kseniia Sumarokova). - Initial user's roles are used now to find row policies, see #31080. #31262 (Vitaly Baranov).
- If some obsolete setting is changed - show warning in
system.warnings
. #31252 (tavplubix). - Improved backoff for background cleanup tasks in
MergeTree
. Settingsmerge_tree_clear_old_temporary_directories_interval_seconds
andmerge_tree_clear_old_parts_interval_seconds
moved from users settings to merge tree settings. #31180 (tavplubix). - Now every replica will send to client only incremental information about profile events counters. #31155 (Dmitry Novik). This makes
--hardware_utilization
option inclickhouse-client
usable. - Enable multiline editing in clickhouse-client by default. This addresses #31121 . #31123 (Amos Bird).
- Function name normalization for
ALTER
queries. This helps avoid metadata mismatch between creating table with indices/projections and adding indices/projections via alter commands. This is a follow-up PR of ClickHouse#20174. Mark as improvements as there are no bug reports and the senario is somehow rare. #31095 (Amos Bird). - Support
IF EXISTS
modifier forRENAME DATABASE
/TABLE
/DICTIONARY
query. If this directive is used, one will not get an error if the DATABASE/TABLE/DICTIONARY to be renamed doesn't exist. #31081 (victorgao). - Cancel vertical merges when partition is dropped. This is a follow-up of ClickHouse#25684 and ClickHouse#30996. #31057 (Amos Bird).
- The local session inside a Clickhouse dictionary source won't send its events to the session log anymore. This fixes a possible deadlock (tsan alert) on shutdown. Also this PR fixes flaky
test_dictionaries_dependency_xml/
. #31013 (Vitaly Baranov). - Less locking in ALTER command. #31010 (Amos Bird).
- Fix
--verbose
option in clickhouse-local interactive mode and allow logging into file. #30881 (Kseniia Sumarokova). - Added
\l
,\d
,\c
commands inclickhouse-client
like in MySQL and PostgreSQL. #30876 (Pavel Medvedev). - For clickhouse-local or clickhouse-client: if there is
--interactive
option with--query
or--queries-file
, then first execute them like in non-interactive and then start interactive mode. #30851 (Kseniia Sumarokova). - Fix possible "The local set of parts of X doesn't look like the set of parts in ZooKeeper" error (if DROP fails during removing znodes from zookeeper). #30826 (Azat Khuzhin).
- Avro format works against Kafka. Setting
output_format_avro_rows_in_file
added. #30351 (Ilya Golshtein). - Allow to specify one or any number of PostgreSQL schemas for one
MaterializedPostgreSQL
database. Closes #28901. Closes #29324. #28933 (Kseniia Sumarokova). - Replaced default ports for clickhouse-keeper internal communication from 44444 to 9234. Fixes #30879. #31799 (alesapin).
- Implement function transform with Decimal arguments. #31839 (李帅).
- Fix abort in debug server and
DB::Exception: std::out_of_range: basic_string
error in release server in case of bad hdfs url by adding additional check of hdfs url structure. #31042 (Kruglov Pavel). - Fix possible assert in
hdfs
table function/engine, add test. #31036 (Kruglov Pavel).
- Fix group by / order by / limit by aliases with positional arguments enabled. Closes #31173. #31741 (Kseniia Sumarokova).
- Fix usage of
Buffer
table engine with typeMap
. Fixes #30546. #31742 (Anton Popov). - Fix reading from
MergeTree
tables with enableduse_uncompressed_cache
. #31826 (Anton Popov). - Fixed the behavior when mutations that have nothing to do are stuck (with enabled setting
empty_result_for_aggregation_by_empty_set
). #32358 (Nikita Mikhaylov). - Fix skipping columns while writing protobuf. This PR fixes #31160, see the comment #31160#issuecomment-980595318. #31988 (Vitaly Baranov).
- Fix bug when remove unneeded columns in subquery. If there is an aggregation function in query without group by, do not remove if it is unneeded. #32289 (dongyifeng).
- Quota limit was not reached, but the limit was exceeded. This PR fixes #31174. #31337 (sunny).
- Fix SHOW GRANTS when partial revokes are used. This PR fixes #31138. #31249 (Vitaly Baranov).
- Memory amount was incorrectly estimated when ClickHouse is run in containers with cgroup limits. #31157 (Pavel Medvedev).
- Fix
ALTER ... MATERIALIZE COLUMN ...
queries in case when data type of default expression is not equal to the data type of column. #32348 (Anton Popov). - Fixed crash with SIGFPE in aggregate function
avgWeighted
withDecimal
argument. Fixes #32053. #32303 (tavplubix). - Server might fail to start with
Cannot attach 1 tables due to cyclic dependencies
error ifDictionary
table looks at XML-dictionary with the same name, it's fixed. Fixes #31315. #32288 (tavplubix). - Fix parsing error while NaN deserializing for
Nullable(Float)
forQuoted
escaping rule. #32190 (Kruglov Pavel). - XML dictionaries: identifiers, used in table create query, can be qualified to
default_database
during upgrade to newer version. Closes #31963. #32187 (Maksim Kita). - Number of active replicas might be determined incorrectly when inserting with quorum if setting
replicated_can_become_leader
is disabled on some replicas. It's fixed. #32157 (tavplubix). - Dictionaries: fix cases when
{condition}
does not work for custom database queries. #32117 (Maksim Kita). - Fix
CAST
fromNullable
withcast_keep_nullable
(PARAMETER_OUT_OF_BOUND
error before for i.e.toUInt32OrDefault(toNullable(toUInt32(1)))
). #32080 (Azat Khuzhin). - Fix CREATE TABLE of Join Storage in some obscure cases. Close #31680. #32066 (SuperDJY).
- Fixed
Directory ... already exists and is not empty
error when detaching part. #32063 (tavplubix). MaterializedMySQL
(experimental feature): Fix misinterpretation ofDECIMAL
data from MySQL. #31990 (Håvard Kvålen).FileLog
(experimental feature) engine unnesessary created meta data directory when create table failed. Fix #31962. #31967 (flynn).- Some
GET_PART
entry might hang in replication queue if part is lost on all replicas and there are no other parts in the same partition. It's fixed in cases when partition key contains only columns of integer types orDate[Time]
. Fixes #31485. #31887 (tavplubix). - Fix functions
empty
andnotEmpty
with arguments ofUUID
type. Fixes #31819. #31883 (Anton Popov). - Change configuration path from
keeper_server.session_timeout_ms
tokeeper_server.coordination_settings.session_timeout_ms
when constructing aKeeperTCPHandler
. Same withoperation_timeout
. #31859 (JackyWoo). - Fix invalid cast of Nullable type when nullable primary key is used. (Nullable primary key is a discouraged feature - please do not use). This fixes #31075. #31823 (Amos Bird).
- Fix crash in recursive UDF in SQL. Closes #30856. #31820 (Maksim Kita).
- Fix crash when function
dictGet
with type is used for dictionary attribute when type isNullable
. Fixes #30980. #31800 (Maksim Kita). - Fix crash with empty result of ODBC query (with some ODBC drivers). Closes #31465. #31766 (Kseniia Sumarokova).
- Fix disabling query profiler (In case of
query_profiler_real_time_period_ns>0
/query_profiler_cpu_time_period_ns>0
query profiler can stayed enabled even after query finished). #31740 (Azat Khuzhin). - Fixed rare segfault on concurrent
ATTACH PARTITION
queries. #31738 (tavplubix). - Fix race in JSONEachRowWithProgress output format when data and lines with progress are mixed in output. #31736 (Kruglov Pavel).
- Fixed
there are no such cluster here
error on execution ofON CLUSTER
query if specified cluster name is name ofReplicated
database. #31723 (tavplubix). - Fix exception on some of the applications of
decrypt
function on Nullable columns. This closes #31662. This closes #31426. #31707 (alexey-milovidov). - Fixed function ngrams when string contains UTF-8 characters. #31706 (yandd).
- Settings
input_format_allow_errors_num
andinput_format_allow_errors_ratio
did not work for parsing of domain types, such asIPv4
, it's fixed. Fixes #31686. #31697 (tavplubix). - Fixed null pointer exception in
MATERIALIZE COLUMN
. #31679 (Nikolai Kochetov). RENAME TABLE
query worked incorrectly on attempt to rename an DDL dictionary inOrdinary
database, it's fixed. #31638 (tavplubix).- Implement
sparkbar
aggregate function as it was intended, see: #26175#issuecomment-960353867, comment. #31624 (小路). - Fix invalid generated JSON when only column names contain invalid UTF-8 sequences. #31534 (Kevin Michel).
- Disable
partial_merge_join_left_table_buffer_bytes
before bug in this optimization is fixed. See #31009). Remove redundant optionpartial_merge_join_optimizations
. #31528 (Vladimir C). - Fix progress for short
INSERT SELECT
queries. #31510 (Azat Khuzhin). - Fix wrong behavior with group by and positional arguments. Closes #31280#issuecomment-968696186. #31420 (Kseniia Sumarokova).
- Resolve
nullptr
in STS credentials provider for S3. #31409 (Vladimir Chebotarev). - Remove
notLike
function from index analysis, because it was wrong. #31169 (sundyli). - Fix bug in Keeper which can lead to inability to start when some coordination logs was lost and we have more fresh snapshot than our latest log. #31150 (alesapin).
- Rewrite right distributed table in local join. solves #25809. #31105 (abel-cheng).
- Fix
Merge
table with aliases and where (it did not work before at all). Closes #28802. #31044 (Kseniia Sumarokova). - Fix JSON_VALUE/JSON_QUERY with quoted identifiers. This allows to have spaces in json path. Closes #30971. #31003 (Kseniia Sumarokova).
- Using
formatRow
function with not row-oriented formats led to segfault. Don't allow to use this function with such formats (because it doesn't make sense). #31001 (Kruglov Pavel). - Fix bug which broke select queries if they happened after dropping materialized view. Found in #30691. #30997 (Kseniia Sumarokova).
- Skip
max_partition_size_to_drop check
in case of ATTACH PARTITION ... FROM and MOVE PARTITION ... #30995 (Amr Alaa). - Fix some corner cases with
INTERSECT
andEXCEPT
operators. Closes #30803. #30965 (Kseniia Sumarokova).
- Fix incorrect filtering result on non-x86 builds. This closes #31417. This closes #31524. #31574 (alexey-milovidov).
- Make ClickHouse build fully reproducible (byte identical on different machines). This closes #22113. #31899 (alexey-milovidov). Remove filesystem path to the build directory from binaries to enable reproducible builds. This needed for #22113. #31838 (alexey-milovidov).
- Use our own CMakeLists for
zlib-ng
,cassandra
,mariadb-connector-c
andxz
,re2
,sentry
,gsasl
,arrow
,protobuf
. This is needed for #20151. Part of #9226. A small step towards removal of annoying trash from the build system. #30599 (alexey-milovidov). - Hermetic builds: use fixed version of libc and make sure that no source or binary files from the host OS are using during build. This closes #27133. This closes #21435. This closes #30462. #30011 (alexey-milovidov).
- Adding function
getFuzzerData()
to easily fuzz particular functions. This closes #23227. #27526 (Alexey Boykov). - More correct setting up capabilities inside Docker. #31802 (Constantine Peresypkin).
- Enable clang
-fstrict-vtable-pointers
,-fwhole-program-vtables
compile options. #20151 (Maksim Kita). - Avoid downloading toolchain tarballs for cross-compiling for FreeBSD. #31672 (alexey-milovidov).
- Initial support for risc-v. See development/build-cross-riscv for quirks and build command that was tested. #31309 (Vladimir Smirnov).
- Support compile in arm machine with parameter "-DENABLE_TESTS=OFF". #31007 (zhanghuajie).
- Change order of json_path and json arguments in SQL/JSON functions (to be consistent with the standard). Closes #30449. #30474 (Kseniia Sumarokova).
- Remove
MergeTree
table settingwrite_final_mark
. It will be alwaystrue
. #30455 (Kseniia Sumarokova). No actions required, all tables are compatible with the new version. - Function
bayesAB
is removed. Please help to return this function back, refreshed. This closes #26233. #29934 (alexey-milovidov). - This is relevant only if you already started using the experimental
clickhouse-keeper
support. Now ClickHouse Keeper snapshots compressed withZSTD
codec by default instead of custom ClickHouse LZ4 block compression. This behavior can be turned off withcompress_snapshots_with_zstd_format
coordination setting (must be equal on all quorum replicas). Backward incompatibility is quite rare and may happen only when new node will send snapshot (happens in case of recovery) to the old node which is unable to read snapshots in ZSTD format. #29417 (alesapin).
- New asynchronous INSERT mode allows to accumulate inserted data and store it in a single batch in background. On client it can be enabled by setting
async_insert
forINSERT
queries with data inlined in query or in separate buffer (e.g. forINSERT
queries via HTTP protocol). Ifwait_for_async_insert
is true (by default) the client will wait until data will be flushed to table. On server-side it controlled by the settingsasync_insert_threads
,async_insert_max_data_size
andasync_insert_busy_timeout_ms
. Implements #18282. #27537 (Anton Popov). #20557 (Ivan). Notes on performance: with asynchronous inserts you can do up to around 10 000 individual INSERT queries per second, so it is still recommended to insert in batches if you want to achieve performance up to millions inserted rows per second. - Add interactive mode for
clickhouse-local
. So, you can just runclickhouse-local
to get a command line ClickHouse interface without connecting to a server and process data from files and external data sources. Also merge the code ofclickhouse-client
andclickhouse-local
together. Closes #7203. Closes #25516. Closes #22401. #26231 (Kseniia Sumarokova). - Added support for executable (scriptable) user defined functions. These are UDFs that can be written in any programming language. #28803 (Maksim Kita).
- Allow predefined connections to external data sources. This allows to avoid specifying credentials or addresses while using external data sources, they can be referenced by names instead. Closes #28367. #28577 (Kseniia Sumarokova).
- Added
INFORMATION_SCHEMA
database withSCHEMATA
,TABLES
,VIEWS
andCOLUMNS
views to the corresponding tables insystem
database. Closes #9770. #28691 (tavplubix). - Support
EXISTS (subquery)
. Closes #6852. #29731 (Kseniia Sumarokova). - Session logging for audit. Logging all successful and failed login and logout events to a new
system.session_log
table. #22415 (Vasily Nemkov) (Vitaly Baranov). - Support multidimensional cosine distance and euclidean distance functions; L1, L2, Lp, Linf distances and norms. Scalar product on tuples and various arithmetic operators on tuples. This fully closes #4509 and even more. #27933 (Alexey Boykov).
- Add support for compression and decompression for
INTO OUTFILE
andFROM INFILE
(with autodetect or with additional optional parameter). #27135 (Filatenkov Artur). - Add CORS (Cross Origin Resource Sharing) support with HTTP
OPTIONS
request. It means, now Grafana will work with serverless requests without a kludges. Closes #18693. #29155 (Filatenkov Artur). - Queries with JOIN ON now supports disjunctions (OR). #21320 (Ilya Golshtein).
- Added function
tokens
. That allow to split string into tokens using non-alpha numeric ASCII characters as separators. #29981 (Maksim Kita). Added functionngrams
to extract ngrams from text. Closes #29699. #29738 (Maksim Kita). - Add functions for Unicode normalization:
normalizeUTF8NFC
,normalizeUTF8NFD
,normalizeUTF8NFKC
,normalizeUTF8NFKD
functions. #28633 (darkkeks). - Streaming consumption of application log files in ClickHouse with
FileLog
table engine. It's likeKafka
orRabbitMQ
engine but for append-only and rotated logs in local filesystem. Closes #6953. #25969 (flynn) (Kseniia Sumarokova). - Add
CapnProto
output format, refactorCapnProto
input format. #29291 (Kruglov Pavel). - Allow to write number in query as binary literal. Example
SELECT 0b001;
. #29304 (Maksim Kita). - Added
hashed_array
dictionary type. It saves memory when using dictionaries with multiple attributes. Closes #30236. #30242 (Maksim Kita). - Added
JSONExtractKeys
function. #30056 (Vitaly). - Add a function
getOSKernelVersion
- it returns a string with OS kernel version. #29755 (Memo). - Added
MD4
andSHA384
functions. MD4 is an obsolete and insecure hash function, it can be used only in rare cases when MD4 is already being used in some legacy system and you need to get exactly the same result. #29602 (Nikita Tikhomirov). - HSTS can be enabled for Clickhouse HTTP server by setting
hsts_max_age
in configuration file with a positive number. #29516 (凌涛). - Huawei OBS Storage support. Closes #24294. #29511 (kevin wan).
- New function
mapContainsKeyLike
to get the map that key matches a simple regular expression. #29471 (凌涛). New functionmapExtractKeyLike
to get the map only kept elements matched specified pattern. #30793 (凌涛). - Implemented
ALTER TABLE x MODIFY COMMENT
. #29264 (Vasily Nemkov). - Adds H3 inspection functions that are missing from ClickHouse but are available via the H3 api: https://h3geo.org/docs/api/inspection. #29209 (Bharat Nallan).
- Allow non-replicated ALTER TABLE FETCH and ATTACH in Replicated databases. #29202 (Kevin Michel).
- Added a setting
output_format_csv_null_representation
: This is the same asoutput_format_tsv_null_representation
but is for CSV output. #29123 (PHO). - Added function
zookeeperSessionUptime()
which returns uptime of current ZooKeeper session in seconds. #28983 (tavplubix). - Implements the
h3ToGeoBoundary
function. #28952 (Ivan Veselov). - Add aggregate function
exponentialMovingAverage
that can be used as window function. This closes #27511. #28914 (alexey-milovidov). - Allow to include subcolumns of table columns into
DESCRIBE
query result (can be enabled by settingdescribe_include_subcolumns
). #28905 (Anton Popov). Executable
,ExecutablePool
added optionsend_chunk_header
. If this option is true then chunk rows_count with line break will be sent to client before chunk. #28833 (Maksim Kita).tokenbf_v1
andngram
support Map with key of String of FixedSring type. It enhance data skipping in query with map key filter.sql CREATE TABLE map_tokenbf ( row_id UInt32, map Map(String, String), INDEX map_tokenbf map TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 ) Engine=MergeTree() Order by id
With table above, the queryselect * from map_tokebf where map['K']='V'
will skip the granule that doesn't contain keyA
. Of course, how many rows will skipped is depended on thegranularity
andindex_granularity
you set. #28511 (凌涛).- Send profile events from server to client. New packet type
ProfileEvents
was introduced. Closes #26177. #28364 (Dmitry Novik). - Bit shift operations for
FixedString
andString
data types. This closes #27763. #28325 (小路). - Support adding / deleting tables to replication from PostgreSQL dynamically in database engine MaterializedPostgreSQL. Support alter for database settings. Closes #27573. #28301 (Kseniia Sumarokova).
- Added function accurateCastOrDefault(x, T). Closes #21330. Authors @taiyang-li. #23028 (Maksim Kita).
- Add Function
toUUIDOrDefault
,toUInt8/16/32/64/256OrDefault
,toInt8/16/32/64/128/256OrDefault
, which enables user defining default value(not null) when string parsing is failed. #21330 (taiyang-li).
- Background merges can be preempted by each other and they are scheduled with appropriate priorities. Now long running merges won't prevent short merges to proceed. This is needed for a better scheduling and controlling of merges execution. It reduces the chances to get "too many parts" error. #22381. #25165 (Nikita Mikhaylov). Added an ability to execute more merges and mutations than the number of threads in background pool. Merges and mutations will be executed step by step according to their sizes (lower is more prioritized). The ratio of the number of tasks to threads to execute is controlled by a setting
background_merges_mutations_concurrency_ratio
, 2 by default. #29140 (Nikita Mikhaylov). - Allow to use asynchronous reads for remote filesystems. Lower the number of seeks while reading from remote filesystems. It improves performance tremendously and makes the experimental
web
ands3
disks to work faster than EBS under certain conditions. #29205 (Kseniia Sumarokova). In the meantime, theweb
disk type (static dataset hosted on a web server) is graduated from being experimental to be production ready. - Queries with
INTO OUTFILE
inclickhouse-client
will use multiple threads. Fix the issue with flickering progress-bar when usingINTO OUTFILE
. This closes #30873. This closes #30872. #30886 (alexey-milovidov). - Reduce amount of redundant compressed data read from disk for some types
SELECT
queries (only forMergeTree
engines family). #30111 (alesapin). - Remove some redundant
seek
calls while reading compressed blocks in MergeTree table engines family. #29766 (alesapin). - Make
url
table function to process multiple URLs in parallel. This closes #29670 and closes #29671. #29673 (alexey-milovidov). - Improve performance of aggregation in order of primary key (with enabled setting
optimize_aggregation_in_order
). #30266 (Anton Popov). - Now clickhouse is using DNS cache while communicating with external S3. #29999 (alesapin).
- Add support for pushdown of
IS NULL
/IS NOT NULL
to external databases (i.e. MySQL). #29463 (Azat Khuzhin). TransformisNull
/isNotNull
toIS NULL
/IS NOT NULL
(for external dbs, i.e. MySQL). #29446 (Azat Khuzhin). - SELECT queries from Dictionary tables will use multiple threads. #30500 (Maksim Kita).
- Improve performance for filtering (WHERE operation) of
Decimal
columns. #30431 (Jun Jin). - Remove branchy code in filter operation with a better implementation with popcnt/ctz which have better performance. #29881 (Jun Jin).
- Improve filter bytemask generator (used for WHERE operator) function all in one with SSE/AVX2/AVX512 instructions. Note that by default ClickHouse is only using SSE, so it's only relevant for custom builds. #30014 (jasperzhu). #30670 (jasperzhu).
- Improve the performance of SUM aggregate function of Nullable floating point numbers. #28906 (Raúl Marín).
- Speed up part loading process with multiple disks are in use. The idea is similar to ClickHouse#16423 . Prod env shows improvement: 24 min -> 16 min . #28363 (Amos Bird).
- Reduce default settings for S3 multipart upload part size to lower memory usage. #28679 (ianton-ru).
- Speed up
bitmapAnd
function. #28332 (dddounaiking). - Removed sub-optimal mutation notifications in
StorageMergeTree
when merges are still going. #27552 (Vladimir Chebotarev). - Attempt to improve performance of string comparison. #28767 (alexey-milovidov).
- Primary key index and partition filter can work in tuple. #29281 (凌涛).
- If query has multiple quantile aggregate functions with the same arguments but different level parameter, they will be fused together and executed in one pass if the setting
optimize_syntax_fuse_functions
is enabled. #26657 (hexiaoting). - Now min-max aggregation over the first expression of primary key is optimized by projection. This is for #329. #29918 (Amos Bird).
- Add ability to change nodes configuration (in
.xml
file) for ClickHouse Keeper. #30372 (alesapin). - Add
sparkbar
aggregate function. This closes #26175. #27481 (小路). Note: there is one flaw in this function, the behaviour will be changed in future releases.
- Allow user to change log levels without restart. #29586 (Nikolay Degterinsky).
- Multiple improvements for SQL UDF. Queries for manipulation of SQL User Defined Functions now support ON CLUSTER clause. Example
CREATE FUNCTION test_function ON CLUSTER 'cluster' AS x -> x + 1;
. Closes #30666. #30734 (Maksim Kita). SupportCREATE OR REPLACE
,CREATE IF NOT EXISTS
syntaxes. #30454 (Maksim Kita). Added DROP IF EXISTS support. ExampleDROP FUNCTION IF EXISTS test_function
. #30437 (Maksim Kita). Support lambdas. ExampleCREATE FUNCTION lambda_function AS x -> arrayMap(element -> element * 2, x);
. #30435 (Maksim Kita). Support SQL user defined functions forclickhouse-local
. #30179 (Maksim Kita). - Enable per-query memory profiler (set to
memory_profiler_step
= 4MiB) globally. #29455 (Azat Khuzhin). - Added columns
data_compressed_bytes
,data_uncompressed_bytes
,marks_bytes
intosystem.data_skipping_indices
. Added columnssecondary_indices_compressed_bytes
,secondary_indices_uncompressed_bytes
,secondary_indices_marks_bytes
intosystem.parts
. Closes #29697. #29896 (Maksim Kita). - Add
table
alias to system.tables anddatabase
alias to system.databases #29677. #29882 (kevin wan). - Correctly resolve interdependencies between tables on server startup. Closes #8004, closes #15170. #28373 (tavplubix).
- Avoid error "Division by zero" when denominator is Nullable in functions
divide
,intDiv
andmodulo
. Closes #22621. #28352 (Kruglov Pavel). - Allow to parse values of
Date
data type in text formats asYYYYMMDD
in addition toYYYY-MM-DD
. This closes #30870. #30871 (alexey-milovidov). - Web UI: render bars in table cells. #29792 (alexey-milovidov).
- User can now create dictionaries with comments:
CREATE DICTIONARY ... COMMENT 'vaue'
... #29899 (Vasily Nemkov). Users now can set comments to database inCREATE DATABASE
statement ... #29429 (Vasily Nemkov). - Introduce
compiled_expression_cache_elements_size
setting. If you will ever want to use this setting, you will already know what it does. #30667 (Maksim Kita). - clickhouse-format now supports option
--query
. In previous versions you have to pass the query to stdin. #29325 (凌涛). - Support
ALTER TABLE
for tables inMemory
databases. Memory databases are used inclickhouse-local
. #30866 (tavplubix). - Arrays of all serializable types are now supported by
arrayStringConcat
. #30840 (Nickita Taranov). - ClickHouse now will account docker/cgroups limitations to get system memory amount. See #25662. #30574 (Pavel Medvedev).
- Fetched table structure for PostgreSQL database is more reliable now. #30477 (Kseniia Sumarokova).
- Full support of positional arguments in GROUP BY and ORDER BY. #30433 (Kseniia Sumarokova).
- Allow extracting non-string element as string using JSONExtractString. This is for pull/25452#issuecomment-927123287. #30426 (Amos Bird).
- Added an ability to use FINAL clause in SELECT queries from
GraphiteMergeTree
. #30360 (Nikita Mikhaylov). - Minor improvements in replica cloning and enqueuing fetch for broken parts, that should avoid extremely rare hanging of
GET_PART
entries in replication queue. #30346 (tavplubix). - Allow symlinks to files in
user_files
directory for file table function. #30309 (Kseniia Sumarokova). - Fixed comparison of
Date32
withDate
,DateTime
,DateTime64
andString
. #30219 (liang.huang). - Allow to remove
SAMPLE BY
expression fromMergeTree
tables (ALTER TABLE <table> REMOVE SAMPLE BY
). #30180 (Anton Popov). - Now
Keeper
(as part ofclickhouse-server
) will start asynchronously if it can connect to some other node. #30170 (alesapin). - Now
clickhouse-client
supports native multi-line editing. #30143 (Amos Bird). polygon
dictionaries (reverse geocoding): added support for reading the dictionary content with SELECT query method if settingstore_polygon_key_column
= true. Closes #30090. #30142 (Maksim Kita).- Add ClickHouse logo to Play UI. #29674 (alexey-milovidov).
- Better exception message while reading column from Arrow-supported formats like
Arrow
,ArrowStream
,Parquet
andORC
. This closes #29926. #29927 (alexey-milovidov). - Fix data-race between flush and startup in
Buffer
tables. This can appear in tests. #29930 (Azat Khuzhin). - Fix
lock-order-inversion
betweenDROP TABLE
forDatabaseMemory
andLiveView
. Live View is an experimental feature. Memory database is used in clickhouse-local. #29929 (Azat Khuzhin). - Fix lock-order-inversion between periodic dictionary reload and config reload. #29928 (Azat Khuzhin).
- Update zoneinfo files to 2021c. #29925 (alexey-milovidov).
- Add ability to configure retries and delays between them for
clickhouse-copier
. #29921 (Azat Khuzhin). - Add
shutdown_wait_unfinished_queries
server setting to allowing waiting for running queries up toshutdown_wait_unfinished
time. This is for #24451. #29914 (Amos Bird). - Add ability to trace peak memory usage (with new trace_type in
system.trace_log
-MemoryPeak
). #29858 (Azat Khuzhin). - PostgreSQL foreign tables: Added partitioned table prefix 'p' for the query for fetching replica identity index. #29828 (Shoh Jahon).
- Apply
max_untracked_memory
/memory_profiler_step
/memory_profiler_sample_probability
during mutate/merge to profile memory usage during merges. #29681 (Azat Khuzhin). - Query obfuscator:
clickhouse-format --obfuscate
now works with more types of queries. #29672 (alexey-milovidov). - Fixed the issue:
clickhouse-format --obfuscate
cannot process queries with embedded dictionaries (functionsregionTo...
). #29667 (alexey-milovidov). - Fix incorrect Nullable processing of JSON functions. This fixes #29615 . Mark as improvement because ClickHouse#28012 is not released. #29659 (Amos Bird).
- Increase
listen_backlog
by default (to match default in newer linux kernel). #29643 (Azat Khuzhin). - Reload dictionaries, models, user defined executable functions if servers config
dictionaries_config
,models_config
,user_defined_executable_functions_config
changes. Closes #28142. #29529 (Maksim Kita). - Get rid of pointless restriction on projection name. Now projection name can start with
tmp_
. #29520 (Amos Bird). - Fixed
There is no query or query context has expired
error in mutations with nested subqueries. Do not allow subqueries in mutation if table is replicated andallow_nondeterministic_mutations
setting is disabled. #29495 (tavplubix). - Apply config changes to
max_concurrent_queries
during runtime (no need to restart). #29414 (Raúl Marín). - Added setting
use_skip_indexes
. #29405 (Maksim Kita). - Add support for
FREEZE
ing in-memory parts (for backups). #29376 (Mo Xuan). - Pass through initial query_id for
clickhouse-benchmark
(previously if you run remote query viaclickhouse-benchmark
, queries on shards will not be linked to the initial query viainitial_query_id
). #29364 (Azat Khuzhin). - Skip indexes
tokenbf_v1
andngrambf_v1
: added support forArray
data type with key ofString
ofFixedString
type. #29280 (Maksim Kita). Skip indexestokenbf_v1
andngrambf_v1
added support forMap
data type with key ofString
ofFixedString
type. Author @lingtaolf. #29220 (Maksim Kita). - Function
has
: added support forMap
data type. #29267 (Maksim Kita). - Add
compress_logs
settings for clickhouse-keeper which allow to compress clickhouse-keeper logs (for replicated state machine) inZSTD
. Implements: #26977. #29223 (alesapin). - Add a setting
external_table_strict_query
- it will force passing the whole WHERE expression in queries to foreign databases even if it is incompatible. #29206 (Azat Khuzhin). - Disable projections when
ARRAY JOIN
is used. In previous versions projection analysis may break aliases in array join. #29139 (Amos Bird). - Support more types in
MsgPack
input/output format. #29077 (Kruglov Pavel). - Allow to input and output
LowCardinality
columns inORC
input/output format. #29062 (Kruglov Pavel). - Select from
system.distributed_ddl_queue
might show incorrect values, it's fixed. #29061 (tavplubix). - Correct behaviour with unknown methods for HTTP connection. Solves #29050. #29057 (Filatenkov Artur).
clickhouse-keeper
: Fix bug inclickhouse-keeper-converter
which can lead to some data loss while restoring from ZooKeeper logs (not snapshot). #29030 (小路). Fix bug inclickhouse-keeper-converter
which can lead to incorrect ZooKeeper log deserialization. #29071 (小路).- Apply settings from
CREATE ... AS SELECT
queries (fixes: #28810). #28962 (Azat Khuzhin). - Respect default database setting for ALTER TABLE ... ON CLUSTER ... REPLACE/MOVE PARTITION FROM/TO ... #28955 (anneji-dev).
- gRPC protocol: Allow change server-side compression from client. #28953 (Vitaly Baranov).
- Skip "no data" exception when reading thermal sensors for asynchronous metrics. This closes #28852. #28882 (alexey-milovidov).
- Fixed logical race condition that might cause
Dictionary not found
error for existing dictionary in rare cases. #28853 (tavplubix). - Relax nested function for If-combinator check (but forbid nested identical combinators). #28828 (Azat Khuzhin).
- Fix possible uncaught exception during server termination. #28761 (Azat Khuzhin).
- Forbid cleaning of tmp directories that can be used by an active mutation/merge if mutation/merge is extraordinarily long. #28760 (Azat Khuzhin).
- Allow optimization
optimize_arithmetic_operations_in_aggregate_functions = 1
when alias is used. #28746 (Amos Bird). - Implement
detach_not_byte_identical_parts
setting forReplicatedMergeTree
, that will detach instead of remove not byte-identical parts (after mege/mutate). #28708 (Azat Khuzhin). - Implement
max_suspicious_broken_parts_bytes
setting forMergeTree
(to limit total size of all broken parts, default is1GiB
). #28707 (Azat Khuzhin). - Enable expanding macros in
RabbitMQ
table settings. #28683 (Vitaly Baranov). - Restore the possibility to read data of a table using the
Log
engine in multiple threads. #28125 (Vitaly Baranov). - Fix misbehavior of NULL column handling in JSON functions. This fixes #27930. #28012 (Amos Bird).
- Allow to set the size of Mark/Uncompressed cache for skip indices separately from columns. #27961 (Amos Bird).
- Allow to mix JOIN with
USING
with other JOIN types. #23881 (darkkeks). - Update aws-sdk submodule for throttling in Yandex Cloud S3. #30646 (ianton-ru).
- Fix releasing query ID and session ID at the end of query processing while handing gRPC call. #29954 (Vitaly Baranov).
- Fix shutdown of
AccessControlManager
to fix flaky test. #29951 (Vitaly Baranov). - Fix failed assertion in reading from
HDFS
. Update libhdfs3 library to be able to run in tests in debug. Closes #29251. Closes #27814. #29276 (Kseniia Sumarokova).
- Add support for FreeBSD builds for Aarch64 machines. #29952 (MikaelUrankar).
- Recursive submodules are no longer needed for ClickHouse. #30315 (alexey-milovidov).
- ClickHouse can be statically built with Musl. This is added as experiment, it does not support building
odbc-bridge
,library-bridge
, integration with CatBoost and some libraries. #30248 (alexey-milovidov). - Enable
Protobuf
,Arrow
,ORC
,Parquet
forAArch64
andDarwin
(macOS) builds. This closes #29248. This closes #28018. #30015 (alexey-milovidov). - Add cross-build for PowerPC (powerpc64le). This closes #9589. Enable support for interaction with MySQL for AArch64 and PowerPC. This closes #26301. #30010 (alexey-milovidov).
- Leave only required files in cross-compile toolchains. Include them as submodules (earlier they were downloaded as tarballs). #29974 (alexey-milovidov).
- Implemented structure-aware fuzzing approach in ClickHouse for select statement parser. #30012 (Paul).
- Turning on experimental constexpr expressions evaluator for clang to speed up template code compilation. #29668 (myrrc).
- Add ability to compile using newer version fo glibc without using new symbols. #29594 (Azat Khuzhin).
- Reduce Debug build binary size by clang optimization option. #28736 (flynn).
- Now all images for CI will be placed in the separate dockerhub repo. #28656 (alesapin).
- Improve support for build with clang-13. #28046 (Sergei Semin).
- Add ability to print raw profile events to
clickhouse-client
(This can be useful for debugging and for testing). #30064 (Azat Khuzhin). - Add time dependency for clickhouse-server unit (systemd and sysvinit init). #28891 (Azat Khuzhin).
- Reload stacktrace cache when symbol is reloaded. #28137 (Amos Bird).
- Functions for case-insensitive search in UTF-8 strings like
positionCaseInsensitiveUTF8
andcountSubstringsCaseInsensitiveUTF8
might find substrings that actually does not match in very rare cases, it's fixed. #30663 (tavplubix). - Fix reading from empty file on encrypted disk. #30494 (Vitaly Baranov).
- Fix transformation of disjunctions chain to
IN
(controlled by settingsoptimize_min_equality_disjunction_chain_length
) in distributed queries with settingslegacy_column_name_of_tuple_literal = 0
. #28658 (Anton Popov). - Allow using a materialized column as the sharding key in a distributed table even if
insert_allow_materialized_columns=0
:. #28637 (Vitaly Baranov). - Fix
ORDER BY ... WITH FILL
with setTO
andFROM
and no rows in result set. #30888 (Anton Popov). - Fix set index not used in AND/OR expressions when there are more than two operands. This fixes #30416 . #30887 (Amos Bird).
- Fix crash when projection with hashing function is materialized. This fixes #30861 . The issue is similar to ClickHouse#28560 which is a lack of proper understanding of the invariant of header's emptyness. #30877 (Amos Bird).
- Fixed ambiguity when extracting auxiliary ZooKeeper name from ZooKeeper path in
ReplicatedMergeTree
. Previously server might fail to start withUnknown auxiliary ZooKeeper name
if ZooKeeper path contains a colon. Fixes #29052. Also it was allowed to specify ZooKeeper path that does not start with slash, but now it's deprecated and creation of new tables with such path is not allowed. Slashes and colons in auxiliary ZooKeeper names are not allowed too. #30822 (tavplubix). - Clean temporary directory when localBackup failed by some reason. #30797 (ianton-ru).
- Fixed a race condition between
REPLACE/MOVE PARTITION
and background merge in non-replicatedMergeTree
that might cause a part of moved/replaced data to remain in partition. Fixes #29327. #30717 (tavplubix). - Fix PREWHERE with WHERE in case of always true PREWHERE. #30668 (Azat Khuzhin).
- Limit push down optimization could cause a error
Cannot find column
. Fixes #30438. #30562 (Nikolai Kochetov). - Add missing parenthesis for
isNotNull
/isNull
rewrites toIS [NOT] NULL
(fixes queries that has something likeisNotNull(1)+isNotNull(2)
). #30520 (Azat Khuzhin). - Fix deadlock on ALTER with scalar subquery to the same table, close #30461. #30492 (Vladimir C).
- Fixed segfault which might happen if session expired during execution of REPLACE PARTITION. #30432 (tavplubix).
- Queries with condition like
IN (subquery)
could return incorrect result in case if aggregate projection applied. Fixed creation of sets for projections. #30310 (Amos Bird). - Fix column alias resolution of JOIN queries when projection is enabled. This fixes #30146. #30293 (Amos Bird).
- Fix some deficiency in
replaceRegexpAll
function. #30292 (Memo). - Fix ComplexKeyHashedDictionary, ComplexKeySparseHashedDictionary parsing
preallocate
option from layout config. #30246 (Maksim Kita). - Fix
[I]LIKE
function. Closes #28661. #30244 (Nikolay Degterinsky). - Fix crash with shortcircuit and lowcardinality in multiIf. #30243 (Raúl Marín).
- FlatDictionary, HashedDictionary fix bytes_allocated calculation for nullable attributes. #30238 (Maksim Kita).
- Allow identifiers starting with numbers in multiple joins. #30230 (Vladimir C).
- Fix reading from
MergeTree
withmax_read_buffer_size = 0
(when the user wants to shoot himself in the foot) (can lead to exceptionsCan't adjust last granule
,LOGICAL_ERROR
, or even data loss). #30192 (Azat Khuzhin). - Fix
pread_fake_async
/pread_threadpool
withmin_bytes_to_use_direct_io
. #30191 (Azat Khuzhin). - Fix INSERT SELECT incorrectly fills MATERIALIZED column based of Nullable column. #30189 (Azat Khuzhin).
- Support nullable arguments in function
initializeAggregation
. #30177 (Anton Popov). - Fix error
Port is already connected
for queries withGLOBAL IN
andWITH TOTALS
. Only for 21.9 and 21.10. #30086 (Nikolai Kochetov). - Fix race between MOVE PARTITION and merges/mutations for MergeTree. #30074 (Azat Khuzhin).
- Dropped
Memory
database might reappear after server restart, it's fixed (#29795). Also addedforce_remove_data_recursively_on_drop
setting as a workaround forDirectory not empty
error when droppingOrdinary
database (because it's not possible to remove data leftovers manually in cloud environment). #30054 (tavplubix). - Fix crash of sample by
tuple()
, closes #30004. #30016 (flynn). - try to close issue: #29965. #29976 (hexiaoting).
- Fix possible data-race between
FileChecker
andStorageLog
/StorageStripeLog
. #29959 (Azat Khuzhin). - Fix data-race between
LogSink::writeMarks()
andLogSource
inStorageLog
. #29946 (Azat Khuzhin). - Fix potential resource leak of the concurrent query limit of merge tree tables introduced in ClickHouse#19544. #29879 (Amos Bird).
- Fix system tables recreation check (fails to detect changes in enum values). #29857 (Azat Khuzhin).
- MaterializedMySQL: Fix an issue where if the connection to MySQL was lost, only parts of a transaction could be processed. #29837 (Håvard Kvålen).
- Avoid
Timeout exceeded: elapsed 18446744073.709553 seconds
error that might happen in extremely rare cases, presumably due to some bug in kernel. Fixes #29154. #29811 (tavplubix). - Fix bad cast in
ATTACH TABLE ... FROM 'path'
query when non-string literal is used instead of path. It may lead to reading of uninitialized memory. #29790 (alexey-milovidov). - Fix concurrent access to
LowCardinality
duringGROUP BY
(in combination withBuffer
tables it may lead to troubles). #29782 (Azat Khuzhin). - Fix incorrect
GROUP BY
(multiple rows with the same keys in result) in case of distributed query when shards had mixed versions<= 21.3
and>= 21.4
,GROUP BY
key had several columns all with fixed size, and two-level aggregation was activated (seegroup_by_two_level_threshold
andgroup_by_two_level_threshold_bytes
). Fixes #29580. #29735 (Nikolai Kochetov). - Fixed incorrect behaviour of setting
materialized_postgresql_tables_list
at server restart. Found in #28529. #29686 (Kseniia Sumarokova). - Condition in filter predicate could be lost after push-down optimisation. #29625 (Nikolai Kochetov).
- Fix JIT expression compilation with aliases and short-circuit expression evaluation. Closes #29403. #29574 (Maksim Kita).
- Fix rare segfault in
ALTER MODIFY
query when using incorrect table identifier inDEFAULT
expression likex.y.z...
Fixes #29184. #29573 (alesapin). - Fix nullptr deference for
GROUP BY WITH TOTALS HAVING
(when the column fromHAVING
wasn't selected). #29553 (Azat Khuzhin). - Avoid deadlocks when reading and writting on Join table engine tables at the same time. #29544 (Raúl Marín).
- Fix bug in check
pathStartsWith
becuase there was bug with the usage ofstd::mismatch
:The behavior is undefined if the second range is shorter than the first range.
. #29531 (Kseniia Sumarokova). - In ODBC bridge add retries for error Invalid cursor state. It is a retriable error. Closes #29473. #29518 (Kseniia Sumarokova).
- Fixed incorrect table name parsing on loading of
Lazy
database. Fixes #29456. #29476 (tavplubix). - Fix possible
Block structure mismatch
for subqueries with pushed-downHAVING
predicate. Fixes #29010. #29475 (Nikolai Kochetov). - Fix Logical error
Cannot capture columns
in functions greatest/least. Closes #29334. #29454 (Kruglov Pavel). - RocksDB table engine: fix race condition during multiple DB opening (and get back some tests that triggers the problem on CI). #29393 (Azat Khuzhin).
- Fix replicated access storage not shutting down cleanly when misconfigured. #29388 (Kevin Michel).
- Remove window function
nth_value
as it is not memory-safe. This closes #29347. #29348 (alexey-milovidov). - Fix vertical merges of projection parts. This fixes #29253 . This PR also fixes several projection merge/mutation issues introduced in ClickHouse#25165. #29337 (Amos Bird).
- Fix hanging DDL queries on Replicated database while adding a new replica. #29328 (Kevin Michel).
- Fix connection timeouts (
send_timeout
/receive_timeout
). #29282 (Azat Khuzhin). - Fix possible
Table columns structure in ZooKeeper is different from local table structure
exception while recreating or creating new replicas ofReplicatedMergeTree
, when one of table columns have default expressions with case-insensitive functions. #29266 (Anton Popov). - Send normal
Database doesn't exist error
(UNKNOWN_DATABASE
) to the client (via TCP) instead ofAttempt to read after eof
(ATTEMPT_TO_READ_AFTER_EOF
). #29229 (Azat Khuzhin). - Fix segfault while inserting into column with type LowCardinality(Nullable) in Avro input format. #29132 (Kruglov Pavel).
- Do not allow to reuse previous credentials in case of inter-server secret (Before INSERT via Buffer/Kafka to Distributed table with interserver secret configured for that cluster, may re-use previously set user for that connection). #29060 (Azat Khuzhin).
- Handle
any_join_distinct_right_table_keys
when join with dictionary, close #29007. #29014 (Vladimir C). - Fix "Not found column ... in block" error, when join on alias column, close #26980. #29008 (Vladimir C).
- Fix the number of threads used in
GLOBAL IN
subquery (it was executed in single threads since #19414 bugfix). #28997 (Nikolai Kochetov). - Fix bad optimizations of ORDER BY if it contains WITH FILL. This closes #28908. This closes #26049. #28910 (alexey-milovidov).
- Fix higher-order array functions (
SIGSEGV
forarrayCompact
/ILLEGAL_COLUMN
forarrayDifference
/arrayCumSumNonNegative
) with consts. #28904 (Azat Khuzhin). - Fix waiting for mutation with
mutations_sync=2
. #28889 (Azat Khuzhin). - Fix queries to external databases (i.e. MySQL) with multiple columns in IN ( i.e.
(k,v) IN ((1, 2))
). #28888 (Azat Khuzhin). - Fix bug with
LowCardinality
in short-curcuit function evaluation. Closes #28884. #28887 (Kruglov Pavel). - Fix reading of subcolumns from compact parts. #28873 (Anton Popov).
- Fixed a race condition between
DROP PART
andREPLACE/MOVE PARTITION
that might cause replicas to diverge in rare cases. #28864 (tavplubix). - Fix expressions compilation with short circuit evaluation. #28821 (Azat Khuzhin).
- Fix extremely rare case when ReplicatedMergeTree replicas can diverge after hard reboot of all replicas. The error looks like
Part ... intersects (previous|next) part ...
. #28817 (alesapin). - Better check for connection usability and also catch any exception in
RabbitMQ
shutdown just in case. #28797 (Kseniia Sumarokova). - Fix benign race condition in ReplicatedMergeTreeQueue. Shouldn't be visible for user, but can lead to subtle bugs. #28734 (alesapin).
- Fix possible crash for
SELECT
with partially created aggregate projection in case of exception. #28700 (Amos Bird). - Fix the coredump in the creation of distributed tables, when the parameters passed in are wrong. #28686 (Zhiyong Wang).
- Add Settings.Names, Settings.Values aliases for system.processes table. #28685 (Vitaly).
- Support for S2 Geometry library: Fix the number of arguments required by
s2RectAdd
ands2RectContains
functions. #28663 (Bharat Nallan). - Fix invalid constant type conversion when Nullable or LowCardinality primary key is used. #28636 (Amos Bird).
- Fix "Column is not under aggregate function and not in GROUP BY" with PREWHERE (Fixes: #28461). #28502 (Azat Khuzhin).
- Now the following MergeTree table-level settings:
replicated_max_parallel_sends
,replicated_max_parallel_sends_for_table
,replicated_max_parallel_fetches
,replicated_max_parallel_fetches_for_table
do nothing. They never worked well and were replaced withmax_replicated_fetches_network_bandwidth
,max_replicated_sends_network_bandwidth
andbackground_fetches_pool_size
. #28404 (alesapin).
- Add feature for creating user-defined functions (UDF) as lambda expressions. Syntax
CREATE FUNCTION {function_name} as ({parameters}) -> {function core}
. ExampleCREATE FUNCTION plus_one as (a) -> a + 1
. Authors @Realist007. #27796 (Maksim Kita) #23978 (Realist007). - Added
Executable
storage engine andexecutable
table function. It enables data processing with external scripts in streaming fashion. #28102 (Maksim Kita) (ruct). - Added
ExecutablePool
storage engine. Similar toExecutable
but it's using a pool of long running processes. #28518 (Maksim Kita). - Add
ALTER TABLE ... MATERIALIZE COLUMN
query. #27038 (Vladimir Chebotarev). - Support for partitioned write into
s3
table function. #23051 (Vladimir Chebotarev). - Support
lz4
compression format (in addition togz
,bz2
,xz
,zstd
) for data import / export. #25310 (Bharat Nallan). - Allow positional arguments under setting
enable_positional_arguments
. Closes #2592. #27530 (Kseniia Sumarokova). - Accept user settings related to file formats in
SETTINGS
clause inCREATE
query for s3 tables. This closes #27580. #28037 (Nikita Mikhaylov). - Allow SSL connection for
RabbitMQ
engine. #28365 (Kseniia Sumarokova). - Add
getServerPort
function to allow getting server port. When the port is not used by the server, throw an exception. #27900 (Amos Bird). - Add conversion functions between "snowflake id" and
DateTime
,DateTime64
. See #27058. #27704 (jasine). - Add function
SHA512
. #27830 (zhanglistar). - Add
log_queries_probability
setting that allows user to write to query_log only a sample of queries. Closes #16609. #27527 (Nikolay Degterinsky).
web
type of disks to store readonly tables on web server in form of static files. See #23982. #25251 (Kseniia Sumarokova). This is mostly needed to faciliate testing of operation on shared storage and for easy importing of datasets. Not recommended to use before release 21.11.- Added new commands
BACKUP
andRESTORE
. #21945 (Vitaly Baranov). This is under development and not intended to be used in current version.
- Speed up
sumIf
andcountIf
aggregation functions. #28272 (Raúl Marín). - Create virtual projection for
minmax
indices. Now, whenallow_experimental_projection_optimization
is enabled, queries will use minmax index instead of reading the data when possible. #26286 (Amos Bird). - Introducing two checks in
sequenceMatch
andsequenceCount
that allow for early exit when some deterministic part of the sequence pattern is missing from the events list. This change unlocks many queries that would previously fail due to reaching operations cap, and generally speeds up the pipeline. #27729 (Jakub Kuklis). - Enhance primary key analysis with always monotonic information of binary functions, notably non-zero constant division. #28302 (Amos Bird).
- Make
hasAll
filter condition leverage bloom filter data-skipping indexes. #27984 (Braulio Valdivielso Martínez). - Speed up data parts loading by delaying table startup process. #28313 (Amos Bird).
- Fixed possible excessive number of conditions moved from
WHERE
toPREWHERE
(optimization controlled by settingsoptimize_move_to_prewhere
). #28139 (lthaooo). - Enable
optimize_distributed_group_by_sharding_key
by default. #28105 (Azat Khuzhin).
- Check cluster name before creating
Distributed
table, do not allow to create a table with incorrect cluster name. Fixes #27832. #27927 (tavplubix). - Add aggregate function
quantileBFloat16Weighted
similarly to other quantile...Weighted functions. This closes #27745. #27758 (Ivan Novitskiy). - Allow to create dictionaries with empty attributes list. #27905 (Maksim Kita).
- Add interactive documentation in
clickhouse-client
about how to reset the password. This is useful in scenario when user has installed ClickHouse, set up the password and instantly forget it. See #27750. #27903 (alexey-milovidov). - Support the case when the data is enclosed in array in
JSONAsString
input format. Closes #25517. #25633 (Kruglov Pavel). - Add new column
last_queue_update_exception
tosystem.replicas
table. #26843 (nvartolomei). - Support reconnections on failover for
MaterializedPostgreSQL
tables. Closes #28529. #28614 (Kseniia Sumarokova). - Generate a unique server UUID on first server start. #20089 (Bharat Nallan).
- Introduce
connection_wait_timeout
(default to 5 seconds, 0 - do not wait) setting forMySQL
engine. #28474 (Azat Khuzhin). - Do not allow creating
MaterializedPostgreSQL
with bad arguments. Closes #28423. #28430 (Kseniia Sumarokova). - Use real tmp file instead of predefined "rows_sources" for vertical merges. This avoids generating garbage directories in tmp disks. #28299 (Amos Bird).
- Added
libhdfs3_conf
in server config instead of export envLIBHDFS3_CONF
in clickhouse-server.service. This is for configuration of interaction with HDFS. #28268 (Zhichang Yu). - Fix removing of parts in a Temporary state which can lead to an unexpected exception (
Part %name% doesn't exist
). Fixes #23661. #28221 #28221) (Azat Khuzhin). - Fix
zookeeper_log.address
(before the first patch in this PR the address was always::
) and reduce number of callsgetpeername(2)
for this column (since each time entry forzookeeper_log
is addedgetpeername()
is called, cache this address in the zookeeper client to avoid this). #28212 (Azat Khuzhin). - Support implicit conversions between index in operator
[]
and key of typeMap
(e.g. differentInt
types,String
andFixedString
). #28096 (Anton Popov). - Support
ON CONFLICT
clause when inserting into PostgreSQL table engine or table function. Closes #27727. #28081 (Kseniia Sumarokova). - Lower restrictions for
Enum
data type to allow attaching compatible data. Closes #26672. #28028 (Dmitry Novik). - Add a setting
empty_result_for_aggregation_by_constant_keys_on_empty_set
to control the behavior of grouping by constant keys on empty set. This is to bring back the old baviour of #6842. #27932 (Amos Bird). - Added
replication_wait_for_inactive_replica_timeout
setting. It allows to specify how long to wait for inactive replicas to executeALTER
/OPTIMZE
/TRUNCATE
query (default is 120 seconds). Ifreplication_alter_partitions_sync
is 2 and some replicas are not active for more thanreplication_wait_for_inactive_replica_timeout
seconds, thenUNFINISHED
will be thrown. #27931 (tavplubix). - Support lambda argument for
APPLY
column transformer which allows applying functions with more than one argument. This is for #27877. #27901 (Amos Bird). - Enable
tcp_keep_alive_timeout
by default. #27882 (Azat Khuzhin). - Improve remote query cancelation (in case of remote server abnormaly terminated). #27881 (Azat Khuzhin).
- Use Multipart copy upload for large S3 objects. #27858 (ianton-ru).
- Allow symlink traversal for library dictionaty path. #27815 (Kseniia Sumarokova).
- Now
ALTER MODIFY COLUM
T
toNullable(T)
doesn't require mutation. #27787 (victorgao). - Don't silently ignore errors and don't count delays in
ReadBufferFromS3
. #27484 (Vladimir Chebotarev). - Improve
ALTER ... MATERIALIZE TTL
by recalculating metadata only without actual TTL action. #27019 (lthaooo). - Allow reading the list of custom top level domains without a new line at EOF. #28213 (Azat Khuzhin).
- Fix cases, when reading compressed data from
carbon-clickhouse
fails with 'attempt to read after end of file'. Closes #26149. #28150 (FArthur-cmd). - Fix checking access grants when executing
GRANT WITH REPLACE
statement withON CLUSTER
clause. This PR improves fix #27001. #27983 (Vitaly Baranov). - Allow selecting with
extremes = 1
from a column of the typeLowCardinality(UUID)
. #27918 (Vitaly Baranov). - Fix PostgreSQL-style cast (
::
operator) with negative numbers. #27876 (Anton Popov). - After #26864. Fix shutdown of
NamedSessionStorage
: session contexts stored inNamedSessionStorage
are now destroyed before destroying the global context. #27875 (Vitaly Baranov). - Bugfix for
windowFunnel
"strict" mode. This fixes #27469. #27563 (achimbab). - Fix infinite loop while reading truncated
bzip2
archive. #28543 (Azat Khuzhin). - Fix UUID overlap in
DROP TABLE
for internal DDL fromMaterializedMySQL
. MaterializedMySQL is an experimental feature. #28533 (Azat Khuzhin). - Fix
There is no subcolumn
error, while select from tables, which haveNested
columns and scalar columns with dot in name and the same prefix asNested
(e.g.n.id UInt32, n.arr1 Array(UInt64), n.arr2 Array(UInt64)
). #28531 (Anton Popov). - Fix bug which can lead to error
Existing table metadata in ZooKeeper differs in sorting key expression.
after ALTER ofReplicatedVersionedCollapsingMergeTree
. Fixes #28515. #28528 (alesapin). - Fixed possible ZooKeeper watches leak (minor issue) on background processing of distributed DDL queue. Closes #26036. #28446 (tavplubix).
- Fix missing quoting of table names in
MaterializedPostgreSQL
engine. Closes #28316. #28433 (Kseniia Sumarokova). - Fix the wrong behaviour of non joined rows from nullable column. Close #27691. #28349 (vdimir).
- Fix NOT-IN index optimization when not all key columns are used. This fixes #28120. #28315 (Amos Bird).
- Fix intersecting parts due to new part had been replaced with an empty part. #28310 (Azat Khuzhin).
- Fix inconsistent result in queries with
ORDER BY
andMerge
tables with enabled settingoptimize_read_in_order
. #28266 (Anton Popov). - Fix possible read of uninitialized memory for queries with
Nullable(LowCardinality)
type and the settingextremes
set to 1. Fixes #28165. #28205 (Nikolai Kochetov). - Multiple small fixes for projections. See detailed description in the PR. #28178 (Amos Bird).
- Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. #28088 (nvartolomei).
- Fix handling null value with type of
Nullable(String)
in functionJSONExtract
. This fixes #27929 and #27930. This was introduced in ClickHouse#25452 . #27939 (Amos Bird). - Multiple fixes for the new
clickhouse-keeper
tool. Fix a rare bug inclickhouse-keeper
when the client can receive a watch response before request-response. #28197 (alesapin). Fix incorrect behavior inclickhouse-keeper
when list watches (getChildren
) triggered withset
requests for children. #28190 (alesapin). Fix rare case when changes ofclickhouse-keeper
settings may lead to lost logs and server hung. #28360 (alesapin). Fix bug inclickhouse-keeper
which can lead to endless logs whenrotate_logs_interval
decreased. #28152 (alesapin).
- Enable Thread Fuzzer in Stress Test. Thread Fuzzer is ClickHouse feature that allows to test more permutations of thread scheduling and discover more potential issues. This closes #9813. This closes #9814. This closes #9515. This closes #9516. #27538 (alexey-milovidov).
- Add new log level
test
for testing environments. It is even more verbose than the defaulttrace
. #28559 (alesapin). - Print out git status information at CMake configure stage. #28047 (Braulio Valdivielso Martínez).
- Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as the default one (archive.ubuntu.com) is not responding from our CI. #28016 (Ilya Yatsishin).
- Do not output trailing zeros in text representation of
Decimal
types. Example:1.23
will be printed instead of1.230000
for decimal with scale 6. This closes #15794. It may introduce slight incompatibility if your applications somehow relied on the trailing zeros. Serialization in output formats can be controlled with the settingoutput_format_decimal_trailing_zeros
. Implementation oftoString
and casting to String is changed unconditionally. #27680 (alexey-milovidov). - Do not allow to apply parametric aggregate function with
-Merge
combinator to aggregate function state if state was produced by aggregate function with different parameters. For example, state offooState(42)(x)
cannot be finalized withfooMerge(s)
orfooMerge(123)(s)
, parameters must be specified explicitly likefooMerge(42)(s)
and must be equal. It does not affect some special aggregate functions likequantile
andsequence*
that use parameters for finalization only. #26847 (tavplubix). - Under clickhouse-local, always treat local addresses with a port as remote. #26736 (Raúl Marín).
- Fix the issue that in case of some sophisticated query with column aliases identical to the names of expressions, bad cast may happen. This fixes #25447. This fixes #26914. This fix may introduce backward incompatibility: if there are different expressions with identical names, exception will be thrown. It may break some rare cases when
enable_optimize_predicate_expression
is set. #26639 (alexey-milovidov). - Now, scalar subquery always returns
Nullable
result if it's type can beNullable
. It is needed because in case of empty subquery it's result should beNull
. Previously, it was possible to get error about incompatible types (type deduction does not execute scalar subquery, and it could use not-nullable type). Scalar subquery with empty result which can't be converted toNullable
(likeArray
orTuple
) now throws error. Fixes #25411. #26423 (Nikolai Kochetov). - Introduce syntax for here documents. Example
SELECT $doc$ VALUE $doc$
. #26671 (Maksim Kita). This change is backward incompatible if in query there are identifiers that contain$
#28768. - Now indices can handle Nullable types, including
isNull
andisNotNull
. #12433 and #12455 (Amos Bird) and #27250 (Azat Khuzhin). But this was done with on-disk format changes, and even though new server can read old data, old server cannot. Also, in case you haveMINMAX
data skipping indices, you may getData after mutation/merge is not byte-identical
error, since new index will have.idx2
extension while before it was.idx
. That said, that you should not delay updating all existing replicas, in this case, otherwise, if old replica (<21.9) will download data from new replica with 21.9+ it will not be able to apply index for downloaded part.
- Implementation of short circuit function evaluation, closes #12587. Add settings
short_circuit_function_evaluation
to configure short circuit function evaluation. #23367 (Kruglov Pavel). - Add support for INTERSECT, EXCEPT, ANY, ALL operators. #24757 (Kirill Ershov). (Kseniia Sumarokova).
- Add support for encryption at the virtual file system level (data encryption at rest) using AES-CTR algorithm. #24206 (Latysheva Alexandra). (Vitaly Baranov) #26733 #26377 #26465.
- Added natural language processing (NLP) functions for tokenization, stemming, lemmatizing and search in synonyms extensions. #24997 (Nikolay Degterinsky).
- Added integration with S2 geometry library. #24980 (Andr0901). (Nikita Mikhaylov).
- Add SQLite table engine, table function, database engine. #24194 (Arslan Gumerov). (Kseniia Sumarokova).
- Added support for custom query for
MySQL
,PostgreSQL
,ClickHouse
,JDBC
,Cassandra
dictionary source. Closes #1270. #26995 (Maksim Kita). - Add shared (replicated) storage of user, roles, row policies, quotas and settings profiles through ZooKeeper. #27426 (Kevin Michel).
- Add compression for
INTO OUTFILE
that automatically choose compression algorithm. Closes #3473. #27134 (Filatenkov Artur). - Add
INSERT ... FROM INFILE
similarly toSELECT ... INTO OUTFILE
. #27655 (Filatenkov Artur). - Added
complex_key_range_hashed
dictionary. Closes #22029. #27629 (Maksim Kita). - Support expressions in JOIN ON section. Close #21868. #24420 (Vladimir C).
- When client connects to server, it receives information about all warnings that are already were collected by server. (It can be disabled by using option
--no-warnings
). Addsystem.warnings
table to collect warnings about server configuration. #26246 (Filatenkov Artur). #26282 (Filatenkov Artur). - Allow using constant expressions from with and select in aggregate function parameters. Close #10945. #27531 (abel-cheng).
- Add
tupleToNameValuePairs
, a function that turns a named tuple into an array of pairs. #27505 (Braulio Valdivielso Martínez). - Add support for
bzip2
compression method for import/export. Closes #22428. #27377 (Nikolay Degterinsky). - Added
bitmapSubsetOffsetLimit(bitmap, offset, cardinality_limit)
function. It creates a subset of bitmap limit the results tocardinality_limit
with offset ofoffset
. #27234 (DHBin). - Add column
default_database
tosystem.users
. #27054 (kevin wan). - Supported
cluster
macros inside table functions 'cluster' and 'clusterAllReplicas'. #26913 (polyprogrammist). - Add new functions
currentRoles()
,enabledRoles()
,defaultRoles()
. #26780 (Vitaly Baranov). - New functions
currentProfiles()
,enabledProfiles()
,defaultProfiles()
. #26714 (Vitaly Baranov). - Add functions that return (initial_)query_id of the current query. This closes #23682. #26410 (Alexey Boykov).
- Add
REPLACE GRANT
feature. #26384 (Caspian). EXPLAIN
query now hasEXPLAIN ESTIMATE ...
mode that will show information about read rows, marks and parts from MergeTree tables. Closes #23941. #26131 (fastio).- Added
system.zookeeper_log
table. All actions of ZooKeeper client are logged into this table. Implements #25449. #26129 (tavplubix). - Zero-copy replication for
ReplicatedMergeTree
overHDFS
storage. #25918 (Zhichang Yu). - Allow to insert Nested type as array of structs in
Arrow
,ORC
andParquet
input format. #25902 (Kruglov Pavel). - Add a new datatype
Date32
(store data as Int32), support date range same withDateTime64
support load parquet date32 to ClickHouseDate32
Add new functiontoDate32
liketoDate
. #25774 (LiuNeng). - Allow setting default database for users. #25268. #25687 (kevin wan).
- Add an optional parameter to
MongoDB
engine to accept connection string options and support SSL connection. Closes #21189. Closes #21041. #22045 (Omar Bazaraa).
- Added a compression codec
AES_128_GCM_SIV
which encrypts columns instead of compressing them. #19896 (PHO). Will be rewritten, do not use. - Rename
MaterializeMySQL
toMaterializedMySQL
. #26822 (tavplubix).
- Improve the performance of fast queries when
max_execution_time = 0
by reducing the number ofclock_gettime
system calls. #27325 (filimonov). - Specialize date time related comparison to achieve better performance. This fixes #27083 . #27122 (Amos Bird).
- Share file descriptors in concurrent reads of the same files. There is no noticeable performance difference on Linux. But the number of opened files will be significantly (10..100 times) lower on typical servers and it makes operations easier. See #26214. #26768 (alexey-milovidov).
- Improve latency of short queries, that require reading from tables with large number of columns. #26371 (Anton Popov).
- Don't build sets for indices when analyzing a query. #26365 (Raúl Marín).
- Vectorize the SUM of Nullable integer types with native representation (David Manzanares, Raúl Marín). #26248 (Raúl Marín).
- Compile expressions involving columns with
Enum
types. #26237 (Maksim Kita). - Compile aggregate functions
groupBitOr
,groupBitAnd
,groupBitXor
. #26161 (Maksim Kita). - Improved memory usage with better block size prediction when reading empty DEFAULT columns. Closes #17317. #25917 (Vladimir Chebotarev).
- Reduce memory usage and number of read rows in queries with
ORDER BY primary_key
. #25721 (Anton Popov). - Enable
distributed_push_down_limit
by default. #27104 (Azat Khuzhin). - Make
toTimeZone
monotonicity when timeZone is a constant value to support partition puring when use sql like:. #26261 (huangzhaowei).
- Mark window functions as ready for general use. Remove the
allow_experimental_window_functions
setting. #27184 (Alexander Kuzmenkov). - Improve compatibility with non-whole-minute timezone offsets. #27080 (Raúl Marín).
- If file descriptor in
File
table is regular file - allow to read multiple times from it. It allowsclickhouse-local
to read multiple times from stdin (with multiple SELECT queries or subqueries) if stdin is a regular file likeclickhouse-local --query "SELECT * FROM table UNION ALL SELECT * FROM table" ... < file
. This closes #11124. Co-authored with (alexey-milovidov). #25960 (BoloniniD). - Remove duplicate index analysis and avoid possible invalid limit checks during projection analysis. #27742 (Amos Bird).
- Enable query parameters to be passed in the body of HTTP requests. #27706 (Hermano Lustosa).
- Disallow
arrayJoin
on partition expressions. #27648 (Raúl Marín). - Log client IP address if authentication fails. #27514 (Misko Lee).
- Use bytes instead of strings for binary data in the GRPC protocol. #27431 (Vitaly Baranov).
- Send response with error message if HTTP port is not set and user tries to send HTTP request to TCP port. #27385 (Braulio Valdivielso Martínez).
- Add
_CAST
function for internal usage, which will not preserve type nullability, but non-internal cast will preserve according to settingcast_keep_nullable
. Closes #12636. #27382 (Kseniia Sumarokova). - Add setting
log_formatted_queries
to log additional formatted query intosystem.query_log
. It's useful for normalized query analysis because functions likenormalizeQuery
andnormalizeQueryKeepNames
don't parse/format queries in order to achieve better performance. #27380 (Amos Bird). - Add two settings
max_hyperscan_regexp_length
andmax_hyperscan_regexp_total_length
to prevent huge regexp being used in hyperscan related functions, such asmultiMatchAny
. #27378 (Amos Bird). - Memory consumed by bitmap aggregate functions now is taken into account for memory limits. This closes #26555. #27252 (alexey-milovidov).
- Add 10 seconds cache for S3 proxy resolver. #27216 (ianton-ru).
- Split global mutex into individual regexp construction. This helps avoid huge regexp construction blocking other related threads. #27211 (Amos Bird).
- Support schema for PostgreSQL database engine. Closes #27166. #27198 (Kseniia Sumarokova).
- Track memory usage in clickhouse-client. #27191 (Filatenkov Artur).
- Try recording
query_kind
insystem.query_log
even when query fails to start. #27182 (Amos Bird). - Added columns
replica_is_active
that maps replica name to is replica active status to tablesystem.replicas
. Closes #27138. #27180 (Maksim Kita). - Allow to pass query settings via server URI in Web UI. #27177 (kolsys).
- Add a new metric called
MaxPushedDDLEntryID
which is the maximum ddl entry id that current node push to zookeeper. #27174 (Fuwang Hu). - Improved the existence condition judgment and empty string node judgment when
clickhouse-keeper
creates znode. #27125 (小路). - Merge JOIN correctly handles empty set in the right. #27078 (Vladimir C).
- Now functions can be shard-level constants, which means if it's executed in the context of some distributed table, it generates a normal column, otherwise it produces a constant value. Notable functions are:
hostName()
,tcpPort()
,version()
,buildId()
,uptime()
, etc. #27020 (Amos Bird). - Updated
extractAllGroupsHorizontal
- upper limit on the number of matches per row can be set via optional third argument. #26961 (Vasily Nemkov). - Expose
RocksDB
statistics via system.rocksdb table. Read rocksdb options from ClickHouse config (rocksdb...
keys). NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. #26821 (Azat Khuzhin). - Less verbose internal RocksDB logs. NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. This closes #26252. #26789 (alexey-milovidov).
- Changing default roles affects new sessions only. #26759 (Vitaly Baranov).
- Watchdog is disabled in docker by default. Fix for not handling ctrl+c. #26757 (Mikhail f. Shiryaev).
SET PROFILE
now applies constraints too if they're set for a passed profile. #26730 (Vitaly Baranov).- Improve handling of
KILL QUERY
requests. #26675 (Raúl Marín). mapPopulatesSeries
function supportsMap
type. #26663 (Ildus Kurbangaliev).- Fix excessive (x2) connect attempts with
skip_unavailable_shards
. #26658 (Azat Khuzhin). - Avoid hanging
clickhouse-benchmark
if connection fails (i.e. on EMFILE). #26656 (Azat Khuzhin). - Allow more threads to be used by the Kafka engine. #26642 (feihengye).
- Add round-robin support for
clickhouse-benchmark
(it does not differ from the regular multi host/port run except for statistics report). #26607 (Azat Khuzhin). - Executable dictionaries (
executable
,executable_pool
) enable creation with DDL query usingclickhouse-local
. Closes #22355. #26510 (Maksim Kita). - Set client query kind for
mysql
andpostgresql
compatibility protocol handlers. #26498 (anneji-dev). - Apply
LIMIT
on the shards for queries likeSELECT * FROM dist ORDER BY key LIMIT 10
w/distributed_push_down_limit=1
. Avoid runningDistinct
/LIMIT BY
steps for queries likeSELECT DISTINCT shading_key FROM dist ORDER BY key
. Nowdistributed_push_down_limit
is respected byoptimize_distributed_group_by_sharding_key
optimization. #26466 (Azat Khuzhin). - Updated protobuf to 3.17.3. Changelogs are available on https://github.com/protocolbuffers/protobuf/releases. #26424 (Ilya Yatsishin).
- Enable
use_hedged_requests
setting that allows to mitigate tail latencies on large clusters. #26380 (alexey-milovidov). - Improve behaviour with non-existing host in user allowed host list. #26368 (ianton-ru).
- Add ability to set
Distributed
directory monitor settings via CREATE TABLE (i.e.CREATE TABLE dist (key Int) Engine=Distributed(cluster, db, table) SETTINGS monitor_batch_inserts=1
and similar). #26336 (Azat Khuzhin). - Save server address in history URLs in web UI if it differs from the origin of web UI. This closes #26044. #26322 (alexey-milovidov).
- Add events to profile calls to
sleep
/sleepEachRow
. #26320 (Raúl Marín). - Allow to reuse connections of shards among different clusters. It also avoids creating new connections when using
cluster
table function. #26318 (Amos Bird). - Control the execution period of clear old temporary directories by parameter with default value. #26212. #26313 (fastio).
- Add a setting
function_range_max_elements_in_block
to tune the safety threshold for data volume generated by functionrange
. This closes #26303. #26305 (alexey-milovidov). - Check hash function at table creation, not at sampling. Add settings for MergeTree, if someone create a table with incorrect sampling column but sampling never be used, disable this settings for starting the server without exception. #26256 (zhaoyu).
- Added
output_format_avro_string_column_pattern
setting to put specified String columns to Avro as string instead of default bytes. Implements #22414. #26245 (Ilya Golshtein). - Add information about column sizes in
system.columns
table forLog
andTinyLog
tables. This closes #9001. #26241 (Nikolay Degterinsky). - Don't throw exception when querying
system.detached_parts
table if there is custom disk configuration anddetached
directory does not exist on some disks. This closes #26078. #26236 (alexey-milovidov). - Check for non-deterministic functions in keys, including constant expressions like
now()
,today()
. This closes #25875. This closes #11333. #26235 (alexey-milovidov). - convert timestamp and timestamptz data types to
DateTime64
in PostgreSQL table engine. #26234 (jasine). - Apply aggressive IN index analysis for projections so that better projection candidate can be selected. #26218 (Amos Bird).
- Remove GLOBAL keyword for IN when scalar function is passed. In previous versions, if user specified
GLOBAL IN f(x)
exception was thrown. #26217 (Amos Bird). - Add error id (like
BAD_ARGUMENTS
) to exception messages. This closes #25862. #26172 (alexey-milovidov). - Fix incorrect output with --progress option for clickhouse-local. Progress bar will be cleared once it gets to 100% - same as it is done for clickhouse-client. Closes #17484. #26128 (Kseniia Sumarokova).
- Add
merge_selecting_sleep_ms
setting. #26120 (lthaooo). - Remove complicated usage of Linux AIO with one block readahead and replace it with plain simple synchronous IO with O_DIRECT. In previous versions, the setting
min_bytes_to_use_direct_io
may not work correctly ifmax_threads
is greater than one. Reading with direct IO (that is disabled by default for queries and enabled by default for large merges) will work in less efficient way. This closes #25997. #26003 (alexey-milovidov). - Flush
Distributed
table onREPLACE TABLE
query. Resolves #24566 - Do not replace (or create) table on[CREATE OR] REPLACE TABLE ... AS SELECT
query if insertion into new table fails. Resolves #23175. #25895 (tavplubix). - Add
views
column to system.query_log containing the names of the (materialized or live) views executed by the query. Adds a new log table (system.query_views_log
) that contains information about each view executed during a query. Modifies view execution: When an exception is thrown while executing a view, any view that has already startedwill continue running until it finishes. This used to be the behaviour under parallel_view_processing=true and now it's always the same behaviour. - Dependent views now report reading progress to the context. #25714 (Raúl Marín). - Do connection draining asynchonously upon finishing executing distributed queries. A new server setting is added
max_threads_for_connection_collector
which specifies the number of workers to recycle connections in background. If the pool is full, connection will be drained synchronously but a bit different than before: It's drained after we send EOS to client, query will succeed immediately after receiving enough data, and any exception will be logged instead of throwing to the client. Added settingdrain_timeout
(3 seconds by default). Connection draining will disconnect upon timeout. #25674 (Amos Bird). - Support for multiple includes in configuration. It is possible to include users configuration, remote servers configuration from multiple sources. Simply place
<include />
element withfrom_zk
,from_env
orincl
attribute and it will be replaced with the substitution. #24404 (nvartolomei). - Fix multiple block insertion into distributed table with
insert_distributed_one_random_shard = 1
. This is a marginal feature. Mark as improvement. #23140 (Amos Bird). - Support
LowCardinality
andFixedString
keys/values forMap
type. #21543 (hexiaoting). - Enable reloading of local disk config. #19526 (taiyang-li).
- Fix a couple of bugs that may cause replicas to diverge. #27808 (tavplubix).
- Fix a rare bug in
DROP PART
which can lead to the errorUnexpected merged part intersects drop range
. #27807 (alesapin). - Prevent crashes for some formats when NULL (tombstone) message was coming from Kafka. Closes #19255. #27794 (filimonov).
- Fix column filtering with union distinct in subquery. Closes #27578. #27689 (Kseniia Sumarokova).
- Fix bad type cast when functions like
arrayHas
are applied to arrays of LowCardinality of Nullable of different non-numeric types likeDateTime
andDateTime64
. In previous versions bad cast occurs. In new version it will lead to exception. This closes #26330. #27682 (alexey-milovidov). - Fix postgresql table function resulting in non-closing connections. Closes #26088. #27662 (Kseniia Sumarokova).
- Fixed another case of
Unexpected merged part ... intersecting drop range ...
error. #27656 (tavplubix). - Fix an error with aliased column in
Distributed
table. #27652 (Vladimir C). - After setting
max_memory_usage*
to non-zero value it was not possible to reset it back to 0 (unlimited). It's fixed. #27638 (tavplubix). - Fixed underflow of the time value when constructing it from components. Closes #27193. #27605 (Vasily Nemkov).
- Fix crash during projection materialization when some parts contain missing columns. This fixes #27512. #27528 (Amos Bird).
- fix metric
BackgroundMessageBrokerSchedulePoolTask
, maybe mistyped. #27452 (Ben). - Fix distributed queries with zero shards and aggregation. #27427 (Azat Khuzhin).
- Compatibility when
/proc/meminfo
does not contain KB suffix. #27361 (Mike Kot). - Fix incorrect result for query with row-level security, PREWHERE and LowCardinality filter. Fixes #27179. #27329 (Nikolai Kochetov).
- Fixed incorrect validation of partition id for MergeTree tables that created with old syntax. #27328 (tavplubix).
- Fix MySQL protocol when using parallel formats (CSV / TSV). #27326 (Raúl Marín).
- Fix
Cannot find column
error for queries with sampling. Was introduced in #24574. Fixes #26522. #27301 (Nikolai Kochetov). - Fix errors like
Expected ColumnLowCardinality, gotUInt8
orBad cast from type DB::ColumnVector<char8_t> to DB::ColumnLowCardinality
for some queries withLowCardinality
inPREWHERE
. And more importantly, fix the lack of whitespace in the error message. Fixes #23515. #27298 (Nikolai Kochetov). - Fix
distributed_group_by_no_merge = 2
withdistributed_push_down_limit = 1
oroptimize_distributed_group_by_sharding_key = 1
withLIMIT BY
andLIMIT OFFSET
. #27249 (Azat Khuzhin). These are obscure combination of settings that no one is using. - Fix mutation stuck on invalid partitions in non-replicated MergeTree. #27248 (Azat Khuzhin).
- In case of ambiguity, lambda functions prefer its arguments to other aliases or identifiers. #27235 (Raúl Marín).
- Fix column structure in merge join, close #27091. #27217 (Vladimir C).
- In rare cases
system.detached_parts
table might contain incorrect information for some parts, it's fixed. Fixes #27114. #27183 (tavplubix). - Fix uninitialized memory in functions
multiSearch*
with empty array, close #27169. #27181 (Vladimir C). - Fix synchronization in GRPCServer. This PR fixes #27024. #27064 (Vitaly Baranov).
- Fixed
cache
,complex_key_cache
,ssd_cache
,complex_key_ssd_cache
configuration parsing. Optionsallow_read_expired_keys
,max_update_queue_size
,update_queue_push_timeout_milliseconds
,query_wait_timeout_milliseconds
were not parsed for dictionaries with noncache
type. #27032 (Maksim Kita). - Fix possible mutation stack due to race with DROP_RANGE. #27002 (Azat Khuzhin).
- Now partition ID in queries like
ALTER TABLE ... PARTITION ID xxx
validates for correctness. Fixes #25718. #26963 (alesapin). - Fix "Unknown column name" error with multiple JOINs in some cases, close #26899. #26957 (Vladimir C).
- Fix reading of custom TLDs (stops processing with lower buffer or bigger file). #26948 (Azat Khuzhin).
- Fix error
Missing columns: 'xxx'
whenDEFAULT
column references other non materialized column withoutDEFAULT
expression. Fixes #26591. #26900 (alesapin). - Fix loading of dictionary keys in
library-bridge
forlibrary
dictionary source. #26834 (Kseniia Sumarokova). - Aggregate function parameters might be lost when applying some combinators causing exceptions like
Conversion from AggregateFunction(topKArray, Array(String)) to AggregateFunction(topKArray(10), Array(String)) is not supported
. It's fixed. Fixes #26196 and #26433. #26814 (tavplubix). - Add
event_time_microseconds
value forREMOVE_PART
insystem.part_log
. In previous versions is was not set. #26720 (Azat Khuzhin). - Do not remove data on ReplicatedMergeTree table shutdown to avoid creating data to metadata inconsistency. #26716 (nvartolomei).
- Sometimes
SET ROLE
could work incorrectly, this PR fixes that. #26707 (Vitaly Baranov). - Some fixes for parallel formatting (ClickHouse#26694). #26703 (Raúl Marín).
- Fix potential nullptr dereference in window functions. This fixes #25276. #26668 (Alexander Kuzmenkov).
- Fix clickhouse-client history file conversion (when upgrading from the format of 3 years old version of clickhouse-client) if file is empty. #26589 (Azat Khuzhin).
- Fix incorrect function names of groupBitmapAnd/Or/Xor (can be displayed in some occasions). This fixes. #26557 (Amos Bird).
- Update
chown
cmd check in clickhouse-server docker entrypoint. It fixes the bug that cluster pod restart failed (or timeout) on kubernetes. #26545 (Ky Li). - Fix crash in
RabbitMQ
shutdown in caseRabbitMQ
setup was not started. Closes #26504. #26529 (Kseniia Sumarokova). - Fix issues with
CREATE DICTIONARY
query if dictionary name or database name was quoted. Closes #26491. #26508 (Maksim Kita). - Fix broken column name resolution after rewriting column aliases. This fixes #26432. #26475 (Amos Bird).
- Fix some fuzzed msan crash. Fixes #22517. #26428 (Nikolai Kochetov).
- Fix infinite non joined block stream in
partial_merge_join
close #26325. #26374 (Vladimir C). - Fix possible crash when login as dropped user. This PR fixes #26073. #26363 (Vitaly Baranov).
- Fix
optimize_distributed_group_by_sharding_key
for multiple columns (leads to incorrect result w/optimize_skip_unused_shards=1
/allow_nondeterministic_optimize_skip_unused_shards=1
and multiple columns in sharding key expression). #26353 (Azat Khuzhin). - Fixed rare bug in lost replica recovery that may cause replicas to diverge. #26321 (tavplubix).
- Fix zstd decompression (for import/export in zstd framing format that is unrelated to tables data) in case there are escape sequences at the end of internal buffer. Closes #26013. #26314 (Kseniia Sumarokova).
- Fix logical error on join with totals, close #26017. #26250 (Vladimir C).
- Remove excessive newline in
thread_name
column insystem.stack_trace
table. This fixes #24124. #26210 (alexey-milovidov). - Fix potential crash if more than one
untuple
expression is used. #26179 (alexey-milovidov). - Don't throw exception in
toString
for Nullable Enum if Enum does not have a value for zero, close #25806. #26123 (Vladimir C). - Fixed incorrect
sequence_id
in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes #21184. #26051 (tavplubix). - Fix for the case that
cutToFirstSignificantSubdomainCustom()
/cutToFirstSignificantSubdomainCustomWithWWW()
/firstSignificantSubdomainCustom()
returns incorrect type for consts, and henceoptimize_skip_unused_shards
does not work:. #26041 (Azat Khuzhin). - Fix possible mismatched header when using normal projection with prewhere. This fixes #26020. #26038 (Amos Bird).
- Fix sharding_key from column w/o function for remote() (before
select * from remote('127.1', system.one, dummy)
leads toUnknown column: dummy, there are only columns .
error). #25824 (Azat Khuzhin). - Fixed
Not found column ...
andMissing column ...
errors when selecting fromMaterializeMySQL
. Fixes #23708, #24830, #25794. #25822 (tavplubix). - Fix
optimize_skip_unused_shards_rewrite_in
for non-UInt64 types (may select incorrect shards eventually or throwCannot infer type of an empty tuple
orFunction tuple requires at least one argument
). #25798 (Azat Khuzhin).
- Now we ran stateful and stateless tests in random timezones. Fixes #12439. Reading String as DateTime and writing DateTime as String in Protobuf format now respect timezone. Reading UInt16 as DateTime in Arrow and Parquet formats now treat it as Date and then converts to DateTime with respect to DateTime's timezone, because Date is serialized in Arrow and Parquet as UInt16. GraphiteMergeTree now respect time zone for rounding of times. Fixes #5098. Author: @alexey-milovidov. #15408 (alesapin).
clickhouse-test
supports SQL tests with Jinja2 templates. #26579 (Vladimir C).- Add support for build with
clang-13
. This closes #27705. #27714 (alexey-milovidov). #27777 (Sergei Semin) - Add CMake options to build with or without specific CPU instruction set. This is for #17469 and #27509. #27508 (alexey-milovidov).
- Fix linking of auxiliar programs when using dynamic libraries. #26958 (Raúl Marín).
- Update RocksDB to
2021-07-16
master. #26411 (alexey-milovidov).
- New version is using
Map
data type for system logs tables (system.query_log
,system.query_thread_log
,system.processes
,system.opentelemetry_span_log
). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes #18698. #23934, #25773 (hexiaoting, sundy-li, Maksim Kita). If you want to downgrade from version 21.8 to older versions, you will need to cleanup system tables with logs manually. Look at/var/lib/clickhouse/data/system/*_log
.
- Add support for a part of SQL/JSON standard. #24148 (l1tsolaiki, Kseniia Sumarokova).
- Collect common system metrics (in
system.asynchronous_metrics
andsystem.asynchronous_metric_log
) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar toatop
in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close #9430. #24416 (alexey-milovidov, Yegor Levankov). - Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. #20470 (Kseniia Sumarokova).
- Add new functions
leftPad()
,rightPad()
,leftPadUTF8()
,rightPadUTF8()
. #26075 (Vitaly Baranov). - Add the
FIRST
keyword to theADD INDEX
command to be able to add the index at the beginning of the indices list. #25904 (xjewer). - Introduce
system.data_skipping_indices
table containing information about existing data skipping indices. Close #7659. #25693 (Dmitry Novik). - Add
bin
/unbin
functions. #25609 (zhaoyu). - Support
Map
andUInt128
,Int128
,UInt256
,Int256
types inmapAdd
andmapSubtract
functions. #25596 (Ildus Kurbangaliev). - Support
DISTINCT ON (columns)
expression, close #25404. #25589 (Zijie Lu). - Add an ability to reset a custom setting to default and remove it from the table's metadata. It allows rolling back the change without knowing the system/config's default. Closes #14449. #17769 (xjewer).
- Render pipelines as graphs in Web UI if
EXPLAIN PIPELINE graph = 1
query is submitted. #26067 (alexey-milovidov).
- Compile aggregate functions. Use option
compile_aggregate_expressions
to enable it. #24789 (Maksim Kita). - Improve latency of short queries that require reading from tables with many columns. #26371 (Anton Popov).
- Use
Map
data type for system logs tables (system.query_log
,system.query_thread_log
,system.processes
,system.opentelemetry_span_log
). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes #18698. #23934, #25773 (hexiaoting, sundy-li, Maksim Kita). - For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions
dictGet
,dictHas
. #26130 (Maksim Kita). - Implement function
bin
/hex
fromAggregateFunction
states. #26094 (zhaoyu). - Support arguments of
UUID
type forempty
andnotEmpty
functions.UUID
is empty if it is all zeros (nil UUID). Closes #3446. #25974 (zhaoyu). - Add support for
SET SQL_SELECT_LIMIT
in MySQL protocol. Closes #17115. #25972 (Kseniia Sumarokova). - More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close #5897. #25962 (alexey-milovidov).
- Add setting
optimize_move_to_prewhere_if_final
. If query hasFINAL
, the optimizationmove_to_prewhere
will be enabled only if bothoptimize_move_to_prewhere
andoptimize_move_to_prewhere_if_final
are enabled. Closes #8684. #25940 (Kseniia Sumarokova). - Allow complex quoted identifiers of JOINed tables. Close #17861. #25924 (alexey-milovidov).
- Add support for Unicode (e.g. Chinese, Cyrillic) components in
Nested
data types. Close #25594. #25923 (alexey-milovidov). - Allow
quantiles*
functions to work withaggregate_functions_null_for_empty
. Close #25892. #25919 (alexey-milovidov). - Allow parameters for parametric aggregate functions to be arbitrary constant expressions (e.g.,
1 + 2
), not just literals. It also allows using the query parameters (in parameterized queries like{param:UInt8}
) inside parametric aggregate functions. Closes #11607. #25910 (alexey-milovidov). - Correctly throw the exception on the attempt to parse an invalid
Date
. Closes #6481. #25909 (alexey-milovidov). - Support for multiple includes in configuration. It is possible to include users configuration, remote server configuration from multiple sources. Simply place
<include />
element withfrom_zk
,from_env
orincl
attribute, and it will be replaced with the substitution. #24404 (nvartolomei). - Support for queries with a column named
"null"
(it must be specified in back-ticks or double quotes) andON CLUSTER
. Closes #24035. #25907 (alexey-milovidov). - Support
LowCardinality
,Decimal
, andUUID
forJSONExtract
. Closes #24606. #25900 (Kseniia Sumarokova). - Convert history file from
readline
format toreplxx
format. #25888 (Azat Khuzhin). - Fix an issue which can lead to intersecting parts after
DROP PART
or background deletion of an empty part. #25884 (alesapin). - Better handling of lost parts for
ReplicatedMergeTree
tables. Fixes rare inconsistencies inReplicationQueue
. Fixes #10368. #25820 (alesapin). - Allow starting clickhouse-client with unreadable working directory. #25817 (ianton-ru).
- Fix "No available columns" error for
Merge
storage. #25801 (Azat Khuzhin). - MySQL Engine now supports the exchange of column comments between MySQL and ClickHouse. #25795 (Storozhuk Kostiantyn).
- Fix inconsistent behaviour of
GROUP BY
constant on empty set. Closes #6842. #25786 (Kseniia Sumarokova). - Cancel already running merges in partition on
DROP PARTITION
andTRUNCATE
forReplicatedMergeTree
. Resolves #17151. #25684 (tavplubix). - Support ENUM` data type for MaterializeMySQL. #25676 (Storozhuk Kostiantyn).
- Support materialized and aliased columns in JOIN, close #13274. #25634 (Vladimir C).
- Fix possible logical race condition between
ALTER TABLE ... DETACH
and background merges. #25605 (Azat Khuzhin). - Make
NetworkReceiveElapsedMicroseconds
metric to correctly include the time spent waiting for data from the client toINSERT
. Close #9958. #25602 (alexey-milovidov). - Support
TRUNCATE TABLE
for S3 and HDFS. Close #25530. #25550 (Kseniia Sumarokova). - Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). #25548 (Nikita Mikhaylov).
- Allow extracting of non-string element as string using
JSONExtract
. This is for #25414. #25452 (Amos Bird). - Support regular expression in
Database
argument forStorageMerge
. Close #776. #25064 (flynn). - Web UI: if the value looks like a URL, automatically generate a link. #25965 (alexey-milovidov).
- Make
sudo service clickhouse-server start
to work on systems withsystemd
like Centos 8. Close #14298. Close #17799. #25921 (alexey-milovidov).
- Fix incorrect
SET ROLE
in some cases. #26707 (Vitaly Baranov). - Fix potential
nullptr
dereference in window functions. Fix #25276. #26668 (Alexander Kuzmenkov). - Fix incorrect function names of
groupBitmapAnd/Or/Xor
. Fix #26557 (Amos Bird). - Fix crash in RabbitMQ shutdown in case RabbitMQ setup was not started. Closes #26504. #26529 (Kseniia Sumarokova).
- Fix issues with
CREATE DICTIONARY
query if dictionary name or database name was quoted. Closes #26491. #26508 (Maksim Kita). - Fix broken name resolution after rewriting column aliases. Fix #26432. #26475 (Amos Bird).
- Fix infinite non-joined block stream in
partial_merge_join
close #26325. #26374 (Vladimir C). - Fix possible crash when login as dropped user. Fix #26073. #26363 (Vitaly Baranov).
- Fix
optimize_distributed_group_by_sharding_key
for multiple columns (leads to incorrect result w/optimize_skip_unused_shards=1
/allow_nondeterministic_optimize_skip_unused_shards=1
and multiple columns in sharding key expression). #26353 (Azat Khuzhin). CAST
fromDate
toDateTime
(orDateTime64
) was not using the timezone of theDateTime
type. It can also affect the comparison betweenDate
andDateTime
. Inference of the common type forDate
andDateTime
also was not using the corresponding timezone. It affected the results of functionif
and array construction. Closes #24128. #24129 (Maksim Kita).- Fixed rare bug in lost replica recovery that may cause replicas to diverge. #26321 (tavplubix).
- Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes #26013. #26314 (Kseniia Sumarokova).
- Fix logical error on join with totals, close #26017. #26250 (Vladimir C).
- Remove excessive newline in
thread_name
column insystem.stack_trace
table. Fix #24124. #26210 (alexey-milovidov). - Fix
joinGet
withLowCarinality
columns, close #25993. #26118 (Vladimir C). - Fix possible crash in
pointInPolygon
if the settingvalidate_polygons
is turned off. #26113 (alexey-milovidov). - Fix throwing exception when iterate over non-existing remote directory. #26087 (ianton-ru).
- Fix rare server crash because of
abort
in ZooKeeper client. Fixes #25813. #26079 (alesapin). - Fix wrong thread count estimation for right subquery join in some cases. Close #24075. #26052 (Vladimir C).
- Fixed incorrect
sequence_id
in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes #21184. #26051 (tavplubix). - Fix possible mismatched header when using normal projection with
PREWHERE
. Fix #26020. #26038 (Amos Bird). - Fix formatting of type
Map
with integer keys toJSON
. #25982 (Anton Popov). - Fix possible deadlock during query profiler stack unwinding. Fix #25968. #25970 (Maksim Kita).
- Fix crash on call
dictGet()
with bad arguments. #25913 (Vitaly Baranov). - Fixed
scram-sha-256
authentication for PostgreSQL engines. Closes #24516. #25906 (Kseniia Sumarokova). - Fix extremely long backoff for background tasks when the background pool is full. Fixes #25836. #25893 (alesapin).
- Fix ARM exception handling with non default page size. Fixes #25512, #25044, #24901, #23183, #20221, #19703, #19028, #18391, #18121, #17994, #12483. #25854 (Maksim Kita).
- Fix sharding_key from column w/o function for
remote()
(beforeselect * from remote('127.1', system.one, dummy)
leads toUnknown column: dummy, there are only columns .
error). #25824 (Azat Khuzhin). - Fixed
Not found column ...
andMissing column ...
errors when selecting fromMaterializeMySQL
. Fixes #23708, #24830, #25794. #25822 (tavplubix). - Fix
optimize_skip_unused_shards_rewrite_in
for non-UInt64 types (may select incorrect shards eventually or throwCannot infer type of an empty tuple
orFunction tuple requires at least one argument
). #25798 (Azat Khuzhin). - Fix rare bug with
DROP PART
query forReplicatedMergeTree
tables which can lead to error messageUnexpected merged part intersecting drop range
. #25783 (alesapin). - Fix bug in
TTL
withGROUP BY
expression which refuses to executeTTL
after first execution in part. #25743 (alesapin). - Allow StorageMerge to access tables with aliases. Closes #6051. #25694 (Kseniia Sumarokova).
- Fix slow dict join in some cases, close #24209. #25618 (Vladimir C).
- Fix
ALTER MODIFY COLUMN
of columns, which participates in TTL expressions. #25554 (Anton Popov). - Fix assertion in
PREWHERE
with non-UInt8 type, close #19589. #25484 (Vladimir C). - Fix some fuzzed msan crash. Fixes #22517. #26428 (Nikolai Kochetov).
- Update
chown
cmd check inclickhouse-server
docker entrypoint. It fixes error 'cluster pod restart failed (or timeout)' on kubernetes. #26545 (Ky Li).
- Improved performance of queries with explicitly defined large sets. Added compatibility setting
legacy_column_name_of_tuple_literal
. It makes sense to set it totrue
, while doing rolling update of cluster from version lower than 21.7 to any higher version. Otherwise distributed queries with explicitly defined sets atIN
clause may fail during update. #25371 (Anton Popov). - Forward/backward incompatible change of maximum buffer size in clickhouse-keeper (an experimental alternative to ZooKeeper). Better to do it now (before production), than later. #25421 (alesapin).
- Support configuration in YAML format as alternative to XML. This closes #3607. #21858 (BoloniniD).
- Provides a way to restore replicated table when the data is (possibly) present, but the ZooKeeper metadata is lost. Resolves #13458. #13652 (Mike Kot).
- Support structs and maps in Arrow/Parquet/ORC and dictionaries in Arrow input/output formats. Present new setting
output_format_arrow_low_cardinality_as_dictionary
. #24341 (Kruglov Pavel). - Added support for
Array
type in dictionaries. #25119 (Maksim Kita). - Added function
bitPositionsToArray
. Closes #23792. Author [Kevin Wan] (@MaxWk). #25394 (Maksim Kita). - Added function
dateName
to return names like 'Friday' or 'April'. Author [Daniil Kondratyev] (@dankondr). #25372 (Maksim Kita). - Add
toJSONString
function to serialize columns to their JSON representations. #25164 (Amos Bird). - Now
query_log
has two new columns:initial_query_start_time
,initial_query_start_time_microsecond
that record the starting time of a distributed query if any. #25022 (Amos Bird). - Add aggregate function
segmentLengthSum
. #24250 (flynn). - Add a new boolean setting
prefer_global_in_and_join
which defaults all IN/JOIN as GLOBAL IN/JOIN. #23434 (Amos Bird). - Support
ALTER DELETE
queries forJoin
table engine. #23260 (foolchi). - Add
quantileBFloat16
aggregate function as well as the correspondingquantilesBFloat16
andmedianBFloat16
. It is very simple and fast quantile estimator with relative error not more than 0.390625%. This closes #16641. #23204 (Ivan Novitskiy). - Implement
sequenceNextNode()
function useful forflow analysis
. #19766 (achimbab).
- Add support for virtual filesystem over HDFS. #11058 (overshov) (Kseniia Sumarokova).
- Now clickhouse-keeper (an experimental alternative to ZooKeeper) supports ZooKeeper-like
digest
ACLs. #24448 (alesapin).
- Added optimization that transforms some functions to reading of subcolumns to reduce amount of read data. E.g., statement
col IS NULL
is transformed to reading of subcolumncol.null
. Optimization can be enabled by settingoptimize_functions_to_subcolumns
which is currently off by default. #24406 (Anton Popov). - Rewrite more columns to possible alias expressions. This may enable better optimization, such as projections. #24405 (Amos Bird).
- Index of type
bloom_filter
can be used for expressions withhasAny
function with constant arrays. This closes: #24291. #24900 (Vasily Nemkov). - Add exponential backoff to reschedule read attempt in case RabbitMQ queues are empty. (ClickHouse has support for importing data from RabbitMQ). Closes #24340. #24415 (Kseniia Sumarokova).
- Allow to limit bandwidth for replication. Add two Replicated*MergeTree settings:
max_replicated_fetches_network_bandwidth
andmax_replicated_sends_network_bandwidth
which allows to limit maximum speed of replicated fetches/sends for table. Add two server-wide settings (indefault
user profile):max_replicated_fetches_network_bandwidth_for_server
andmax_replicated_sends_network_bandwidth_for_server
which limit maximum speed of replication for all tables. The settings are not followed perfectly accurately. Turned off by default. Fixes #1821. #24573 (alesapin). - Resource constraints and isolation for ODBC and Library bridges. Use separate
clickhouse-bridge
group and user for bridge processes. Set oom_score_adj so the bridges will be first subjects for OOM killer. Set set maximum RSS to 1 GiB. Closes #23861. #25280 (Kseniia Sumarokova). - Add standalone
clickhouse-keeper
symlink to the mainclickhouse
binary. Now it's possible to run coordination without the main clickhouse server. #24059 (alesapin). - Use global settings for query to
VIEW
. Fixed the behavior when queries toVIEW
use local settings, that leads to errors if setting onCREATE VIEW
andSELECT
were different. As for now,VIEW
won't use these modified settings, but you can still pass additional settings inSETTINGS
section ofCREATE VIEW
query. Close #20551. #24095 (Vladimir). - On server start, parts with incorrect partition ID would not be ever removed, but always detached. #25070. #25166 (Nikolai Kochetov).
- Increase size of background schedule pool to 128 (
background_schedule_pool_size
setting). It allows avoiding replication queue hung on slow zookeeper connection. #25072 (alesapin). - Add merge tree setting
max_parts_to_merge_at_once
which limits the number of parts that can be merged in the background at once. Doesn't affectOPTIMIZE FINAL
query. Fixes #1820. #24496 (alesapin). - Allow
NOT IN
operator to be used in partition pruning. #24894 (Amos Bird). - Recognize IPv4 addresses like
127.0.1.1
as local. This is controversial and closes #23504. Michael Filimonov will test this feature. #24316 (alexey-milovidov). - ClickHouse database created with MaterializeMySQL (it is an experimental feature) now contains all column comments from the MySQL database that materialized. #25199 (Storozhuk Kostiantyn).
- Add settings (
connection_auto_close
/connection_max_tries
/connection_pool_size
) for MySQL storage engine. #24146 (Azat Khuzhin). - Improve startup time of Distributed engine. #25663 (Azat Khuzhin).
- Improvement for Distributed tables. Drop replicas from dirname for internal_replication=true (allows INSERT into Distributed with cluster from any number of replicas, before only 15 replicas was supported, everything more will fail with ENAMETOOLONG while creating directory for async blocks). #25513 (Azat Khuzhin).
- Added support
Interval
type forLowCardinality
. It is needed for intermediate values of some expressions. Closes #21730. #25410 (Vladimir). - Add
==
operator on time conditions forsequenceMatch
andsequenceCount
functions. For eg: sequenceMatch('(?1)(?t==1)(?2)')(time, data = 1, data = 2). #25299 (Christophe Kalenzaga). - Add settings
http_max_fields
,http_max_field_name_size
,http_max_field_value_size
. #25296 (Ivan). - Add support for function
if
withDecimal
andInt
types on its branches. This closes #20549. This closes #10142. #25283 (alexey-milovidov). - Update prompt in
clickhouse-client
and display a message when reconnecting. This closes #10577. #25281 (alexey-milovidov). - Correct memory tracking in aggregate function
topK
. This closes #25259. #25260 (alexey-milovidov). - Fix
topLevelDomain
for IDN hosts (i.e.example.рф
), before it returns empty string for such hosts. #25103 (Azat Khuzhin). - Detect Linux kernel version at runtime (for worked nested epoll, that is required for
async_socket_for_remote
/use_hedged_requests
, otherwise remote queries may stuck). #25067 (Azat Khuzhin). - For distributed query, when
optimize_skip_unused_shards=1
, allow to skip shard with condition like(sharding key) IN (one-element-tuple)
. (Tuples with many elements were supported. Tuple with single element did not work because it is parsed as literal). #24930 (Amos Bird). - Improved log messages of S3 errors, no more double whitespaces in case of empty keys and buckets. #24897 (Vladimir Chebotarev).
- Some queries require multi-pass semantic analysis. Try reusing built sets for
IN
in this case. #24874 (Amos Bird). - Respect
max_distributed_connections
forinsert_distributed_sync
(otherwise for huge clusters and sync insert it may run out ofmax_thread_pool_size
). #24754 (Azat Khuzhin). - Avoid hiding errors like
Limit for rows or bytes to read exceeded
for scalar subqueries. #24545 (nvartolomei). - Make String-to-Int parser stricter so that
toInt64('+')
will throw. #24475 (Amos Bird). - If
SSD_CACHE
is created with DDL query, it can be created only insideuser_files
directory. #24466 (Maksim Kita). - PostgreSQL support for specifying non default schema for insert queries. Closes #24149. #24413 (Kseniia Sumarokova).
- Fix IPv6 addresses resolving (i.e. fixes
select * from remote('[::1]', system.one)
). #24319 (Azat Khuzhin). - Fix trailing whitespaces in FROM clause with subqueries in multiline mode, and also changes the output of the queries slightly in a more human friendly way. #24151 (Azat Khuzhin).
- Improvement for Distributed tables. Add ability to split distributed batch on failures (i.e. due to memory limits, corruptions), under
distributed_directory_monitor_split_batch_on_failure
(OFF by default). #23864 (Azat Khuzhin). - Handle column name clashes for
Join
table engine. Closes #20309. #23769 (Vladimir). - Display progress for
File
table engine inclickhouse-local
and on INSERT query inclickhouse-client
when data is passed to stdin. Closes #18209. #23656 (Kseniia Sumarokova). - Bugfixes and improvements of
clickhouse-copier
. Allow to copy tables with different (but compatible schemas). Closes #9159. Added test to copy ReplacingMergeTree. Closes #22711. Support TTL on columns and Data Skipping Indices. It simply removes it to create internal Distributed table (underlying table will have TTL and skipping indices). Closes #19384. Allow to copy MATERIALIZED and ALIAS columns. There are some cases in which it could be helpful (e.g. if this column is in PRIMARY KEY). Now it could be allowed by settingallow_to_copy_alias_and_materialized_columns
property to true in task configuration. Closes #9177. Closes [#11007] (ClickHouse#11007). Closes #9514. Added a propertyallow_to_drop_target_partitions
in task configuration to drop partition in original table before moving helping tables. Closes #20957. Get rid ofOPTIMIZE DEDUPLICATE
query. This hack was needed, becauseALTER TABLE MOVE PARTITION
was retried many times and plain MergeTree tables don't have deduplication. Closes #17966. Write progress to ZooKeeper node on pathtask_path + /status
in JSON format. Closes #20955. Support for ReplicatedTables without arguments. Closes #24834 .#23518 (Nikita Mikhaylov). - Added sleep with backoff between read retries from S3. #23461 (Vladimir Chebotarev).
- Respect
insert_allow_materialized_columns
(allows materialized columns) for INSERT intoDistributed
table. #23349 (Azat Khuzhin). - Add ability to push down LIMIT for distributed queries. #23027 (Azat Khuzhin).
- Fix zero-copy replication with several S3 volumes (Fixes #22679). #22864 (ianton-ru).
- Resolve the actual port number bound when a user requests any available port from the operating system to show it in the log message. #25569 (bnaecker).
- Fixed case, when sometimes conversion of postgres arrays resulted in String data type, not n-dimensional array, because
attndims
works incorrectly in some cases. Closes #24804. #25538 (Kseniia Sumarokova). - Fix convertion of DateTime with timezone for MySQL, PostgreSQL, ODBC. Closes #5057. #25528 (Kseniia Sumarokova).
- Distinguish KILL MUTATION for different tables (fixes unexpected
Cancelled mutating parts
error). #25025 (Azat Khuzhin). - Allow to declare S3 disk at root of bucket (S3 virtual filesystem is an experimental feature under development). #24898 (Vladimir Chebotarev).
- Enable reading of subcolumns (e.g. components of Tuples) for distributed tables. #24472 (Anton Popov).
- A feature for MySQL compatibility protocol: make
user
function to return correct output. Closes #25697. #25697 (sundyli).
- Improvement for backward compatibility. Use old modulo function version when used in partition key. Closes #23508. #24157 (Kseniia Sumarokova).
- Fix extremely rare bug on low-memory servers which can lead to the inability to perform merges without restart. Possibly fixes #24603. #24872 (alesapin).
- Fix extremely rare error
Tagging already tagged part
in replication queue during concurrentalter move/replace partition
. Possibly fixes #22142. #24961 (alesapin). - Fix potential crash when calculating aggregate function states by aggregation of aggregate function states of other aggregate functions (not a practical use case). See #24523. #25015 (alexey-milovidov).
- Fixed the behavior when query
SYSTEM RESTART REPLICA
orSYSTEM SYNC REPLICA
does not finish. This was detected on server with extremely low amount of RAM. #24457 (Nikita Mikhaylov). - Fix bug which can lead to ZooKeeper client hung inside clickhouse-server. #24721 (alesapin).
- If ZooKeeper connection was lost and replica was cloned after restoring the connection, its replication queue might contain outdated entries. Fixed failed assertion when replication queue contains intersecting virtual parts. It may rarely happen if some data part was lost. Print error in log instead of terminating. #24777 (tavplubix).
- Fix lost
WHERE
condition in expression-push-down optimization of query plan (settingquery_plan_filter_push_down = 1
by default). Fixes #25368. #25370 (Nikolai Kochetov). - Fix bug which can lead to intersecting parts after merges with TTL:
Part all_40_40_0 is covered by all_40_40_1 but should be merged into all_40_41_1. This shouldn't happen often.
. #25549 (alesapin). - On ZooKeeper connection loss
ReplicatedMergeTree
table might wait for background operations to complete before trying to reconnect. It's fixed, now background operations are stopped forcefully. #25306 (tavplubix). - Fix error
Key expression contains comparison between inconvertible types
for queries withARRAY JOIN
in case if array is used in primary key. Fixes #8247. #25546 (Anton Popov). - Fix wrong totals for query
WITH TOTALS
andWITH FILL
. Fixes #20872. #25539 (Anton Popov). - Fix data race when querying
system.clusters
while reloading the cluster configuration at the same time. #25737 (Amos Bird). - Fixed
No such file or directory
error on movingDistributed
table between databases. Fixes #24971. #25667 (tavplubix). REPLACE PARTITION
might be ignored in rare cases if the source partition was empty. It's fixed. Fixes #24869. #25665 (tavplubix).- Fixed a bug in
Replicated
database engine that might rarely cause some replica to skip enqueued DDL query. #24805 (tavplubix). - Fix null pointer dereference in
EXPLAIN AST
without query. #25631 (Nikolai Kochetov). - Fix waiting of automatic dropping of empty parts. It could lead to full filling of background pool and stuck of replication. #23315 (Anton Popov).
- Fix restore of a table stored in S3 virtual filesystem (it is an experimental feature not ready for production). #25601 (ianton-ru).
- Fix nullptr dereference in
Arrow
format when usingDecimal256
. AddDecimal256
support forArrow
format. #25531 (Kruglov Pavel). - Fix excessive underscore before the names of the preprocessed configuration files. #25431 (Vitaly Baranov).
- A fix for
clickhouse-copier
tool: Fix segfault when sharding_key is absent in task config for copier. #25419 (Nikita Mikhaylov). - Fix
REPLACE
column transformer when used in DDL by correctly quoting the formated query. This fixes #23925. #25391 (Amos Bird). - Fix the possibility of non-deterministic behaviour of the
quantileDeterministic
function and similar. This closes #20480. #25313 (alexey-milovidov). - Support
SimpleAggregateFunction(LowCardinality)
forSummingMergeTree
. Fixes #25134. #25300 (Nikolai Kochetov). - Fix logical error with exception message "Cannot sum Array/Tuple in min/maxMap". #25298 (Kruglov Pavel).
- Fix error
Bad cast from type DB::ColumnLowCardinality to DB::ColumnVector<char8_t>
for queries whereLowCardinality
argument was used for IN (this bug appeared in 21.6). Fixes #25187. #25290 (Nikolai Kochetov). - Fix incorrect behaviour of
joinGetOrNull
with not-nullable columns. This fixes #24261. #25288 (Amos Bird). - Fix incorrect behaviour and UBSan report in big integers. In previous versions
CAST(1e19 AS UInt128)
returned zero. #25279 (alexey-milovidov). - Fixed an error which occurred while inserting a subset of columns using CSVWithNames format. Fixes #25129. #25169 (Nikita Mikhaylov).
- Do not use table's projection for
SELECT
withFINAL
. It is not supported yet. #25163 (Amos Bird). - Fix possible parts loss after updating up to 21.5 in case table used
UUID
in partition key. (It is not recommended to useUUID
in partition key). Fixes #25070. #25127 (Nikolai Kochetov). - Fix crash in query with cross join and
joined_subquery_requires_alias = 0
. Fixes #24011. #25082 (Nikolai Kochetov). - Fix bug with constant maps in mapContains function that lead to error
empty column was returned by function mapContains
. Closes #25077. #25080 (Kruglov Pavel). - Remove possibility to create tables with columns referencing themselves like
a UInt32 ALIAS a + 1
orb UInt32 MATERIALIZED b
. Fixes #24910, #24292. #25059 (alesapin). - Fix wrong result when using aggregate projection with not empty
GROUP BY
key to execute query withGROUP BY
by empty key. #25055 (Amos Bird). - Fix serialization of splitted nested messages in Protobuf format. This PR fixes #24647. #25000 (Vitaly Baranov).
- Fix limit/offset settings for distributed queries (ignore on the remote nodes). #24940 (Azat Khuzhin).
- Fix possible heap-buffer-overflow in
Arrow
format. #24922 (Kruglov Pavel). - Fixed possible error 'Cannot read from istream at offset 0' when reading a file from DiskS3 (S3 virtual filesystem is an experimental feature under development that should not be used in production). #24885 (Pavel Kovalenko).
- Fix "Missing columns" exception when joining Distributed Materialized View. #24870 (Azat Khuzhin).
- Allow
NULL
values in postgresql compatibility protocol. Closes #22622. #24857 (Kseniia Sumarokova). - Fix bug when exception
Mutation was killed
can be thrown to the client on mutation wait when mutation not loaded into memory yet. #24809 (alesapin). - Fixed bug in deserialization of random generator state with might cause some data types such as
AggregateFunction(groupArraySample(N), T))
to behave in a non-deterministic way. #24538 (tavplubix). - Disallow building uniqXXXXStates of other aggregation states. #24523 (Raúl Marín). Then allow it back by actually eliminating the root cause of the related issue. (alexey-milovidov).
- Fix usage of tuples in
CREATE .. AS SELECT
queries. #24464 (Anton Popov). - Fix computation of total bytes in
Buffer
table. In current ClickHouse version total_writes.bytes counter decreases too much during the buffer flush. It leads to counter overflow and totalBytes return something around 17.44 EB some time after the flush. #24450 (DimasKovas). - Fix incorrect information about the monotonicity of toWeek function. This fixes #24422 . This bug was introduced in ClickHouse#5212 , and was exposed later by smarter partition pruner. #24446 (Amos Bird).
- When user authentication is managed by LDAP. Fixed potential deadlock that can happen during LDAP role (re)mapping, when LDAP group is mapped to a nonexistent local role. #24431 (Denis Glazachev).
- In "multipart/form-data" message consider the CRLF preceding a boundary as part of it. Fixes #23905. #24399 (Ivan).
- Fix drop partition with intersect fake parts. In rare cases there might be parts with mutation version greater than current block number. #24321 (Amos Bird).
- Fixed a bug in moving Materialized View from Ordinary to Atomic database (
RENAME TABLE
query). Now inner table is moved to new database together with Materialized View. Fixes #23926. #24309 (tavplubix). - Allow empty HTTP headers. Fixes #23901. #24285 (Ivan).
- Correct processing of mutations (ALTER UPDATE/DELETE) in Memory tables. Closes #24274. #24275 (flynn).
- Make column LowCardinality property in JOIN output the same as in the input, close #23351, close #20315. #24061 (Vladimir).
- A fix for Kafka tables. Fix the bug in failover behavior when Engine = Kafka was not able to start consumption if the same consumer had an empty assignment previously. Closes #21118. #21267 (filimonov).
- Add
darwin-aarch64
(Mac M1 / Apple Silicon) builds in CI #25560 (Ivan) and put the links to the docs and website (alexey-milovidov). - Adds cross-platform embedding of binary resources into executables. It works on Illumos. #25146 (bnaecker).
- Add join related options to stress tests to improve fuzzing. #25200 (Vladimir).
- Enable build with s3 module in osx #25217. #25218 (kevin wan).
- Add integration test cases to cover JDBC bridge. #25047 (Zhichun Wu).
- Integration tests configuration has special treatment for dictionaries. Removed remaining dictionaries manual setup. #24728 (Ilya Yatsishin).
- Add libfuzzer tests for YAMLParser class. #24480 (BoloniniD).
- Ubuntu 20.04 is now used to run integration tests, docker-compose version used to run integration tests is updated to 1.28.2. Environment variables now take effect on docker-compose. Rework test_dictionaries_all_layouts_separate_sources to allow parallel run. #20393 (Ilya Yatsishin).
- Fix TOCTOU error in installation script. #25277 (alexey-milovidov).
zstd
compression library is updated to v1.5.0. You may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour.- The setting
compile_expressions
is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off. - Values of
UUID
type cannot be compared with integer. For example, instead of writinguuid != 0
typeuuid != '00000000-0000-0000-0000-000000000000'
.
- Add Postgres-like cast operator (
::
). E.g.:[1, 2]::Array(UInt8)
,0.1::Decimal(4, 4)
,number::UInt16
. #23871 (Anton Popov). - 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. 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). - Support
Array
data type for inserting and selecting data inArrow
,Parquet
andORC
formats. #21770 (taylor12805). - Implement table comments. Closes #23225. #23548 (flynn).
- Support creating dictionaries with DDL queries in
clickhouse-local
. Closes #22354. Added support forDETACH DICTIONARY PERMANENTLY
. Added support forEXCHANGE DICTIONARIES
forAtomic
database engine. Added support for moving dictionaries between databases usingRENAME DICTIONARY
. #23436 (Maksim Kita). - Add aggregate function
uniqTheta
to support Theta Sketch in ClickHouse. #23894. #22609 (Ping Yu). - Add function
splitByRegexp
. #24077 (abel-cheng). - Add 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
thread_name
column insystem.stack_trace
. This closes #23256. #24124 (abel-cheng). - If
insert_null_as_default
= 1, insert default values instead of NULL inINSERT ... SELECT
andINSERT ... SELECT ... UNION ALL ...
queries. Closes #22832. #23524 (Kseniia Sumarokova). - Add support for progress indication in
clickhouse-local
with--progress
option. #23196 (Egor Savin). - Add support for HTTP compression (determined by
Content-Encoding
HTTP header) inhttp
dictionary source. This fixes #8912. #23946 (FArthur-cmd). - Added
SYSTEM QUERY RELOAD MODEL
,SYSTEM QUERY RELOAD MODELS
. Closes #18722. #23182 (Maksim Kita). - 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). - 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). - LDAP: implemented user DN detection functionality to use when mapping Active Directory groups to ClickHouse roles. #22228 (Denis Glazachev).
- New aggregate function
deltaSumTimestamp
for summing the difference between consecutive rows while maintaining ordering during merge by storing timestamps. #21888 (Russ Frank). - Added less secure IMDS credentials provider for S3 which works under docker correctly. #21852 (Vladimir Chebotarev).
- Add back
indexHint
function. This is for #21238. This reverts #9542. This fixes #9540. #21304 (Amos Bird).
- 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 (Maksim Kita, alexey-milovidov). Note: if you feel in trouble, turn this option off. - Update
re2
library. Performance of regular expressions matching is improved. Also this PR adds compatibility with gcc-11. #24196 (Raúl Marín). - 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).
- Fusion of aggregate functions
sum
,count
andavg
in a query into single aggregate function. The optimization is controlled with theoptimize_fuse_sum_count_avg
setting. This is implemented with a new aggregate functionsumCount
. This function returns a tuple of two fields:sum
andcount
. #21337 (hexiaoting). - Update
zstd
to v1.5.0. The performance of compression is improved for single digits percentage. #24135 (Raúl Marín). Note: you may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. - Improved performance of
Buffer
tables: do not acquire lock for total_bytes/total_rows forBuffer
engine. #24066 (Azat Khuzhin). - Preallocate support for hashed/sparse_hashed dictionaries is returned. #23979 (Azat Khuzhin).
- Enable
async_socket_for_remote
by default (lower amount of threads in querying Distributed tables with large fanout). #23683 (Nikolai Kochetov).
- 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). - Added
region
parameter for S3 storage and disk. #23846 (Vladimir Chebotarev). - Allow configuring different log levels for different logging channels. Closes #19569. #23857 (filimonov).
- Keep default timezone on
DateTime
operations if it was not provided explicitly. For example, if you add one second to a value ofDateTime
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). - 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). - 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). - Function
now64
now supports optional timezone argument. #24091 (Vasily Nemkov). - 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).
- Fix crash when memory allocation fails in simdjson. simdjson/simdjson#1567 . Mark as improvement because it's a very rare bug. #24147 (Amos Bird).
- Preserve dictionaries until storage shutdown (this will avoid possible
external dictionary 'DICT' not found
errors at server shutdown during final flush of theBuffer
engine). #24068 (Azat Khuzhin). - Flush
Buffer
tables before shutting down tables (within one database), to avoid discarding blocks due to underlying table had been already detached (andDestination table default.a_data_01870 doesn't exist. Block of data is discarded
error in the log). #24067 (Azat Khuzhin). - Now
prefer_column_name_to_alias = 1
will also favor column names forgroup by
,having
andorder by
. This fixes #23882. #24022 (Amos Bird). - Add support for
ORDER BY WITH FILL
withDateTime64
. #24016 (kevin wan). - Enable
DateTime64
to be a version column inReplacingMergeTree
. #23992 (kevin wan). - Log information about OS name, kernel version and CPU architecture on server startup. #23988 (Azat Khuzhin).
- Support specifying table schema for
postgresql
dictionary source. Closes #23958. #23980 (Kseniia Sumarokova). - Add hints for names of
Enum
elements (suggest names in case of typos). Closes #17112. #23919 (flynn). - Measure found rate (the percentage for which the value was found) for dictionaries (see
found_rate
insystem.dictionaries
). #23916 (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). - 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). - Querying
system.tables
does not go to ZooKeeper anymore. #23793 (Fuwang Hu). - Respect
lock_acquire_timeout_for_background_operations
forOPTIMIZE
queries. #23623 (Azat Khuzhin). - Possibility to change
S3
disk settings in runtime via newSYSTEM RESTART DISK
SQL command. #23429 (Pavel Kovalenko). - 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). - Disable
min_bytes_to_use_mmap_io
by default. #23322 (Azat Khuzhin). - Support
LowCardinality
nullability withjoin_use_nulls
, close #15101. #23237 (vdimir). - Added possibility to restore
MergeTree
parts todetached
directory forS3
disk. #23112 (Pavel Kovalenko). - Retries on HTTP connection drops in S3. #22988 (Vladimir Chebotarev).
- 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). MaterializeMySQL
(experimental feature): Previously, MySQL 5.7.9 was not supported due to SQL incompatibility. Now leave MySQL parameter verification to the MaterializeMySQL. #23413 (TCeason).- Enable reading of subcolumns for distributed tables. #24472 (Anton Popov).
- Fix usage of tuples in
CREATE .. AS SELECT
queries. #24464 (Anton Popov). - Support for
Parquet
format inKafka
tables. #23412 (Chao Ma).
- Use old modulo function version when used in partition key and primary key. Closes #23508. #24157 (Kseniia Sumarokova). It was a source of backward incompatibility in previous releases.
- Fixed the behavior when query
SYSTEM RESTART REPLICA
orSYSTEM SYNC REPLICA
is being processed infinitely. This was detected on server with extremely little amount of RAM. #24457 (Nikita Mikhaylov). - Fix incorrect monotonicity of
toWeek
function. This fixes #24422 . This bug was introduced in #5212, and was exposed later by smarter partition pruner. #24446 (Amos Bird). - Fix drop partition with intersect fake parts. In rare cases there might be parts with mutation version greater than current block number. #24321 (Amos Bird).
- Fixed a bug in moving Materialized View from Ordinary to Atomic database (
RENAME TABLE
query). Now inner table is moved to new database together with Materialized View. Fixes #23926. #24309 (tavplubix). - Allow empty HTTP headers in client requests. Fixes #23901. #24285 (Ivan).
- Set
max_threads = 1
to fix mutation fail ofMemory
tables. Closes #24274. #24275 (flynn). - Fix typo in implementation of
Memory
tables, this bug was introduced at #15127. Closes #24192. #24193 (张中南). - Fix abnormal server termination due to
HDFS
becoming not accessible during query execution. Closes #24117. #24191 (Kseniia Sumarokova). - Fix crash on updating of
Nested
column with const condition. #24183 (hexiaoting). - Fix race condition which could happen in RBAC under a heavy load. This PR fixes #24090, #24134,. #24176 (Vitaly Baranov).
- 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 an issue:
EXPLAIN PIPELINE
withSELECT xxx FINAL
showed a wrong pipeline. (hexiaoting). - Fixed using const
DateTime
value vsDateTime64
column inWHERE
. #24100 (Vasily Nemkov). - Fix crash in merge JOIN, closes #24010. #24013 (vdimir).
- 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 (tavplubix). - 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). - Fix keys metrics accounting for
CACHE
dictionary with duplicates in the source (leads toDictCacheKeysRequestedMiss
overflows). #23929 (Azat Khuzhin). - Fix implementation of connection pool of
PostgreSQL
engine. Closes #23897. #23909 (Kseniia Sumarokova). - 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). - A fix for
s3
table function: better handling of HTTP errors. Response bodies of HTTP errors were being ignored earlier. #23844 (Vladimir Chebotarev). - A fix for
s3
table function: better handling of URI's. Fixed an incompatibility with URLs containing+
symbol, data with such keys could not be read previously. #23822 (Vladimir Chebotarev). - Fix error
Can't initialize pipeline with empty pipe
for queries withGLOBAL IN/JOIN
anduse_hedged_requests
. Fixes #23431. #23805 (Nikolai Kochetov). - Fix
CLEAR COLUMN
does not work when it is referenced by materialized view. Close #23764. #23781 (flynn). - Fix heap use after free when reading from HDFS if
Values
format is used. #23761 (Kseniia Sumarokova). - Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. #23744 (Azat Khuzhin).
- 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 (tavplubix). - Fix a bug with
Join
andWITH TOTALS
, close #17718. #23549 (vdimir). - Fix possible
Block structure mismatch
error for queries withUNION
which could possibly happen after filter-pushdown optimization. Fixes #23029. #23359 (Nikolai Kochetov). - Add type conversion when the setting
optimize_skip_unused_shards_rewrite_in
is enabled. This fixes MSan report. #23219 (Azat Khuzhin). - Add a missing check when updating nested subcolumns, close issue: #22353. #22503 (hexiaoting).
- Support building on Illumos. #24144. Adds support for building on Solaris-derived operating systems. #23746 (bnaecker).
- 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).
- Update librdkafka 1.6.0-RC3 to 1.6.1. #23874 (filimonov).
- Always enable
asynchronous-unwind-tables
explicitly. It may fix query profiler on AArch64. #23602 (alexey-milovidov). - Avoid possible build dependency on locale and filesystem order. This allows reproducible builds. #23600 (alexey-milovidov).
- 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).
- Add simple tool for benchmarking (Zoo)Keeper. #23038 (alesapin).
- Change comparison of integers and floating point numbers when integer is not exactly representable in the floating point data type. In new version comparison will return false as the rounding error will occur. Example:
9223372036854775808.0 != 9223372036854775808
, because the number9223372036854775808
is not representable as floating point number exactly (and9223372036854775808.0
is rounded to9223372036854776000.0
). But in previous version the comparison will return as the numbers are equal, because if the floating point number9223372036854776000.0
get converted back to UInt64, it will yield9223372036854775808
. For the reference, the Python programming language also treats these numbers as equal. But this behaviour was dependend on CPU model (different results on AMD64 and AArch64 for some out-of-range numbers), so we make the comparison more precise. It will treat int and float numbers equal only if int is represented in floating point type exactly. #22595 (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).
- Added functions
dictGetChildren(dictionary, key)
,dictGetDescendants(dictionary, key, level)
. FunctiondictGetChildren
return all children as an array if indexes. It is a inverse transformation fordictGetHierarchy
. FunctiondictGetDescendants
return all descendants as ifdictGetChildren
was appliedlevel
times recursively. Zerolevel
value is equivalent to infinity. Improved performance ofdictGetHierarchy
,dictIsIn
functions. Closes #14656. #22096 (Maksim Kita). - Added function
dictGetOrNull
. It works likedictGet
, but returnNull
in case key was not found in dictionary. Closes #22375. #22413 (Maksim Kita). - Added a table function
s3Cluster
, which allows to process files froms3
in parallel on every node of a specified cluster. #22012 (Nikita Mikhaylov). - Added support for replicas and shards in MySQL/PostgreSQL table engine / table function. You can write
SELECT * FROM mysql('host{1,2}-{1|2}', ...)
. Closes #20969. #22217 (Kseniia Sumarokova). - Added
ALTER TABLE ... FETCH PART ...
query. It's similar toFETCH PARTITION
, but fetches only one part. #22706 (turbo jason). - Added a setting
max_distributed_depth
that limits the depth of recursive queries toDistributed
tables. Closes #20229. #21942 (flynn).
- Improved performance of
intDiv
by dynamic dispatch for AVX2. This closes #22314. #23000 (alexey-milovidov). - Improved performance of reading from
ArrowStream
input format for sources other then local file (e.g. URL). #22673 (nvartolomei). - Disabled compression by default when interacting with localhost (with clickhouse-client or server to server with distributed queries) via native protocol. It may improve performance of some import/export operations. This closes #22234. #22237 (alexey-milovidov).
- Exclude values that does not belong to the shard from right part of IN section for distributed queries (under
optimize_skip_unused_shards_rewrite_in
, enabled by default, since it still requiresoptimize_skip_unused_shards
). #21511 (Azat Khuzhin). - Improved performance of reading a subset of columns with File-like table engine and column-oriented format like Parquet, Arrow or ORC. This closes #issue:20129. #21302 (keenwolf).
- Allow to move more conditions to
PREWHERE
as it was before version 21.1 (adjustment of internal heuristics). Insufficient number of moved condtions could lead to worse performance. #23397 (Anton Popov). - Improved performance of ODBC connections and fixed all the outstanding issues from the backlog. Using
nanodbc
library instead ofPoco::ODBC
. Closes #9678. Add support for DateTime64 and Decimal* for ODBC table engine. Closes #21961. Fixed issue with cyrillic text being truncated. Closes #16246. Added connection pools for odbc bridge. #21972 (Kseniia Sumarokova).
- Increase
max_uri_size
(the maximum size of URL in HTTP interface) to 1 MiB by default. This closes #21197. #22997 (alexey-milovidov). - Set
background_fetches_pool_size
to8
that is better for production usage with frequent small insertions or slow ZooKeeper cluster. #22945 (alexey-milovidov). - FlatDictionary added
initial_array_size
,max_array_size
options. #22521 (Maksim Kita). - Add new setting
non_replicated_deduplication_window
for non-replicated MergeTree inserts deduplication. #22514 (alesapin). - Update paths to the
CatBoost
model configs in config reloading. #22434 (Kruglov Pavel). - Added
Decimal256
type support in dictionaries.Decimal256
is experimental feature. Closes #20979. #22960 (Maksim Kita). - Enabled
async_socket_for_remote
by default (using less amount of OS threads for distributed queries). #23683 (Nikolai Kochetov). - 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). - Make function name
unhex
case insensitive for compatibility with MySQL. #23229 (alexey-milovidov). - Implement functions
arrayHasAny
,arrayHasAll
,has
,indexOf
,countEqual
for generic case when types of array elements are different. In previous versions the functionsarrayHasAny
,arrayHasAll
returned false andhas
,indexOf
,countEqual
thrown exception. Also add support forDecimal
and big integer types in functionshas
and similar. This closes #20272. #23044 (alexey-milovidov). - Raised the threshold on max number of matches in result of the function
extractAllGroupsHorizontal
. #23036 (Vasily Nemkov). - Do not perform
optimize_skip_unused_shards
for cluster with one node. #22999 (Azat Khuzhin). - Added ability to run clickhouse-keeper (experimental drop-in replacement to ZooKeeper) with SSL. Config settings
keeper_server.tcp_port_secure
can be used for secure interaction between client and keeper-server.keeper_server.raft_configuration.secure
can be used to enable internal secure communication between nodes. #22992 (alesapin). - Added ability to flush buffer only in background for
Buffer
tables. #22986 (Azat Khuzhin). - When selecting from MergeTree table with NULL in WHERE condition, in rare cases, exception was thrown. This closes #20019. #22978 (alexey-milovidov).
- Fix error handling in Poco HTTP Client for AWS. #22973 (kreuzerkrieg).
- Respect
max_part_removal_threads
forReplicatedMergeTree
. #22971 (Azat Khuzhin). - Fix obscure corner case of MergeTree settings inactive_parts_to_throw_insert = 0 with inactive_parts_to_delay_insert > 0. #22947 (Azat Khuzhin).
dateDiff
now works withDateTime64
arguments (even for values outside ofDateTime
range) #22931 (Vasily Nemkov).- MaterializeMySQL (experimental feature): added an ability to replicate MySQL databases containing views without failing. This is accomplished by ignoring the views. #22760 (Christian).
- Allow RBAC row policy via postgresql protocol. Closes #22658. PostgreSQL protocol is enabled in configuration by default. #22755 (Kseniia Sumarokova).
- Add metric to track how much time is spend during waiting for Buffer layer lock. #22725 (Azat Khuzhin).
- Allow to use CTE in VIEW definition. This closes #22491. #22657 (Amos Bird).
- Clear the rest of the screen and show cursor in
clickhouse-client
if previous program has left garbage in terminal. This closes #16518. #22634 (alexey-milovidov). - Make
round
function to behave consistently on non-x86_64 platforms. Rounding half to nearest even (Banker's rounding) is used. #22582 (alexey-milovidov). - Correctly check structure of blocks of data that are sending by Distributed tables. #22325 (Azat Khuzhin).
- Allow publishing Kafka errors to a virtual column of Kafka engine, controlled by the
kafka_handle_error_mode
setting. #21850 (fastio). - Add aliases
simpleJSONExtract/simpleJSONHas
tovisitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}
. Fixes #21383. #21519 (fastio). - Add
clickhouse-library-bridge
for library dictionary source. Closes #9502. #21509 (Kseniia Sumarokova). - Forbid to drop a column if it's referenced by materialized view. Closes #21164. #21303 (flynn).
- Support dynamic interserver credentials (rotating credentials without downtime). #14113 (johnskopis).
- Add support for Kafka storage with
Arrow
andArrowStream
format messages. #23415 (Chao Ma). - Fixed missing semicolon in exception message. The user may find this exception message unpleasant to read. #23208 (alexey-milovidov).
- Fixed missing whitespace in some exception messages about
LowCardinality
type. #23207 (alexey-milovidov). - Some values were formatted with alignment in center in table cells in
Markdown
format. Not anymore. #23096 (alexey-milovidov). - Remove non-essential details from suggestions in clickhouse-client. This closes #22158. #23040 (alexey-milovidov).
- Correct calculation of
bytes_allocated
field in system.dictionaries for sparse_hashed dictionaries. #22867 (Azat Khuzhin). - Fixed approximate total rows accounting for reverse reading from MergeTree. #22726 (Azat Khuzhin).
- Fix the case when it was possible to configure dictionary with clickhouse source that was looking to itself that leads to infinite loop. Closes #14314. #22479 (Maksim Kita).
- Multiple fixes for hedged requests. Fixed an error
Can't initialize pipeline with empty pipe
for queries withGLOBAL IN/JOIN
when the settinguse_hedged_requests
is enabled. Fixes #23431. #23805 (Nikolai Kochetov). Fixed a race condition in hedged connections which leads to crash. This fixes #22161. #22443 (Kruglov Pavel). Fix possible crash in case ifunknown packet
was received from remote query (withasync_socket_for_remote
enabled). Fixes #21167. #23309 (Nikolai Kochetov). - 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). - Fixed remote JDBC bridge timeout connection issue. Closes #9609. #23771 (Maksim Kita, alexey-milovidov).
- Fix the logic of initial load of
complex_key_hashed
ifupdate_field
is specified. Closes #23800. #23824 (Maksim Kita). - Fixed crash when
PREWHERE
and row policy filter are both in effect with empty result. #23763 (Amos Bird). - Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. #23744 (Azat Khuzhin).
- Added an exception in case of completely the same values in both samples in aggregate function
mannWhitneyUTest
. This fixes #23646. #23654 (Nikita Mikhaylov). - Fixed server fault when inserting data through HTTP caused an exception. This fixes #23512. #23643 (Nikita Mikhaylov).
- Fixed misinterpretation of some
LIKE
expressions with escape sequences. #23610 (alexey-milovidov). - Fixed restart / stop command hanging. Closes #20214. #23552 (filimonov).
- Fixed
COLUMNS
matcher in case of multiple JOINs in select query. Closes #22736. #23501 (Maksim Kita). - Fixed a crash when modifying column's default value when a column itself is used as
ReplacingMergeTree
's parameter. #23483 (hexiaoting). - Fixed 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). - Fixed bug that does not allow cast from empty array literal, to array with dimensions greater than 1, e.g.
CAST([] AS Array(Array(String)))
. Closes #14476. #23456 (Maksim Kita). - Fixed a bug when
deltaSum
aggregate function produced incorrect result after resetting the counter. #23437 (Russ Frank). - Fixed
Cannot unlink file
error on unsuccessful creation of ReplicatedMergeTree table with multidisk configuration. This closes #21755. #23433 (tavplubix). - Fixed incompatible constant expression generation during partition pruning based on virtual columns. This fixes ClickHouse#21401 (comment). #23366 (Amos Bird).
- Fixed a crash when setting join_algorithm is set to 'auto' and Join is performed with a Dictionary. Close #23002. #23312 (Vladimir).
- Don't relax NOT conditions during partition pruning. This fixes #23305 and #21539. #23310 (Amos Bird).
- 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 (tavplubix).
- 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 (tavplubix). - Fixed simple key dictionary from DDL creation if primary key is not first attribute. Fixes #23236. #23262 (Maksim Kita).
- Fixed reading from ODBC when there are many long column names in a table. Closes #8853. #23215 (Kseniia Sumarokova).
- MaterializeMySQL (experimental feature): fixed
Not found column
error when selecting fromMaterializeMySQL
with condition on key column. Fixes #22432. #23200 (tavplubix). - Correct aliases handling if subquery was optimized to constant. Fixes #22924. Fixes #10401. #23191 (Maksim Kita).
- Server might fail to start if
data_type_default_nullable
setting is enabled in default profile, it's fixed. Fixes #22573. #23185 (tavplubix). - Fixed a crash on shutdown which happened because of wrong accounting of current connections. #23154 (Vitaly Baranov).
- Fixed
Table .inner_id... doesn't exist
error when selecting from Materialized View after detaching it from Atomic database and attaching back. #23047 (tavplubix). - Fix error
Cannot find column in ActionsDAG result
which may happen if subquery usesuntuple
. Fixes #22290. #22991 (Nikolai Kochetov). - Fix usage of constant columns of type
Map
with nullable values. #22939 (Anton Popov). - fixed
formatDateTime()
onDateTime64
and "%C" format specifier fixedtoDateTime64()
for large values and non-zero scale. #22937 (Vasily Nemkov). - Fixed a crash when using
mannWhitneyUTest
andrankCorr
with window functions. This fixes #22728. #22876 (Nikita Mikhaylov). - LIVE VIEW (experimental feature): fixed possible hanging in concurrent DROP/CREATE of TEMPORARY LIVE VIEW in
TemporaryLiveViewCleaner
, see. #22858 (Vitaly Baranov). - Fixed pushdown of
HAVING
in case, when filter column is used in aggregation. #22763 (Anton Popov). - Fixed possible hangs in Zookeeper requests in case of OOM exception. Fixes #22438. #22684 (Nikolai Kochetov).
- Fixed wait for mutations on several replicas for ReplicatedMergeTree table engines. Previously, mutation/alter query may finish before mutation actually executed on other replicas. #22669 (alesapin).
- Fixed exception for Log with nested types without columns in the SELECT clause. #22654 (Azat Khuzhin).
- Fix unlimited wait for auxiliary AWS requests. #22594 (Vladimir Chebotarev).
- Fixed a crash when client closes connection very early #22579. #22591 (nvartolomei).
Map
data type (experimental feature): fixed an incorrect formatting of functionmap
in distributed queries. #22588 (foolchi).- Fixed deserialization of empty string without newline at end of TSV format. This closes #20244. Possible workaround without version update: set
input_format_null_as_default
to zero. It was zero in old versions. #22527 (alexey-milovidov). - Fixed wrong cast of a column of
LowCardinality
type in Merge Join algorithm. Close #22386, close #22388. #22510 (Vladimir). - Buffer overflow (on read) was possible in
tokenbf_v1
full text index. The excessive bytes are not used but the read operation may lead to crash in rare cases. This closes #19233. #22421 (alexey-milovidov). - Do not limit HTTP chunk size. Fixes #21907. #22322 (Ivan).
- Fixed a bug, which leads to underaggregation of data in case of enabled
optimize_aggregation_in_order
and many parts in table. Slightly improve performance of aggregation with enabledoptimize_aggregation_in_order
. #21889 (Anton Popov). - Check if table function view is used as a column. This complements #20350. #21465 (Amos Bird).
- Fix "unknown column" error for tables with
Merge
engine in queris withJOIN
and aggregation. Closes #18368, close #22226. #21370 (Vladimir). - Fixed name clashes in pushdown optimization. It caused incorrect
WHERE
filtration after FULL JOIN. Close #20497. #20622 (Vladimir). - Fixed very rare bug when quorum insert with
quorum_parallel=1
is not really "quorum" because of deduplication. #18215 (filimonov - reported, alesapin - fixed).
- Run stateless tests in parallel in CI. #22300 (alesapin).
- Simplify debian packages. This fixes #21698. #22976 (alexey-milovidov).
- Added support for ClickHouse build on Apple M1. #21639 (changvvb).
- Fixed ClickHouse Keeper build for MacOS. #22860 (alesapin).
- Fixed some tests on AArch64 platform. #22596 (alexey-milovidov).
- Added function alignment for possibly better performance. #21431 (Danila Kutenin).
- Adjust some tests to output identical results on amd64 and aarch64 (qemu). The result was depending on implementation specific CPU behaviour. #22590 (alexey-milovidov).
- Allow query profiling only on x86_64. See #15174 and #15638. This closes #15638. #22580 (alexey-milovidov).
- Allow building with unbundled xz (lzma) using
USE_INTERNAL_XZ_LIBRARY=OFF
CMake option. #22571 (Kfir Itzhak). - Enable bundled
openldap
onppc64le
#22487 (Kfir Itzhak). - Disable incompatible libraries (platform specific typically) on
ppc64le
#22475 (Kfir Itzhak). - Add Jepsen test in CI for clickhouse Keeper. #22373 (alesapin).
- Build
jemalloc
with support for heap profiling. #22834 (nvartolomei). - Avoid UB in
*Log
engines for rwlock unlock due to unlock from another thread. #22583 (Azat Khuzhin). - Fixed UB by unlocking the rwlock of the TinyLog from the same thread. #22560 (Azat Khuzhin).
- The
toStartOfIntervalFunction
will align hour intervals to the midnight (in previous versions they were aligned to the start of unix epoch). For example,toStartOfInterval(x, INTERVAL 11 HOUR)
will split every day into three intervals:00:00:00..10:59:59
,11:00:00..21:59:59
and22:00:00..23:59:59
. This behaviour is more suited for practical needs. This closes #9510. #22060 (alexey-milovidov). Age
andPrecision
in graphite rollup configs should increase from retention to retention. Now it's checked and the wrong config raises an exception. #21496 (Mikhail f. Shiryaev).- Fix
cutToFirstSignificantSubdomainCustom()
/firstSignificantSubdomainCustom()
returning wrong result for 3+ level domains present in custom top-level domain list. For input domains matching these custom top-level domains, the third-level domain was considered to be the first significant one. This is now fixed. This change may introduce incompatibility if the function is used in e.g. the sharding key. #21946 (Azat Khuzhin). - Column
keys
in tablesystem.dictionaries
was replaced to columnskey.names
andkey.types
. Columnskey.names
,key.types
,attribute.names
,attribute.types
fromsystem.dictionaries
table does not require dictionary to be loaded. #21884 (Maksim Kita). - Now replicas that are processing the
ALTER TABLE ATTACH PART[ITION]
command search in theirdetached/
folders before fetching the data from other replicas. As an implementation detail, a new commandATTACH_PART
is introduced in the replicated log. Parts are searched and compared by their checksums. #18978 (Mike Kot). Note:ATTACH PART[ITION]
queries may not work during cluster upgrade.- It's not possible to rollback to older ClickHouse version after executing
ALTER ... ATTACH
query in new version as the old servers would fail to pass theATTACH_PART
entry in the replicated log.
- In this version, empty
<remote_url_allow_hosts></remote_url_allow_hosts>
will block all access to remote hosts while in previous versions it did nothing. If you want to keep old behaviour and you have emptyremote_url_allow_hosts
element in configuration file, remove it. #20058 (Vladimir Chebotarev).
- Extended range of
DateTime64
to support dates from year 1925 to 2283. Improved support ofDateTime
around zero date (1970-01-01
). #9404 (alexey-milovidov, Vasily Nemkov). Not every time and date functions are working for extended range of dates. - Added support of Kerberos authentication for preconfigured users and HTTP requests (GSS-SPNEGO). #14995 (Denis Glazachev).
- Add
prefer_column_name_to_alias
setting to use original column names instead of aliases. it is needed to be more compatible with common databases' aliasing rules. This is for #9715 and #9887. #22044 (Amos Bird). - Added functions
dictGetChildren(dictionary, key)
,dictGetDescendants(dictionary, key, level)
. FunctiondictGetChildren
return all children as an array if indexes. It is a inverse transformation fordictGetHierarchy
. FunctiondictGetDescendants
return all descendants as ifdictGetChildren
was appliedlevel
times recursively. Zerolevel
value is equivalent to infinity. Closes #14656. #22096 (Maksim Kita). - Added
executable_pool
dictionary source. Close #14528. #21321 (Maksim Kita). - Added table function
dictionary
. It works the same way asDictionary
engine. Closes #21560. #21910 (Maksim Kita). - Support
Nullable
type forPolygonDictionary
attribute. #21890 (Maksim Kita). - Functions
dictGet
,dictHas
use current database name if it is not specified for dictionaries created with DDL. Closes #21632. #21859 (Maksim Kita). - Added function
dictGetOrNull
. It works likedictGet
, but returnNull
in case key was not found in dictionary. Closes #22375. #22413 (Maksim Kita). - Added async update in
ComplexKeyCache
,SSDCache
,SSDComplexKeyCache
dictionaries. Added support forNullable
type inCache
,ComplexKeyCache
,SSDCache
,SSDComplexKeyCache
dictionaries. Added support for multiple attributes fetch withdictGet
,dictGetOrDefault
functions. Fixes #21517. #20595 (Maksim Kita). - Support
dictHas
function forRangeHashedDictionary
. Fixes #6680. #19816 (Maksim Kita). - Add function
timezoneOf
that returns the timezone name ofDateTime
orDateTime64
data types. This does not close #9959. Fix inconsistencies in function names: add aliasestimezone
andtimeZone
as well astoTimezone
andtoTimeZone
andtimezoneOf
andtimeZoneOf
. #22001 (alexey-milovidov). - Add new optional clause
GRANTEES
forCREATE/ALTER USER
commands. It specifies users or roles which are allowed to receive grants from this user on condition this user has also all required access granted with grant option. By defaultGRANTEES ANY
is used which means a user with grant option can grant to anyone. Syntax:CREATE USER ... GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]
. #21641 (Vitaly Baranov). - Add new column
slowdowns_count
tosystem.clusters
. When using hedged requests, it shows how many times we switched to another replica because this replica was responding slowly. Also show actual value oferrors_count
insystem.clusters
. #21480 (Kruglov Pavel). - Add
_partition_id
virtual column forMergeTree*
engines. Allow to prune partitions by_partition_id
. AddpartitionID()
function to calculate partition id string. #21401 (Amos Bird). - Add function
isIPAddressInRange
to test if an IPv4 or IPv6 address is contained in a given CIDR network prefix. #21329 (PHO). - Added new SQL command
ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'
. This command is needed to properly remove 'freezed' partitions from all disks. #21142 (Pavel Kovalenko). - Supports implicit key type conversion for JOIN. #19885 (Vladimir).
- Support
RANGE OFFSET
frame (for window functions) for floating point types. ImplementlagInFrame
/leadInFrame
window functions, which are analogous tolag
/lead
, but respect the window frame. They are identical when the frame isbetween unbounded preceding and unbounded following
. This closes #5485. #21895 (Alexander Kuzmenkov). - Zero-copy replication for
ReplicatedMergeTree
over S3 storage. #16240 (ianton-ru). - Added possibility to migrate existing S3 disk to the schema with backup-restore capabilities. #22070 (Pavel Kovalenko).
- Supported parallel formatting in
clickhouse-local
and everywhere else. #21630 (Nikita Mikhaylov). - Support parallel parsing for
CSVWithNames
andTSVWithNames
formats. This closes #21085. #21149 (Nikita Mikhaylov). - Enable read with mmap IO for file ranges from 64 MiB (the settings
min_bytes_to_use_mmap_io
). It may lead to moderate performance improvement. #22326 (alexey-milovidov). - Add cache for files read with
min_bytes_to_use_mmap_io
setting. It makes significant (2x and more) performance improvement when the value of the setting is small by avoiding frequent mmap/munmap calls and the consequent page faults. Note that mmap IO has major drawbacks that makes it less reliable in production (e.g. hung or SIGBUS on faulty disks; less controllable memory usage). Nevertheless it is good in benchmarks. #22206 (alexey-milovidov). - Avoid unnecessary data copy when using codec
NONE
. Please note that codecNONE
is mostly useless - it's recommended to always use compression (LZ4
is by default). Despite the common belief, disabling compression may not improve performance (the opposite effect is possible). TheNONE
codec is useful in some cases: - when data is uncompressable; - for synthetic benchmarks. #22145 (alexey-milovidov). - Faster
GROUP BY
with smallmax_rows_to_group_by
andgroup_by_overflow_mode='any'
. #21856 (Nikolai Kochetov). - Optimize performance of queries like
SELECT ... FINAL ... WHERE
. Now in queries withFINAL
it's allowed to move toPREWHERE
columns, which are in sorting key. #21830 (foolchi). - Improved performance by replacing
memcpy
to another implementation. This closes #18583. #21520 (alexey-milovidov). - Improve performance of aggregation in order of sorting key (with enabled setting
optimize_aggregation_in_order
). #19401 (Anton Popov).
- Add connection pool for PostgreSQL table/database engine and dictionary source. Should fix #21444. #21839 (Kseniia Sumarokova).
- Support non-default table schema for postgres storage/table-function. Closes #21701. #21711 (Kseniia Sumarokova).
- Support replicas priority for postgres dictionary source. #21710 (Kseniia Sumarokova).
- Introduce a new merge tree setting
min_bytes_to_rebalance_partition_over_jbod
which allows assigning new parts to different disks of a JBOD volume in a balanced way. #16481 (Amos Bird). - Added
Grant
,Revoke
andSystem
values ofquery_kind
column for corresponding queries insystem.query_log
. #21102 (Vasily Nemkov). - Allow customizing timeouts for HTTP connections used for replication independently from other HTTP timeouts. #20088 (nvartolomei).
- Better exception message in client in case of exception while server is writing blocks. In previous versions client may get misleading message like
Data compressed with different methods
. #22427 (alexey-milovidov). - Fix error
Directory tmp_fetch_XXX already exists
which could happen after failed fetch part. Delete temporary fetch directory if it already exists. Fixes #14197. #22411 (nvartolomei). - Fix MSan report for function
range
withUInt256
argument (support for large integers is experimental). This closes #22157. #22387 (alexey-milovidov). - Add
current_database
column tosystem.processes
table. It contains the current database of the query. #22365 (Alexander Kuzmenkov). - Add case-insensitive history search/navigation and subword movement features to
clickhouse-client
. #22105 (Amos Bird). - If tuple of NULLs, e.g.
(NULL, NULL)
is on the left hand side ofIN
operator with tuples of non-NULLs on the right hand side, e.g.SELECT (NULL, NULL) IN ((0, 0), (3, 1))
return 0 instead of throwing an exception about incompatible types. The expression may also appear due to optimization of something likeSELECT (NULL, NULL) = (8, 0) OR (NULL, NULL) = (3, 2) OR (NULL, NULL) = (0, 0) OR (NULL, NULL) = (3, 1)
. This closes #22017. #22063 (alexey-milovidov). - Update used version of simdjson to 0.9.1. This fixes #21984. #22057 (Vitaly Baranov).
- Added case insensitive aliases for
CONNECTION_ID()
andVERSION()
functions. This fixes #22028. #22042 (Eugene Klimov). - Add option
strict_increase
towindowFunnel
function to calculate each event once (resolve #21835). #22025 (Vladimir). - If partition key of a
MergeTree
table does not includeDate
orDateTime
columns but includes exactly oneDateTime64
column, expose its values in themin_time
andmax_time
columns insystem.parts
andsystem.parts_columns
tables. Addmin_time
andmax_time
columns tosystem.parts_columns
table (these was inconsistency to thesystem.parts
table). This closes #18244. #22011 (alexey-milovidov). - Supported
replication_alter_partitions_sync=1
setting inclickhouse-copier
for moving partitions from helping table to destination. Decreased default timeouts. Fixes #21911. #21912 (turbo jason). - Show path to data directory of
EmbeddedRocksDB
tables in system tables. #21903 (tavplubix). - Add profile event
HedgedRequestsChangeReplica
, change read data timeout from sec to ms. #21886 (Kruglov Pavel). - DiskS3 (experimental feature under development). Fixed bug with the impossibility to move directory if the destination is not empty and cache disk is used. #21837 (Pavel Kovalenko).
- Better formatting for
Array
andMap
data types in Web UI. #21798 (alexey-milovidov). - Update clusters only if their configurations were updated. #21685 (Kruglov Pavel).
- Propagate query and session settings for distributed DDL queries. Set
distributed_ddl_entry_format_version
to 2 to enable this. Addeddistributed_ddl_output_mode
setting. Supported modes:none
,throw
(default),null_status_on_timeout
andnever_throw
. Miscellaneous fixes and improvements forReplicated
database engine. #21535 (tavplubix). - If
PODArray
was instantiated with element size that is neither a fraction or a multiple of 16, buffer overflow was possible. No bugs in current releases exist. #21533 (alexey-milovidov). - Add
last_error_time
/last_error_message
/last_error_stacktrace
/remote
columns forsystem.errors
. #21529 (Azat Khuzhin). - Add aliases
simpleJSONExtract/simpleJSONHas
tovisitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}
. Fixes #21383. #21519 (fastio). - Add setting
optimize_skip_unused_shards_limit
to limit the number of sharding key values foroptimize_skip_unused_shards
. #21512 (Azat Khuzhin). - Improve
clickhouse-format
to not throw exception when there are extra spaces or comment after the last query, and throw exception early with readable message when formatASTInsertQuery
with data . #21311 (flynn). - Improve support of integer keys in data type
Map
. #21157 (Anton Popov). - MaterializeMySQL: attempt to reconnect to MySQL if the connection is lost. #20961 (Håvard Kvålen).
- Support more cases to rewrite
CROSS JOIN
toINNER JOIN
. #20392 (Vladimir). - Do not create empty parts on INSERT when
optimize_on_insert
setting enabled. Fixes #20304. #20387 (Kruglov Pavel). MaterializeMySQL
: add minmax skipping index for_version
column. #20382 (Stig Bakken).- Add option
--backslash
forclickhouse-format
, which can add a backslash at the end of each line of the formatted query. #21494 (flynn). - Now clickhouse will not throw
LOGICAL_ERROR
exception when we try to mutate the already covered part. Fixes #22013. #22291 (alesapin).
- Remove socket from epoll before cancelling packet receiver in
HedgedConnections
to prevent possible race. Fixes #22161. #22443 (Kruglov Pavel). - Add (missing) memory accounting in parallel parsing routines. In previous versions OOM was possible when the resultset contains very large blocks of data. This closes #22008. #22425 (alexey-milovidov).
- Fix exception which may happen when
SELECT
has constantWHERE
condition and source table has columns which names are digits. #22270 (LiuNeng). - Fix query cancellation with
use_hedged_requests=0
andasync_socket_for_remote=1
. #22183 (Azat Khuzhin). - Fix uncaught exception in
InterserverIOHTTPHandler
. #22146 (Azat Khuzhin). - Fix docker entrypoint in case
http_port
is not in the config. #22132 (Ewout). - Fix error
Invalid number of rows in Chunk
inJOIN
withTOTALS
andarrayJoin
. Closes #19303. #22129 (Vladimir). - Fix the background thread pool name which used to poll message from Kafka. The Kafka engine with the broken thread pool will not consume the message from message queue. #22122 (fastio).
- Fix waiting for
OPTIMIZE
andALTER
queries forReplicatedMergeTree
table engines. Now the query will not hang when the table was detached or restarted. #22118 (alesapin). - Disable
async_socket_for_remote
/use_hedged_requests
for buggy Linux kernels. #22109 (Azat Khuzhin). - Docker entrypoint: avoid chown of
.
in case whenLOG_PATH
is empty. Closes #22100. #22102 (filimonov). - The function
decrypt
was lacking a check for the minimal size of data encrypted inAEAD
mode. This closes #21897. #22064 (alexey-milovidov). - In rare case, merge for
CollapsingMergeTree
may create granule withindex_granularity + 1
rows. Because of this, internal check, added in #18928 (affects 21.2 and 21.3), may fail with errorIncomplete granules are not allowed while blocks are granules size
. This error did not allow parts to merge. #21976 (Nikolai Kochetov). - Reverted #15454 that may cause significant increase in memory usage while loading external dictionaries of hashed type. This closes #21935. #21948 (Maksim Kita).
- Prevent hedged connections overlaps (
Unknown packet 9 from server
error). #21941 (Azat Khuzhin). - Fix reading the HTTP POST request with "multipart/form-data" content type in some cases. #21936 (Ivan).
- Fix wrong
ORDER BY
results when a query contains window functions, and optimization for reading in primary key order is applied. Fixes #21828. #21915 (Alexander Kuzmenkov). - Fix deadlock in first catboost model execution. Closes #13832. #21844 (Kruglov Pavel).
- Fix incorrect query result (and possible crash) which could happen when
WHERE
orHAVING
condition is pushed beforeGROUP BY
. Fixes #21773. #21841 (Nikolai Kochetov). - Better error handling and logging in
WriteBufferFromS3
. #21836 (Pavel Kovalenko). - Fix possible crashes in aggregate functions with combinator
Distinct
, while using two-level aggregation. This is a follow-up fix of #18365 . Can only reproduced in production env. #21818 (Amos Bird). - Fix scalar subquery index analysis. This fixes #21717 , which was introduced in #18896. #21766 (Amos Bird).
- Fix bug for
ReplicatedMerge
table engines whenALTER MODIFY COLUMN
query doesn't change the type ofDecimal
column if its size (32 bit or 64 bit) doesn't change. #21728 (alesapin). - Fix possible infinite waiting when concurrent
OPTIMIZE
andDROP
are run forReplicatedMergeTree
. #21716 (Azat Khuzhin). - Fix function
arrayElement
with typeMap
for constant integer arguments. #21699 (Anton Popov). - Fix SIGSEGV on not existing attributes from
ip_trie
withaccess_to_key_from_attributes
. #21692 (Azat Khuzhin). - Server now start accepting connections only after
DDLWorker
and dictionaries initialization. #21676 (Azat Khuzhin). - Add type conversion for keys of tables of type
Join
(previously led to SIGSEGV). #21646 (Azat Khuzhin). - Fix distributed requests cancellation (for example simple select from multiple shards with limit, i.e.
select * from remote('127.{2,3}', system.numbers) limit 100
) withasync_socket_for_remote=1
. #21643 (Azat Khuzhin). - Fix
fsync_part_directory
for horizontal merge. #21642 (Azat Khuzhin). - Remove unknown columns from joined table in
WHERE
for queries to external database engines (MySQL, PostgreSQL). close #14614, close #19288 (dup), close #19645 (dup). #21640 (Vladimir). std::terminate
was called if there is an error writing data into s3. #21624 (Vladimir).- Fix possible error
Cannot find column
whenoptimize_skip_unused_shards
is enabled and zero shards are used. #21579 (Azat Khuzhin). - In case if query has constant
WHERE
condition, and settingoptimize_skip_unused_shards
enabled, all shards may be skipped and query could return incorrect empty result. #21550 (Amos Bird). - Fix table function
clusterAllReplicas
returns wrong_shard_num
. close #21481. #21498 (flynn). - Fix that S3 table holds old credentials after config update. #21457 (Grigory Pervakov).
- Fixed race on SSL object inside
SecureSocket
in Poco. #21456 (Nikita Mikhaylov). - Fix
Avro
format parsing forKafka
. Fixes #21437. #21438 (Ilya Golshtein). - Fix receive and send timeouts and non-blocking read in secure socket. #21429 (Kruglov Pavel).
force_drop_table
flag didn't work forMATERIALIZED VIEW
, it's fixed. Fixes #18943. #20626 (tavplubix).- Fix name clashes in
PredicateRewriteVisitor
. It caused incorrectWHERE
filtration after full join. Close #20497. #20622 (Vladimir).
- Add Jepsen tests for ClickHouse Keeper. #21677 (alesapin).
- Run stateless tests in parallel in CI. Depends on #22181. #22300 (alesapin).
- Enable status check for SQLancer CI run. #22015 (Ilya Yatsishin).
- Multiple preparations for PowerPC builds: Enable the bundled openldap on
ppc64le
. #22487 (Kfir Itzhak). Enable compiling onppc64le
with Clang. #22476 (Kfir Itzhak). Fix compiling boost onppc64le
. #22474 (Kfir Itzhak). Fix CMake error about internal CMake variableCMAKE_ASM_COMPILE_OBJECT
not set onppc64le
. #22469 (Kfir Itzhak). Fix Fedora/RHEL/CentOS not findinglibclang_rt.builtins
onppc64le
. #22458 (Kfir Itzhak). Enable building withjemalloc
onppc64le
. #22447 (Kfir Itzhak). Fix ClickHouse's config embedding and cctz's timezone embedding onppc64le
. #22445 (Kfir Itzhak). Fixed compiling onppc64le
and use the correct instruction pointer register onppc64le
. #22430 (Kfir Itzhak). - Re-enable the S3 (AWS) library on
aarch64
. #22484 (Kfir Itzhak). - Add
tzdata
to Docker containers because readingORC
formats requires it. This closes #14156. #22000 (alexey-milovidov). - Introduce 2 arguments for
clickhouse-server
image Dockerfile:deb_location
&single_binary_location
. #21977 (filimonov). - Allow to use clang-tidy with release builds by enabling assertions if it is used. #21914 (alexey-milovidov).
- Add llvm-12 binaries name to search in cmake scripts. Implicit constants conversions to mute clang warnings. Updated submodules to build with CMake 3.19. Mute recursion in macro expansion in
readpassphrase
library. Deprecated-fuse-ld
changed to--ld-path
for clang. #21597 (Ilya Yatsishin). - Updating
docker/test/testflows/runner/dockerd-entrypoint.sh
to use Yandex dockerhub-proxy, because Docker Hub has enabled very restrictive rate limits #21551 (vzakaznikov). - Fix macOS shared lib build. #20184 (nvartolomei).
- Add
ctime
option tozookeeper-dump-tree
. It allows to dump node creation time. #21842 (Ilya).
- Now it's not allowed to create MergeTree tables in old syntax with table TTL because it's just ignored. Attach of old tables is still possible. #20282 (alesapin).
- Now all case-insensitive function names will be rewritten to their canonical representations. This is needed for projection query routing (the upcoming feature). #20174 (Amos Bird).
- Fix creation of
TTL
in cases, when its expression is a function and it is the same asORDER BY
key. Now it's allowed to set custom aggregation to primary key columns inTTL
withGROUP BY
. Backward incompatible: For primary key columns, which are not inGROUP BY
and aren't set explicitly now is applied functionany
instead ofmax
, when TTL is expired. Also if you use TTL withWHERE
orGROUP BY
you can see exceptions at merges, while making rolling update. #15450 (Anton Popov).
- Add file engine settings:
engine_file_empty_if_not_exists
andengine_file_truncate_on_insert
. #20620 (M0r64n). - Add aggregate function
deltaSum
for summing the differences between consecutive rows. #20057 (Russ Frank). - New
event_time_microseconds
column insystem.part_log
table. #20027 (Bharat Nallan). - Added
timezoneOffset(datetime)
function which will give the offset from UTC in seconds. This close #issue:19850. #19962 (keenwolf). - Add setting
insert_shard_id
to support insert data into specific shard from distributed table. #19961 (flynn). - Function
reinterpretAs
updated to support big integers. Fixes #19691. #19858 (Maksim Kita). - Added Server Side Encryption Customer Keys (the
x-amz-server-side-encryption-customer-(key/md5)
header) support in S3 client. See the link. Closes #19428. #19748 (Vladimir Chebotarev). - Added
implicit_key
option forexecutable
dictionary source. It allows to avoid printing key for every record if records comes in the same order as the input keys. Implements #14527. #19677 (Maksim Kita). - Add quota type
query_selects
andquery_inserts
. #19603 (JackyWoo). - Add function
extractTextFromHTML
#19600 (zlx19950903), (alexey-milovidov). - Tables with
MergeTree*
engine now have two new table-level settings for query concurrency control. Settingmax_concurrent_queries
limits the number of concurrently executed queries which are related to this table. Settingmin_marks_to_honor_max_concurrent_queries
tells to apply previous setting only if query reads at least this number of marks. #19544 (Amos Bird). - Added
file
function to read file from user_files directory as a String. This is different from thefile
table function. This implements #issue:18851. #19204 (keenwolf).
- Add experimental
Replicated
database engine. It replicates DDL queries across multiple hosts. #16193 (tavplubix). - Introduce experimental support for window functions, enabled with
allow_experimental_window_functions = 1
. This is a preliminary, alpha-quality implementation that is not suitable for production use and will change in backward-incompatible ways in future releases. Please see the documentation for the list of supported features. #20337 (Alexander Kuzmenkov). - Add the ability to backup/restore metadata files for DiskS3. #18377 (Pavel Kovalenko).
- Hedged requests for remote queries. When setting
use_hedged_requests
enabled (off by default), allow to establish many connections with different replicas for query. New connection is enabled in case existent connection(s) with replica(s) were not established withinhedged_connection_timeout
or no data was received withinreceive_data_timeout
. Query uses the first connection which send non empty progress packet (or data packet, ifallow_changing_replica_until_first_data_packet
); other connections are cancelled. Queries withmax_parallel_replicas > 1
are supported. #19291 (Kruglov Pavel). This allows to significantly reduce tail latencies on very large clusters. - Added support for
PREWHERE
(and enable the corresponding optimization) when tables have row-level security expressions specified. #19576 (Denis Glazachev). - The setting
distributed_aggregation_memory_efficient
is enabled by default. It will lower memory usage and improve performance of distributed queries. #20599 (alexey-milovidov). - Improve performance of GROUP BY multiple fixed size keys. #20472 (alexey-milovidov).
- Improve performance of aggregate functions by more strict aliasing. #19946 (alexey-milovidov).
- Speed up reading from
Memory
tables in extreme cases (when reading speed is in order of 50 GB/sec) by simplification of pipeline and (consequently) less lock contention in pipeline scheduling. #20468 (alexey-milovidov). - Partially reimplement HTTP server to make it making less copies of incoming and outgoing data. It gives up to 1.5 performance improvement on inserting long records over HTTP. #19516 (Ivan).
- Add
compress
setting forMemory
tables. If it's enabled the table will use less RAM. On some machines and datasets it can also work faster on SELECT, but it is not always the case. This closes #20093. Note: there are reasons why Memory tables can work slower than MergeTree: (1) lack of compression (2) static size of blocks (3) lack of indices and prewhere... #20168 (alexey-milovidov). - Slightly better code in aggregation. #20978 (alexey-milovidov).
- Add back
intDiv
/modulo
specializations for better performance. This fixes #21293 . The regression was introduced in ClickHouse#18145 . #21307 (Amos Bird). - Do not squash blocks too much on INSERT SELECT if inserting into Memory table. In previous versions inefficient data representation was created in Memory table after INSERT SELECT. This closes #13052. #20169 (alexey-milovidov).
- Fix at least one case when DataType parser may have exponential complexity (found by fuzzer). This closes #20096. #20132 (alexey-milovidov).
- Parallelize SELECT with FINAL for single part with level > 0 when
do_not_merge_across_partitions_select_final
setting is 1. #19375 (Kruglov Pavel). - Fill only requested columns when querying
system.parts
andsystem.parts_columns
. Closes #19570. #21035 (Anmol Arora). - Perform algebraic optimizations of arithmetic expressions inside
avg
aggregate function. close #20092. #20183 (flynn).
- Case-insensitive compression methods for table functions. Also fixed LZMA compression method which was checked in upper case. #21416 (Vladimir Chebotarev).
- Add two settings to delay or throw error during insertion when there are too many inactive parts. This is useful when server fails to clean up parts quickly enough. #20178 (Amos Bird).
- Provide better compatibility for mysql clients. 1. mysql jdbc 2. mycli. #21367 (Amos Bird).
- Forbid to drop a column if it's referenced by materialized view. Closes #21164. #21303 (flynn).
- MySQL dictionary source will now retry unexpected connection failures (Lost connection to MySQL server during query) which sometimes happen on SSL/TLS connections. #21237 (Alexander Kazakov).
- Usability improvement: more consistent
DateTime64
parsing: recognize the case when unix timestamp with subsecond resolution is specified as scaled integer (like1111111111222
instead of1111111111.222
). This closes #13194. #21053 (alexey-milovidov). - Do only merging of sorted blocks on initiator with distributed_group_by_no_merge. #20882 (Azat Khuzhin).
- When loading config for mysql source ClickHouse will now randomize the list of replicas with the same priority to ensure the round-robin logics of picking mysql endpoint. This closes #20629. #20632 (Alexander Kazakov).
- Function 'reinterpretAs(x, Type)' renamed into 'reinterpret(x, Type)'. #20611 (Maksim Kita).
- Support vhost for RabbitMQ engine #20576. #20596 (Kseniia Sumarokova).
- Improved serialization for data types combined of Arrays and Tuples. Improved matching enum data types to protobuf enum type. Fixed serialization of the
Map
data type. Omitted values are now set by default. #20506 (Vitaly Baranov). - Fixed race between execution of distributed DDL tasks and cleanup of DDL queue. Now DDL task cannot be removed from ZooKeeper if there are active workers. Fixes #20016. #20448 (tavplubix).
- Make FQDN and other DNS related functions work correctly in alpine images. #20336 (filimonov).
- Do not allow early constant folding of explicitly forbidden functions. #20303 (Azat Khuzhin).
- Implicit conversion from integer to Decimal type might succeeded if integer value doe not fit into Decimal type. Now it throws
ARGUMENT_OUT_OF_BOUND
. #20232 (tavplubix). - Lockless
SYSTEM FLUSH DISTRIBUTED
. #20215 (Azat Khuzhin). - Normalize count(constant), sum(1) to count(). This is needed for projection query routing. #20175 (Amos Bird).
- Support all native integer types in bitmap functions. #20171 (Amos Bird).
- Updated
CacheDictionary
,ComplexCacheDictionary
,SSDCacheDictionary
,SSDComplexKeyDictionary
to use LRUHashMap as underlying index. #20164 (Maksim Kita). - The setting
access_management
is now configurable on startup by providingCLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT
, defaults to disabled (0
) which was the prior value. #20139 (Marquitos). - Fix toDateTime64(toDate()/toDateTime()) for DateTime64 - Implement DateTime64 clamping to match DateTime behaviour. #20131 (Azat Khuzhin).
- Quota improvements: SHOW TABLES is now considered as one query in the quota calculations, not two queries. SYSTEM queries now consume quota. Fix calculation of interval's end in quota consumption. #20106 (Vitaly Baranov).
- Supports
path IN (set)
expressions forsystem.zookeeper
table. #20105 (小路). - Show full details of
MaterializeMySQL
tables insystem.tables
. #20051 (Stig Bakken). - Fix data race in executable dictionary that was possible only on misuse (when the script returns data ignoring its input). #20045 (alexey-milovidov).
- The value of MYSQL_OPT_RECONNECT option can now be controlled by "opt_reconnect" parameter in the config section of mysql replica. #19998 (Alexander Kazakov).
- If user calls
JSONExtract
function withFloat32
type requested, allow inaccurate conversion to the result type. For example the number0.1
in JSON is double precision and is not representable in Float32, but the user still wants to get it. Previous versions return 0 for non-Nullable type and NULL for Nullable type to indicate that conversion is imprecise. The logic was 100% correct but it was surprising to users and leading to questions. This closes #13962. #19960 (alexey-milovidov). - Add conversion of block structure for INSERT into Distributed tables if it does not match. #19947 (Azat Khuzhin).
- Improvement for the
system.distributed_ddl_queue
table. Initialize MaxDDLEntryID to the last value after restarting. Before this PR, MaxDDLEntryID will remain zero until a new DDLTask is processed. #19924 (Amos Bird). - Show
MaterializeMySQL
tables insystem.parts
. #19770 (Stig Bakken). - Add separate config directive for
Buffer
profile. #19721 (Azat Khuzhin). - Move conditions that are not related to JOIN to WHERE clause. #18720. #19685 (hexiaoting).
- Add ability to throttle INSERT into Distributed based on amount of pending bytes for async send (
bytes_to_delay_insert
/max_delay_to_insert
andbytes_to_throw_insert
settings forDistributed
engine has been added). #19673 (Azat Khuzhin). - Fix some rare cases when write errors can be ignored in destructors. #19451 (Azat Khuzhin).
- Print inline frames in stack traces for fatal errors. #19317 (Ivan).
- Fix redundant reconnects to ZooKeeper and the possibility of two active sessions for a single clickhouse server. Both problems introduced in #14678. #21264 (alesapin).
- Fix error
Bad cast from type ... to DB::ColumnLowCardinality
while inserting into table withLowCardinality
column fromValues
format. Fixes #21140 #21357 (Nikolai Kochetov). - Fix a deadlock in
ALTER DELETE
mutations for non replicated MergeTree table engines when the predicate contains the table itself. Fixes #20558. #21477 (alesapin). - Fix SIGSEGV for distributed queries on failures. #21434 (Azat Khuzhin).
- Now
ALTER MODIFY COLUMN
queries will correctly affect changes in partition key, skip indices, TTLs, and so on. Fixes #13675. #21334 (alesapin). - Fix bug with
join_use_nulls
and joiningTOTALS
from subqueries. This closes #19362 and #21137. #21248 (vdimir). - Fix crash in
EXPLAIN
for query withUNION
. Fixes #20876, #21170. #21246 (flynn). - Now mutations allowed only for table engines that support them (MergeTree family, Memory, MaterializedView). Other engines will report a more clear error. Fixes #21168. #21183 (alesapin).
- Fixes #21112. Fixed bug that could cause duplicates with insert query (if one of the callbacks came a little too late). #21138 (Kseniia Sumarokova).
- Fix
input_format_null_as_default
take effective when types are nullable. This fixes #21116 . #21121 (Amos Bird). - fix bug related to cast Tuple to Map. Closes #21029. #21120 (hexiaoting).
- Fix the metadata leak when the Replicated*MergeTree with custom (non default) ZooKeeper cluster is dropped. #21119 (fastio).
- Fix type mismatch issue when using LowCardinality keys in joinGet. This fixes #21114. #21117 (Amos Bird).
- fix default_replica_path and default_replica_name values are useless on Replicated(*)MergeTree engine when the engine needs specify other parameters. #21060 (mxzlxy).
- Out of bound memory access was possible when formatting specifically crafted out of range value of type
DateTime64
. This closes #20494. This closes #20543. #21023 (alexey-milovidov). - Block parallel insertions into storage join. #21009 (vdimir).
- Fixed behaviour, when
ALTER MODIFY COLUMN
created mutation, that will knowingly fail. #21007 (Anton Popov). - Closes #9969. Fixed Brotli http compression error, which reproduced for large data sizes, slightly complicated structure and with json output format. Update Brotli to the latest version to include the "fix rare access to uninitialized data in ring-buffer". #20991 (Kseniia Sumarokova).
- Fix 'Empty task was returned from async task queue' on query cancellation. #20881 (Azat Khuzhin).
USE database;
query did not work when using MySQL 5.7 client to connect to ClickHouse server, it's fixed. Fixes #18926. #20878 (tavplubix).- Fix usage of
-Distinct
combinator with-State
combinator in aggregate functions. #20866 (Anton Popov). - Fix subquery with union distinct and limit clause. close #20597. #20610 (flynn).
- Fixed inconsistent behavior of dictionary in case of queries where we look for absent keys in dictionary. #20578 (Nikita Mikhaylov).
- Fix the number of threads for scalar subqueries and subqueries for index (after #19007 single thread was always used). Fixes #20457, #20512. #20550 (Nikolai Kochetov).
- Fix crash which could happen if unknown packet was received from remove query (was introduced in #17868). #20547 (Azat Khuzhin).
- Add proper checks while parsing directory names for async INSERT (fixes SIGSEGV). #20498 (Azat Khuzhin).
- Fix function
transform
does not work properly for floating point keys. Closes #20460. #20479 (flynn). - Fix infinite loop when propagating WITH aliases to subqueries. This fixes #20388. #20476 (Amos Bird).
- Fix abnormal server termination when http client goes away. #20464 (Azat Khuzhin).
- Fix
LOGICAL_ERROR
forjoin_use_nulls=1
when JOIN contains const from SELECT. #20461 (Azat Khuzhin). - Check if table function
view
is used in expression list and throw an error. This fixes #20342. #20350 (Amos Bird). - Avoid invalid dereference in RANGE_HASHED() dictionary. #20345 (Azat Khuzhin).
- Fix null dereference with
join_use_nulls=1
. #20344 (Azat Khuzhin). - Fix incorrect result of binary operations between two constant decimals of different scale. Fixes #20283. #20339 (Maksim Kita).
- Fix too often retries of failed background tasks for
ReplicatedMergeTree
table engines family. This could lead to too verbose logging and increased CPU load. Fixes #20203. #20335 (alesapin). - Restrict to
DROP
orRENAME
version column of*CollapsingMergeTree
andReplacingMergeTree
table engines. #20300 (alesapin). - Fixed the behavior when in case of broken JSON we tried to read the whole file into memory which leads to exception from the allocator. Fixes #19719. #20286 (Nikita Mikhaylov).
- Fix exception during vertical merge for
MergeTree
table engines family which don't allow to perform vertical merges. Fixes #20259. #20279 (alesapin). - Fix rare server crash on config reload during the shutdown. Fixes #19689. #20224 (alesapin).
- Fix CTE when using in INSERT SELECT. This fixes #20187, fixes #20195. #20211 (Amos Bird).
- Fixes #19314. #20156 (Ivan).
- fix toMinute function to handle special timezone correctly. #20149 (keenwolf).
- Fix server crash after query with
if
function withTuple
type of then/else branches result.Tuple
type must containArray
or another complex type. Fixes #18356. #20133 (alesapin). - The
MongoDB
table engine now establishes connection only when it's going to read data.ATTACH TABLE
won't try to connect anymore. #20110 (Vitaly Baranov). - Bugfix in StorageJoin. #20079 (vdimir).
- Fix the case when calculating modulo of division of negative number by small divisor, the resulting data type was not large enough to accomodate the negative result. This closes #20052. #20067 (alexey-milovidov).
- MaterializeMySQL: Fix replication for statements that update several tables. #20066 (Håvard Kvålen).
- Prevent "Connection refused" in docker during initialization script execution. #20012 (filimonov).
EmbeddedRocksDB
is an experimental storage. Fix the issue with lack of proper type checking. Simplified code. This closes #19967. #19972 (alexey-milovidov).- Fix a segfault in function
fromModifiedJulianDay
when the argument type isNullable(T)
for any integral types other than Int32. #19959 (PHO). - BloomFilter index crash fix. Fixes #19757. #19884 (Maksim Kita).
- Deadlock was possible if system.text_log is enabled. This fixes #19874. #19875 (alexey-milovidov).
- Fix starting the server with tables having default expressions containing dictGet(). Allow getting return type of dictGet() without loading dictionary. #19805 (Vitaly Baranov).
- Fix clickhouse-client abort exception while executing only
select
. #19790 (taiyang-li). - Fix a bug that moving pieces to destination table may failed in case of launching multiple clickhouse-copiers. #19743 (madianjun).
- Background thread which executes
ON CLUSTER
queries might hang waiting for dropped replicated table to do something. It's fixed. #19684 (yiguolei).
- Allow to build ClickHouse with AVX-2 enabled globally. It gives slight performance benefits on modern CPUs. Not recommended for production and will not be supported as official build for now. #20180 (alexey-milovidov).
- Fix some of the issues found by Coverity. See #19964. #20010 (alexey-milovidov).
- Allow to start up with modified binary under gdb. In previous version if you set up breakpoint in gdb before start, server will refuse to start up due to failed integrity check. #21258 (alexey-milovidov).
- Add a test for different compression methods in Kafka. #21111 (filimonov).
- Fixed port clash from test_storage_kerberized_hdfs test. #19974 (Ilya Yatsishin).
- Print
stdout
andstderr
to log when failed to start docker in integration tests. Before this PR there was a very short error message in this case which didn't help to investigate the problems. #20631 (Vitaly Baranov).
- Bitwise functions (
bitAnd
,bitOr
, etc) are forbidden for floating point arguments. Now you have to do explicit cast to integer. #19853 (Azat Khuzhin). - Forbid
lcm
/gcd
for floats. #19532 (Azat Khuzhin). - Fix memory tracking for
OPTIMIZE TABLE
/merges; account query memory limits and sampling forOPTIMIZE TABLE
/merges. #18772 (Azat Khuzhin). - Disallow floating point column as partition key, see #18421. #18464 (hexiaoting).
- Excessive parenthesis in type definitions no longer supported, example:
Array((UInt8))
.
- Added
PostgreSQL
table engine (both select/insert, with support for multidimensional arrays), also as table function. AddedPostgreSQL
dictionary source. AddedPostgreSQL
database engine. #18554 (Kseniia Sumarokova). - Data type
Nested
now supports arbitrary levels of nesting. Introduced subcolumns of complex types, such assize0
inArray
,null
inNullable
, names ofTuple
elements, which can be read without reading of whole column. #17310 (Anton Popov). - Added
Nullable
support forFlatDictionary
,HashedDictionary
,ComplexKeyHashedDictionary
,DirectDictionary
,ComplexKeyDirectDictionary
,RangeHashedDictionary
. #18236 (Maksim Kita). - Adds a new table called
system.distributed_ddl_queue
that displays the queries in the DDL worker queue. #17656 (Bharat Nallan). - Added support of mapping LDAP group names, and attribute values in general, to local roles for users from ldap user directories. #17211 (Denis Glazachev).
- Support insert into table function
cluster
, and for both table functionsremote
andcluster
, support distributing data across nodes by specify sharding key. Close #16752. #18264 (flynn). - Add function
decodeXMLComponent
to decode characters for XML. Example:SELECT decodeXMLComponent('Hello,"world"!')
#17659. #18542 (nauta). - Added functions
parseDateTimeBestEffortUSOrZero
,parseDateTimeBestEffortUSOrNull
. #19712 (Maksim Kita). - Add
sign
math function. #19527 (flynn). - Add information about used features (functions, table engines, etc) into system.query_log. #18495. #19371 (Kseniia Sumarokova).
- Function
formatDateTime
support the%Q
modification to format date to quarter. #19224 (Jianmei Zhang). - Support MetaKey+Enter hotkey binding in play UI. #19012 (sundyli).
- Add three functions for map data type: 1.
mapContains(map, key)
to check weather map.keys include the second parameter key. 2.mapKeys(map)
return all the keys in Array format 3.mapValues(map)
return all the values in Array format. #18788 (hexiaoting). - Add
log_comment
setting related to #18494. #18549 (Zijie Lu). - Add support of tuple argument to
argMin
andargMax
functions. #17359 (Ildus Kurbangaliev). - Support
EXISTS VIEW
syntax. #18552 (Du Chuan). - Add
SELECT ALL
syntax. closes #18706. #18723 (flynn).
- Faster parts removal by lowering the number of
stat
syscalls. This returns the optimization that existed while ago. More safe interface ofIDisk
. This closes #19065. #19086 (alexey-milovidov). - Aliases declared in
WITH
statement are properly used in index analysis. Queries likeWITH column AS alias SELECT ... WHERE alias = ...
may use index now. #18896 (Amos Bird). - Add
optimize_alias_column_prediction
(on by default), that will: - Respect aliased columns in WHERE during partition pruning and skipping data using secondary indexes; - Respect aliased columns in WHERE for trivial count queries for optimize_trivial_count; - Respect aliased columns in GROUP BY/ORDER BY for optimize_aggregation_in_order/optimize_read_in_order. #16995 (sundyli). - Speed up aggregate function
sum
. Improvement only visible on synthetic benchmarks and not very practical. #19216 (alexey-milovidov). - Update libc++ and use another ABI to provide better performance. #18914 (Danila Kutenin).
- Rewrite
sumIf()
andsum(if())
function tocountIf()
function when logically equivalent. #17041 (flynn). - Use a connection pool for S3 connections, controlled by the
s3_max_connections
settings. #13405 (Vladimir Chebotarev). - Add support for zstd long option for better compression of string columns to save space. #17184 (ygrek).
- Slightly improve server latency by removing access to configuration on every connection. #19863 (alexey-milovidov).
- Reduce lock contention for multiple layers of the
Buffer
engine. #19379 (Azat Khuzhin). - Support splitting
Filter
step of query plan intoExpression + Filter
pair. Together withExpression + Expression
merging optimization (#17458) it may delay execution for some expressions afterFilter
step. #19253 (Nikolai Kochetov).
SELECT count() FROM table
now can be executed if only one any column can be selected from thetable
. This PR fixes #10639. #18233 (Vitaly Baranov).- Set charset to
utf8mb4
when interacting with remote MySQL servers. Fixes #19795. #19800 (alexey-milovidov). S3
table function now supportsauto
compression mode (autodetect). This closes #18754. #19793 (Vladimir Chebotarev).- Correctly output infinite arguments for
formatReadableTimeDelta
function. In previous versions, there was implicit conversion to implementation specific integer value. #19791 (alexey-milovidov). - Table function
S3
will use global region if the region can't be determined exactly. This closes #10998. #19750 (Vladimir Chebotarev). - In distributed queries if the setting
async_socket_for_remote
is enabled, it was possible to get stack overflow at least in debug build configuration if very deeply nested data type is used in table (e.g.Array(Array(Array(...more...)))
). This fixes #19108. This change introduces minor backward incompatibility: excessive parenthesis in type definitions no longer supported, example:Array((UInt8))
. #19736 (alexey-milovidov). - Add separate pool for message brokers (RabbitMQ and Kafka). #19722 (Azat Khuzhin).
- Fix rare
max_number_of_merges_with_ttl_in_pool
limit overrun (more merges with TTL can be assigned) for non-replicated MergeTree. #19708 (alesapin). - Dictionary: better error message during attribute parsing. #19678 (Maksim Kita).
- Add an option to disable validation of checksums on reading. Should never be used in production. Please do not expect any benefits in disabling it. It may only be used for experiments and benchmarks. The setting only applicable for tables of MergeTree family. Checksums are always validated for other table engines and when receiving data over network. In my observations there is no performance difference or it is less than 0.5%. #19588 (alexey-milovidov).
- Support constant result in function
multiIf
. #19533 (Maksim Kita). - Enable function length/empty/notEmpty for datatype Map, which returns keys number in Map. #19530 (taiyang-li).
- Add
--reconnect
option toclickhouse-benchmark
. When this option is specified, it will reconnect before every request. This is needed for testing. #19872 (alexey-milovidov). - Support using the new location of
.debug
file. This fixes #19348. #19520 (Amos Bird). toIPv6
function parsesIPv4
addresses. #19518 (Bharat Nallan).- Add
http_referer
field tosystem.query_log
,system.processes
, etc. This closes #19389. #19390 (alexey-milovidov). - Improve MySQL compatibility by making more functions case insensitive and adding aliases. #19387 (Daniil Kondratyev).
- Add metrics for MergeTree parts (Wide/Compact/InMemory) types. #19381 (Azat Khuzhin).
- Allow docker to be executed with arbitrary uid. #19374 (filimonov).
- Fix wrong alignment of values of
IPv4
data type in Pretty formats. They were aligned to the right, not to the left. This closes #19184. #19339 (alexey-milovidov). - Allow change
max_server_memory_usage
without restart. This closes #18154. #19186 (alexey-milovidov). - The exception when function
bar
is called with certain NaN argument may be slightly misleading in previous versions. This fixes #19088. #19107 (alexey-milovidov). - Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server images. #19096 (filimonov).
- Fixed
PeekableReadBuffer: Memory limit exceed
error when inserting data with huge strings. Fixes #18690. #18979 (tavplubix). - Docker image: several improvements for clickhouse-server entrypoint. #18954 (filimonov).
- Add
normalizeQueryKeepNames
andnormalizedQueryHashKeepNames
to normalize queries without masking long names with?
. This helps better analyze complex query logs. #18910 (Amos Bird). - Check per-block checksum of the distributed batch on the sender before sending (without reading the file twice, the checksums will be verified while reading), this will avoid stuck of the INSERT on the receiver (on truncated .bin file on the sender). Avoid reading .bin files twice for batched INSERT (it was required to calculate rows/bytes to take squashing into account, now this information included into the header, backward compatible is preserved). #18853 (Azat Khuzhin).
- Fix issues with RIGHT and FULL JOIN of tables with aggregate function states. In previous versions exception about
cloneResized
method was thrown. #18818 (templarzq). - Added prefix-based S3 endpoint settings. #18812 (Vladimir Chebotarev).
- Add [UInt8, UInt16, UInt32, UInt64] arguments types support for bitmapTransform, bitmapSubsetInRange, bitmapSubsetLimit, bitmapContains functions. This closes #18713. #18791 (sundyli).
- Allow CTE (Common Table Expressions) to be further aliased. Propagate CSE (Common Subexpressions Elimination) to subqueries in the same level when
enable_global_with_statement = 1
. This fixes #17378 . This fixes ClickHouse#16575 (comment) . #18684 (Amos Bird). - Update librdkafka to v1.6.0-RC2. Fixes #18668. #18671 (filimonov).
- In case of unexpected exceptions automatically restart background thread which is responsible for execution of distributed DDL queries. Fixes #17991. #18285 (徐炘).
- Updated AWS C++ SDK in order to utilize global regions in S3. #17870 (Vladimir Chebotarev).
- Added support for
WITH ... [AND] [PERIODIC] REFRESH [interval_in_sec]
clause when creatingLIVE VIEW
tables. #14822 (vzakaznikov). - Restrict
MODIFY TTL
queries forMergeTree
tables created in old syntax. Previously the query succeeded, but actually it had no effect. #19064 (Anton Popov).
- Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes #18364. #18373 (Amos Bird).
- Fix starting the server with tables having default expressions containing dictGet(). Allow getting return type of dictGet() without loading dictionary. #19805 (Vitaly Baranov).
- Fix server crash after query with
if
function withTuple
type of then/else branches result.Tuple
type must containArray
or another complex type. Fixes #18356. #20133 (alesapin). MaterializeMySQL
(experimental feature): Fix replication for statements that update several tables. #20066 (Håvard Kvålen).- Prevent "Connection refused" in docker during initialization script execution. #20012 (filimonov).
EmbeddedRocksDB
is an experimental storage. Fix the issue with lack of proper type checking. Simplified code. This closes #19967. #19972 (alexey-milovidov).- Fix a segfault in function
fromModifiedJulianDay
when the argument type isNullable(T)
for any integral types other than Int32. #19959 (PHO). - The function
greatCircleAngle
returned inaccurate results in previous versions. This closes #19769. #19789 (alexey-milovidov). - Fix rare bug when some replicated operations (like mutation) cannot process some parts after data corruption. Fixes #19593. #19702 (alesapin).
- Background thread which executes
ON CLUSTER
queries might hang waiting for dropped replicated table to do something. It's fixed. #19684 (yiguolei). - Fix wrong deserialization of columns description. It makes INSERT into a table with a column named
\
impossible. #19479 (alexey-milovidov). - Mark distributed batch as broken in case of empty data block in one of files. #19449 (Azat Khuzhin).
- Fixed very rare bug that might cause mutation to hang after
DROP/DETACH/REPLACE/MOVE PARTITION
. It was partially fixed by #15537 for the most cases. #19443 (tavplubix). - Fix possible error
Extremes transform was already added to pipeline
. Fixes #14100. #19430 (Nikolai Kochetov). - Fix default value in join types with non-zero default (e.g. some Enums). Closes #18197. #19360 (vdimir).
- Do not mark file for distributed send as broken on EOF. #19290 (Azat Khuzhin).
- Fix leaking of pipe fd for
async_socket_for_remote
. #19153 (Azat Khuzhin). - Fix infinite reading from file in
ORC
format (was introduced in #10580). Fixes #19095. #19134 (Nikolai Kochetov). - Fix issue in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes #18913. #19123 (alesapin).
- Fix startup bug when clickhouse was not able to read compression codec from
LowCardinality(Nullable(...))
and throws exceptionAttempt to read after EOF
. Fixes #18340. #19101 (alesapin). - Simplify the implementation of
tupleHammingDistance
. Support for tuples of any equal length. Fixes #19029. #19084 (Nikolai Kochetov). - Make sure
groupUniqArray
returns correct type for argument of Enum type. This closes #17875. #19019 (alexey-milovidov). - Fix possible error
Expected single dictionary argument for function
if use functionignore
withLowCardinality
argument. Fixes #14275. #19016 (Nikolai Kochetov). - Fix inserting of
LowCardinality
column to table withTinyLog
engine. Fixes #18629. #19010 (Nikolai Kochetov). - Fix minor issue in JOIN: Join tries to materialize const columns, but our code waits for them in other places. #18982 (Nikita Mikhaylov).
- Disable
optimize_move_functions_out_of_any
because optimization is not always correct. This closes #18051. This closes #18973. #18981 (alexey-milovidov). - Fix possible exception
QueryPipeline stream: different number of columns
caused by merging of query plan'sExpression
steps. Fixes #18190. #18980 (Nikolai Kochetov). - Fixed very rare deadlock at shutdown. #18977 (tavplubix).
- Fixed rare crashes when server run out of memory. #18976 (tavplubix).
- Fix incorrect behavior when
ALTER TABLE ... DROP PART 'part_name'
query removes all deduplication blocks for the whole partition. Fixes #18874. #18969 (alesapin). - Fixed issue #18894 Add a check to avoid exception when long column alias('table.column' style, usually auto-generated by BI tools like Looker) equals to long table name. #18968 (Daniel Qin).
- Fix error
Task was not found in task queue
(possible only for remote queries, withasync_socket_for_remote = 1
). #18964 (Nikolai Kochetov). - Fix bug when mutation with some escaped text (like
ALTER ... UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1')
serialized incorrectly. Fixes #18878. #18944 (alesapin). - ATTACH PARTITION will reset mutations. #18804. #18935 (fastio).
- Fix issue with
bitmapOrCardinality
that may lead to nullptr dereference. This closes #18911. #18912 (sundyli). - Fixed
Attempt to read after eof
error when trying toCAST
NULL
fromNullable(String)
toNullable(Decimal(P, S))
. Now functionCAST
returnsNULL
when it cannot parse decimal from nullable string. Fixes #7690. #18718 (Winter Zhang). - Fix data type convert issue for MySQL engine. #18124 (bo zeng).
- Fix clickhouse-client abort exception while executing only
select
. #19790 (taiyang-li).
- Run SQLancer (logical SQL fuzzer) in CI. #19006 (Ilya Yatsishin).
- Query Fuzzer will fuzz newly added tests more extensively. This closes #18916. #19185 (alexey-milovidov).
- Integrate with Big List of Naughty Strings for better fuzzing. #19480 (alexey-milovidov).
- Add integration tests run with MSan. #18974 (alesapin).
- Fixed MemorySanitizer errors in cyrus-sasl and musl. #19821 (Ilya Yatsishin).
- Insuffiient arguments check in
positionCaseInsensitiveUTF8
function triggered address sanitizer. #19720 (alexey-milovidov). - Remove --project-directory for docker-compose in integration test. Fix logs formatting from docker container. #19706 (Ilya Yatsishin).
- Made generation of macros.xml easier for integration tests. No more excessive logging from dicttoxml. dicttoxml project is not active for 5+ years. #19697 (Ilya Yatsishin).
- Allow to explicitly enable or disable watchdog via environment variable
CLICKHOUSE_WATCHDOG_ENABLE
. By default it is enabled if server is not attached to terminal. #19522 (alexey-milovidov). - Allow building ClickHouse with Kafka support on arm64. #19369 (filimonov).
- Allow building librdkafka without ssl. #19337 (filimonov).
- Restore Kafka input in FreeBSD builds. #18924 (Alexandre Snarskii).
- Fix potential nullptr dereference in table function
VALUES
. #19357 (alexey-milovidov). - Avoid UBSan reports in
arrayElement
function,substring
andarraySum
. Fixes #19305. Fixes #19287. This closes #19336. #19347 (alexey-milovidov).
- BloomFilter index crash fix. Fixes #19757. #19884 (Maksim Kita).
- Fix crash when pushing down predicates to union distinct subquery. This fixes #19855. #19861 (Amos Bird).
- Fix filtering by UInt8 greater than 127. #19799 (Anton Popov).
- In previous versions, unusual arguments for function arrayEnumerateUniq may cause crash or infinite loop. This closes #19787. #19788 (alexey-milovidov).
- Fixed stack overflow when using accurate comparison of arithmetic type with string type. #19773 (tavplubix).
- Fix crash when nested column name was used in
WHERE
orPREWHERE
. Fixes #19755. #19763 (Nikolai Kochetov). - Fix a segmentation fault in
bitmapAndnot
function. Fixes #19668. #19713 (Maksim Kita). - Some functions with big integers may cause segfault. Big integers is experimental feature. This closes #19667. #19672 (alexey-milovidov).
- Fix wrong result of function
neighbor
forLowCardinality
argument. Fixes #10333. #19617 (Nikolai Kochetov). - Fix use-after-free of the CompressedWriteBuffer in Connection after disconnect. #19599 (Azat Khuzhin).
DROP/DETACH TABLE table ON CLUSTER cluster SYNC
query might hang, it's fixed. Fixes #19568. #19572 (tavplubix).- Query CREATE DICTIONARY id expression fix. #19571 (Maksim Kita).
- Fix SIGSEGV with merge_tree_min_rows_for_concurrent_read/merge_tree_min_bytes_for_concurrent_read=0/UINT64_MAX. #19528 (Azat Khuzhin).
- Buffer overflow (on memory read) was possible if
addMonth
function was called with specifically crafted arguments. This fixes #19441. This fixes #19413. #19472 (alexey-milovidov). - Uninitialized memory read was possible in encrypt/decrypt functions if empty string was passed as IV. This closes #19391. #19397 (alexey-milovidov).
- Fix possible buffer overflow in Uber H3 library. See uber/h3#392. This closes #19219. #19383 (alexey-milovidov).
- Fix system.parts _state column (LOGICAL_ERROR when querying this column, due to incorrect order). #19346 (Azat Khuzhin).
- Fixed possible wrong result or segfault on aggregation when Materialized View and its target table have different structure. Fixes #18063. #19322 (tavplubix).
- Fix error
Cannot convert column now64() because it is constant but values of constants are different in source and result
. Continuation of #7156. #19316 (Nikolai Kochetov). - Fix bug when concurrent
ALTER
andDROP
queries may hang while processing ReplicatedMergeTree table. #19237 (alesapin). - Fixed
There is no checkpoint
error when inserting data through http interface usingTemplate
orCustomSeparated
format. Fixes #19021. #19072 (tavplubix). - Disable constant folding for subqueries on the analysis stage, when the result cannot be calculated. #18446 (Azat Khuzhin).
- Mutation might hang waiting for some non-existent part after
MOVE
orREPLACE PARTITION
or, in rare cases, afterDETACH
orDROP PARTITION
. It's fixed. #15537 (tavplubix).
- The setting
input_format_null_as_default
is enabled by default. #17525 (alexey-milovidov). - Check settings constraints for profile settings from config. Server will fail to start if users.xml contain settings that do not meet corresponding constraints. #18486 (tavplubix).
- Restrict
ALTER MODIFY SETTING
from changing storage settings that affects data parts (write_final_mark
andenable_mixed_granularity_parts
). #18306 (Amos Bird). - Set
insert_quorum_parallel
to 1 by default. It is significantly more convenient to use than "sequential" quorum inserts. But if you rely to sequential consistency, you should set the setting back to zero. #17567 (alexey-milovidov). - Remove
sumburConsistentHash
function. This closes #18120. #18656 (alexey-milovidov). - Removed aggregate functions
timeSeriesGroupSum
,timeSeriesGroupRateSum
because a friend of mine said they never worked. This fixes #16869. If you have luck using these functions, write a email to [email protected]. #17423 (alexey-milovidov). - Prohibit toUnixTimestamp(Date()) (before it just returns UInt16 representation of Date). #17376 (Azat Khuzhin).
- Allow using extended integer types (
Int128
,Int256
,UInt256
) inavg
andavgWeighted
functions. Also allow using different types (integer, decimal, floating point) for value and for weight inavgWeighted
function. This is a backward-incompatible change: now theavg
andavgWeighted
functions always returnFloat64
(as documented). Before this change the return type forDecimal
arguments was alsoDecimal
. #15419 (Mike). - Expression
toUUID(N)
no longer works. Replace withtoUUID('00000000-0000-0000-0000-000000000000')
. This change is motivated by non-obvious results oftoUUID(N)
where N is non zero. - SSL Certificates with incorrect "key usage" are rejected. In previous versions they are used to work. See #19262.
incl
references to substitutions file (/etc/metrika.xml
) were removed from the default config (<remote_servers>
,<zookeeper>
,<macros>
,<compression>
,<networks>
). If you were using substitutions file and were relying on those implicit references, you should put them back manually and explicitly by adding corresponding sections withincl="..."
attributes before the update. See #18740 (alexey-milovidov).
- Implement gRPC protocol in ClickHouse. #15111 (Vitaly Baranov).
- Allow to use multiple zookeeper clusters. #17070 (fastio).
- Implemented
REPLACE TABLE
andCREATE OR REPLACE TABLE
queries. #18521 (tavplubix). - Implement
UNION DISTINCT
and treat the plainUNION
clause asUNION DISTINCT
by default. Add a settingunion_default_mode
that allows to treat it asUNION ALL
or require explicit mode specification. #16338 (flynn). - Added function
accurateCastOrNull
. This closes #10290. Add type conversions inx IN (subquery)
expressions. This closes #10266. #16724 (Maksim Kita). - IP Dictionary supports
IPv4
/IPv6
types directly. #17571 (vdimir). - IP Dictionary supports key fetching. Resolves #18241. #18480 (vdimir).
- Add
*.zst
compression/decompression support for data import and export. It enables using*.zst
infile()
function andContent-encoding: zstd
in HTTP client. This closes #16791 . #17144 (Abi Palagashvili). - Added
mannWitneyUTest
,studentTTest
andwelchTTest
aggregate functions. RefactoredrankCorr
a bit. #16883 (Nikita Mikhaylov). - Add functions
countMatches
/countMatchesCaseInsensitive
. #17459 (Azat Khuzhin). - Implement
countSubstrings()
/countSubstringsCaseInsensitive()
/countSubstringsCaseInsensitiveUTF8()
(Count the number of substring occurrences). #17347 (Azat Khuzhin). - Add information about used databases, tables and columns in system.query_log. Add
query_kind
andnormalized_query_hash
fields. #17726 (Amos Bird). - Add a setting
optimize_on_insert
. When enabled, do the same transformation for INSERTed block of data as if merge was done on this block (e.g. Replacing, Collapsing, Aggregating...). This setting is enabled by default. This can influence Materialized View and MaterializeMySQL behaviour (see detailed description). This closes #10683. #16954 (Kruglov Pavel). - Kerberos Authenticaiton for HDFS. #16621 (Ilya Golshtein).
- Support
SHOW SETTINGS
statement to show parameters in system.settings.SHOW CHANGED SETTINGS
andLIKE/ILIKE
clause are also supported. #18056 (Jianmei Zhang). - Function
position
now supportsPOSITION(needle IN haystack)
synax for SQL compatibility. This closes #18701. ... #18779 (Jianmei Zhang). - Now we have a new storage setting
max_partitions_to_read
for tables in the MergeTree family. It limits the max number of partitions that can be accessed in one query. A user settingforce_max_partition_limit
is also added to enforce this constraint. #18712 (Amos Bird). - Add
query_id
column tosystem.part_log
for inserted parts. Closes #10097. #18644 (flynn). - Allow create table as select with columns specification. Example
CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1;
. #18060 (Maksim Kita). - Added
arrayMin
,arrayMax
,arrayAvg
aggregation functions. #18032 (Maksim Kita). - Implemented
ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...
query. It creates new table with provided structure and attaches table data from provided directory inuser_files
. #17903 (tavplubix). - Add mutation support for StorageMemory. This closes #9117. #15127 (flynn).
- Support syntax
EXISTS DATABASE name
. #18458 (Du Chuan). - Support builtin function
isIPv4String
&&isIPv6String
like MySQL. #18349 (Du Chuan). - Add a new setting
insert_distributed_one_random_shard = 1
to allow insertion into multi-sharded distributed table without any distributed key. #18294 (Amos Bird). - Add settings
min_compress_block_size
andmax_compress_block_size
to MergeTreeSettings, which have higher priority than the global settings and take effect when they are set. close 13890. #17867 (flynn). - Add support for 64bit roaring bitmaps. #17858 (Andy Yang).
- Extended
OPTIMIZE ... DEDUPLICATE
syntax to allow explicit (or implicit with asterisk/column transformers) list of columns to check for duplicates on. ... #17846 (Vasily Nemkov). - Added functions
toModifiedJulianDay
,fromModifiedJulianDay
,toModifiedJulianDayOrNull
, andfromModifiedJulianDayOrNull
. These functions convert between Proleptic Gregorian calendar date and Modified Julian Day number. #17750 (PHO). - Add ability to use custom TLD list: added functions
firstSignificantSubdomainCustom
,cutToFirstSignificantSubdomainCustom
. #17748 (Azat Khuzhin). - Add support for
PROXYv1
protocol to wrap native TCP interface. Allow quotas to be keyed by proxy-forwarded IP address (applied forPROXYv1
address and forX-Forwarded-For
from HTTP interface). This is useful when you provide access to ClickHouse only via trusted proxy (e.g. CloudFlare) but want to account user resources by their original IP addresses. This fixes #17268. #17707 (alexey-milovidov). - Now clickhouse-client supports opening
EDITOR
to edit commands.Alt-Shift-E
. #17665 (Amos Bird). - Add function
encodeXMLComponent
to escape characters to place string into XML text node or attribute. #17659 (nauta). - Introduce
DETACH TABLE/VIEW ... PERMANENTLY
syntax, so that after restarting the table does not reappear back automatically on restart (only by explicit request). The table can still be attached back using the short syntax ATTACH TABLE. Implements #5555. Fixes #13850. #17642 (filimonov). - Add asynchronous metrics on total amount of rows, bytes and parts in MergeTree tables. This fix #11714. #17639 (flynn).
- Add settings
limit
andoffset
for out-of-SQL pagination: #16176 They are useful for building APIs. These two settings will affect SELECT query as if it is added likeselect * from (your_original_select_query) t limit xxx offset xxx;
. #17633 (hexiaoting). - Provide a new aggregator combinator :
-SimpleState
to buildSimpleAggregateFunction
types via query. It's useful for defining MaterializedView of AggregatingMergeTree engine, and will benefit projections too. #16853 (Amos Bird). - Added
queries-file
parameter forclickhouse-client
andclickhouse-local
. #15930 (Maksim Kita). - Added
query
parameter forclickhouse-benchmark
. #17832 (Maksim Kita). EXPLAIN AST
now support queries other thenSELECT
. #18136 (taiyang-li).
- Added functions for calculation of minHash and simHash of text n-grams and shingles. They are intended for semi-duplicate search. Also functions
bitHammingDistance
andtupleHammingDistance
are added. #7649 (flynn). - Add new data type
Map
. See #1841. First version for Map only supportsString
type of key and value. #15806 (hexiaoting). - Implement alternative SQL parser based on ANTLR4 runtime and generated from EBNF grammar. #11298 (Ivan).
- New IP Dictionary implementation with lower memory consumption, improved performance for some cases, and fixed bugs. #16804 (vdimir).
- Parallel formatting for data export. #11617 (Nikita Mikhaylov).
- LDAP integration: Added
verification_cooldown
parameter in LDAP server connection configuration to allow caching of successful "bind" attempts for configurable period of time. #15988 (Denis Glazachev). - Add
--no-system-table
option forclickhouse-local
to run without system tables. This avoids initialization ofDateLUT
that may take noticeable amount of time (tens of milliseconds) at startup. #18899 (alexey-milovidov). - Replace
PODArray
withPODArrayWithStackMemory
inAggregateFunctionWindowFunnelData
to improvewindowFunnel
function performance. #18817 (flynn). - Don't send empty blocks to shards on synchronous INSERT into Distributed table. This closes #14571. #18775 (alexey-milovidov).
- Optimized read for StorageMemory. #18052 (Maksim Kita).
- Using Dragonbox algorithm for float to string conversion instead of ryu. This improves performance of float to string conversion significantly. #17831 (Maksim Kita).
- Speedup
IPv6CIDRToRange
implementation. #17569 (vdimir). - Add
remerge_sort_lowered_memory_bytes_ratio
setting (If memory usage after remerge does not reduced by this ratio, remerge will be disabled). #17539 (Azat Khuzhin). - Improve performance of AggregatingMergeTree with SimpleAggregateFunction(String) in PK. #17109 (Azat Khuzhin).
- Now the
-If
combinator is devirtualized, andcount
is properly vectorized. It is for this PR. #17043 (Amos Bird). - Fix performance of reading from
Merge
tables over huge number ofMergeTree
tables. Fixes #7748. #16988 (Anton Popov). - Improved performance of function
repeat
. #16937 (satanson). - Slightly improved performance of float parsing. #16809 (Maksim Kita).
- Add possibility to skip merged partitions for
OPTIMIZE TABLE ... FINAL
. #15939 (Kruglov Pavel). - Integrate with fast_float from Daniel Lemire to parse floating point numbers. #16787 (Maksim Kita). It is not enabled, because performance its performance is still lower than rough float parser in ClickHouse.
- Fix max_distributed_connections (affects
prefer_localhost_replica = 1
andmax_threads != max_distributed_connections
). #17848 (Azat Khuzhin). - Adaptive choice of single/multi part upload when sending data to S3. Single part upload is controlled by a new setting
max_single_part_upload_size
. #17934 (Pavel Kovalenko). - Support for async tasks in
PipelineExecutor
. Initial support of async sockets for remote queries. #17868 (Nikolai Kochetov). - Allow to use
optimize_move_to_prewhere
optimization with compact parts, when sizes of columns are unknown. #17330 (Anton Popov).
- Avoid deadlock when executing INSERT SELECT into itself from a table with
TinyLog
orLog
table engines. This closes #6802. This closes #18691. This closes #16812. This closes #14570. #15260 (alexey-milovidov). - Support
SHOW CREATE VIEW name
syntax like MySQL. #18095 (Du Chuan). - All queries of type
Decimal * Float
or vice versa are allowed, including aggregate ones (e.g.SELECT sum(decimal_field * 1.1)
orSELECT dec_col * float_col
), the result type is Float32 or Float64. #18145 (Mike). - Improved minimal Web UI: add history; add sharing support; avoid race condition of different requests; add request in-flight and ready indicators; add favicon; detect Ctrl+Enter if textarea is not in focus. #17293 #17770 (alexey-milovidov).
- clickhouse-server didn't send
close
request to ZooKeeper server. #16837 (alesapin). - Avoid server abnormal termination in case of too low memory limits (
max_memory_usage = 1
/max_untracked_memory = 1
). #17453 (Azat Khuzhin). - Fix non-deterministic result of
windowFunnel
function in case of same timestamp for different events. #18884 (Fuwang Hu). - Docker: Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server Docker images. #19096 (filimonov).
- Asynchronous INSERTs to
Distributed
tables: Two new settings (by analogy with MergeTree family) has been added: -fsync_after_insert
- Do fsync for every inserted. Will decreases performance of inserts. -fsync_directories
- Do fsync for temporary directory (that is used for async INSERT only) after all operations (writes, renames, etc.). #18864 (Azat Khuzhin). SYSTEM KILL
command started to work in Docker. This closes #18847. #18848 (alexey-milovidov).- Expand macros in the zk path when executing
FETCH PARTITION
. #18839 (fastio). - Apply
ALTER TABLE <replicated_table> ON CLUSTER MODIFY SETTING ...
to all replicas. Because we don't replicate such alter commands. #18789 (Amos Bird). - Allow column transformer
EXCEPT
to accept a string as regular expression matcher. This resolves #18685 . #18699 (Amos Bird). - Fix SimpleAggregateFunction in SummingMergeTree. Now it works like AggregateFunction. In previous versions values were summed together regardless to the aggregate function. This fixes #18564 . #8052. #18637 (Amos Bird). Another fix of using
SimpleAggregateFunction
inSummingMergeTree
. This fixes #18676 . #18677 (Amos Bird). - Fixed assertion error inside allocator in case when last argument of function bar is NaN. Now simple ClickHouse's exception is being thrown. This fixes #17876. #18520 (Nikita Mikhaylov).
- Fix usability issue: no newline after exception message in some tools. #18444 (alexey-milovidov).
- Add ability to modify primary and partition key column type from
LowCardinality(Type)
toType
and vice versa. Also add an ability to modify primary key column type fromEnumX
toIntX
type. Fixes #5604. #18362 (alesapin). - Implement
untuple
field access. #18133. #18309 (hexiaoting). - Allow to parse Array fields from CSV if it is represented as a string containing array that was serialized as nested CSV. Example:
"[""Hello"", ""world"", ""42"""" TV""]"
will parse as['Hello', 'world', '42" TV']
. Allow to parse array in CSV in a string without enclosing braces. Example:"'Hello', 'world', '42"" TV'"
will parse as['Hello', 'world', '42" TV']
. #18271 (alexey-milovidov). - Make better adaptive granularity calculation for merge tree wide parts. #18223 (alesapin).
- Now
clickhouse install
could work on Mac. The problem was that there is no procfs on this platform. #18201 (Nikita Mikhaylov). - Better hints for
SHOW ...
query syntax. #18183 (Du Chuan). - Array aggregation
arrayMin
,arrayMax
,arraySum
,arrayAvg
support forInt128
,Int256
,UInt256
. #18147 (Maksim Kita). - Add
disk
to Set and Join storage settings. #18112 (Grigory Pervakov). - Access control: Now table function
merge()
requires current user to haveSELECT
privilege on each table it receives data from. This PR fixes #16964. #18104 #17983 (Vitaly Baranov). - Temporary tables are visible in the system tables
system.tables
andsystem.columns
now only in those session where they have been created. The internal database_temporary_and_external_tables
is now hidden in those system tables; temporary tables are shown as tables with empty database with theis_temporary
flag set instead. #18014 (Vitaly Baranov). - Fix clickhouse-client rendering issue when the size of terminal window changes. #18009 (Amos Bird).
- Decrease log verbosity of the events when the client drops the connection from Warning to Information. #18005 (filimonov).
- Forcibly removing empty or bad metadata files from filesystem for DiskS3. S3 is an experimental feature. #17935 (Pavel Kovalenko).
- Access control:
allow_introspection_functions=0
prohibits usage of introspection functions but doesn't prohibit giving grants for them anymore (the grantee will need to setallow_introspection_functions=1
for himself to be able to use that grant). Similarlyallow_ddl=0
prohibits usage of DDL commands but doesn't prohibit giving grants for them anymore. #17908 (Vitaly Baranov). - Usability improvement: hints for column names. #17112. #17857 (fastio).
- Add diagnostic information when two merge tables try to read each other's data. #17854 (徐炘).
- Let the possibility to override timeout value for running script using the ClickHouse docker image. #17818 (Guillaume Tassery).
- Check system log tables' engine definition grammar to prevent some configuration errors. Notes that this grammar check is not semantical, that means such mistakes as non-existent columns / expression functions would be not found out util the table is created. #17739 (Du Chuan).
- Removed exception throwing at
RabbitMQ
table initialization if there was no connection (it will be reconnecting in the background). #17709 (Kseniia Sumarokova). - Do not ignore server memory limits during Buffer flush. #17646 (Azat Khuzhin).
- Switch to patched version of RocksDB (from ClickHouse-Extras) to fix use-after-free error. #17643 (Nikita Mikhaylov).
- Added an offset to exception message for parallel parsing. This fixes #17457. #17641 (Nikita Mikhaylov).
- Don't throw "Too many parts" error in the middle of INSERT query. #17566 (alexey-milovidov).
- Allow query parameters in UPDATE statement of ALTER query. Fixes #10976. #17563 (alexey-milovidov).
- Query obfuscator: avoid usage of some SQL keywords for identifier names. #17526 (alexey-milovidov).
- Export current max ddl entry executed by DDLWorker via server metric. It's useful to check if DDLWorker hangs somewhere. #17464 (Amos Bird).
- Export asynchronous metrics of all servers current threads. It's useful to track down issues like this. #17463 (Amos Bird).
- Include dynamic columns like MATERIALIZED / ALIAS for wildcard query when settings
asterisk_include_materialized_columns
andasterisk_include_alias_columns
are turned on. #17462 (Ken Chen). - Allow specifying TTL to remove old entries from system log tables, using the
<ttl>
attribute inconfig.xml
. #17438 (Du Chuan). - Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the
interface
column of the tablesystem.query_log
):4
for MySQL, and5
for PostgreSQL, instead of formerly used1
which is now used for the native protocol only. #17437 (Vitaly Baranov). - Fix parsing of SETTINGS clause of the
INSERT ... SELECT ... SETTINGS
query. #17414 (Azat Khuzhin). - Correctly account memory in RadixSort. #17412 (Nikita Mikhaylov).
- Add eof check in
receiveHello
in server to prevent gettingAttempt to read after eof
exception. #17365 (Kruglov Pavel). - Avoid possible stack overflow in bigint conversion. Big integers are experimental. #17269 (flynn).
- Now
set
indices will work withGLOBAL IN
. This fixes #17232 , #5576 . #17253 (Amos Bird). - Add limit for http redirects in request to S3 storage (
s3_max_redirects
). #17220 (ianton-ru). - When
-OrNull
combinator combined-If
,-Merge
,-MergeState
,-State
combinators, we should put-OrNull
in front. #16935 (flynn). - Support HTTP proxy and HTTPS S3 endpoint configuration. #16861 (Pavel Kovalenko).
- Added proper authentication using environment,
~/.aws
andAssumeRole
for S3 client. #16856 (Vladimir Chebotarev). - Add more OpenTelemetry spans. Add an example of how to export the span data to Zipkin. #16535 (Alexander Kuzmenkov).
- Cache dictionaries: Completely eliminate callbacks and locks for acquiring them. Keys are not divided into "not found" and "expired", but stored in the same map during query. #14958 (Nikita Mikhaylov).
- Fix never worked
fsync_part_directory
/fsync_after_insert
/in_memory_parts_insert_sync
(experimental feature). #18845 (Azat Khuzhin). - Allow using
Atomic
engine for nested database ofMaterializeMySQL
engine. #14849 (tavplubix).
- Fix the issue when server can stop accepting connections in very rare cases. #17542 (Amos Bird, alexey-milovidov).
- Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes #18364. #18373 (Amos Bird).
- Fix possible wrong index analysis when the types of the index comparison are different. This fixes #17122. #17145 (Amos Bird).
- Disable write with AIO during merges because it can lead to extremely rare data corruption of primary key columns during merge. #18481 (alesapin).
- Restrict merges from wide to compact parts. In case of vertical merge it led to broken result part. #18381 (Anton Popov).
- Fix possible incomplete query result while reading from
MergeTree*
in case of read backoff (message<Debug> MergeTreeReadPool: Will lower number of threads
in logs). Was introduced in #16423. Fixes #18137. #18216 (Nikolai Kochetov). - Fix use after free bug in
rocksdb
library. #18862 (sundyli). - Fix infinite reading from file in
ORC
format (was introduced in #10580). Fixes #19095. #19134 (Nikolai Kochetov). - Fix bug in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes #18913. #19123 (alesapin).
- Fix startup bug when clickhouse was not able to read compression codec from
LowCardinality(Nullable(...))
and throws exceptionAttempt to read after EOF
. Fixes #18340. #19101 (alesapin). - Restrict
MODIFY TTL
queries forMergeTree
tables created in old syntax. Previously the query succeeded, but actually it had no effect. #19064 (Anton Popov). - Make sure
groupUniqArray
returns correct type for argument of Enum type. This closes #17875. #19019 (alexey-milovidov). - Fix possible error
Expected single dictionary argument for function
if use functionignore
withLowCardinality
argument. Fixes #14275. #19016 (Nikolai Kochetov). - Fix inserting of
LowCardinality
column to table withTinyLog
engine. Fixes #18629. #19010 (Nikolai Kochetov). - Join tries to materialize const columns, but our code wants them in other places. #18982 (Nikita Mikhaylov).
- Disable
optimize_move_functions_out_of_any
because optimization is not always correct. This closes #18051. This closes #18973. #18981 (alexey-milovidov). - Fix possible exception
QueryPipeline stream: different number of columns
caused by merging of query plan'sExpression
steps. Fixes #18190. #18980 (Nikolai Kochetov). - Fixed very rare deadlock at shutdown. #18977 (tavplubix).
- Fix incorrect behavior when
ALTER TABLE ... DROP PART 'part_name'
query removes all deduplication blocks for the whole partition. Fixes #18874. #18969 (alesapin). - Attach partition should reset the mutation. #18804. #18935 (fastio).
- Fix issue with
bitmapOrCardinality
that may lead to nullptr dereference. This closes #18911. #18912 (sundyli). - Fix possible hang at shutdown in
clickhouse-local
. This fixes #18891. #18893 (alexey-milovidov). - Queries for external databases (MySQL, ODBC, JDBC) were incorrectly rewritten if there was an expression in form of
x IN table
. This fixes #9756. #18876 (alexey-milovidov). - Fix *If combinator with unary function and Nullable types. #18806 (Azat Khuzhin).
- Fix the issue that asynchronous distributed INSERTs can be rejected by the server if the setting
network_compression_method
is globally set to non-default value. This fixes #18741. #18776 (alexey-milovidov). - Fixed
Attempt to read after eof
error when trying toCAST
NULL
fromNullable(String)
toNullable(Decimal(P, S))
. Now functionCAST
returnsNULL
when it cannot parse decimal from nullable string. Fixes #7690. #18718 (Winter Zhang). - Fix minor issue with logging. #18717 (sundyli).
- Fix removing of empty parts in
ReplicatedMergeTree
tables, created with old syntax. Fixes #18582. #18614 (Anton Popov). - Fix previous bug when date overflow with different values. Strict Date value limit to "2106-02-07", cast date > "2106-02-07" to value 0. #18565 (hexiaoting).
- Add FixedString data type support for replication from MySQL. Replication from MySQL is an experimental feature. This patch fixes #18450 Also fixes #6556. #18553 (awesomeleo).
- Fix possible
Pipeline stuck
error while usingORDER BY
after subquery withRIGHT
orFULL
join. #18550 (Nikolai Kochetov). - Fix bug which may lead to
ALTER
queries hung after corresponding mutation kill. Found by thread fuzzer. #18518 (alesapin). - Proper support for 12AM in
parseDateTimeBestEffort
function. This fixes #18402. #18449 (vladimir-golovchenko). - Fixed
value is too short
error when executingtoType(...)
functions (toDate
,toUInt32
, etc) with argument of typeNullable(String)
. Now such functions returnNULL
on parsing errors instead of throwing exception. Fixes #7673. #18445 (tavplubix). - Fix the unexpected behaviour of
SHOW TABLES
. #18431 (fastio). - Fix -SimpleState combinator generates incompatible arugment type and return type. #18404 (Amos Bird).
- Fix possible race condition in concurrent usage of
Set
orJoin
tables and selects fromsystem.tables
. #18385 (alexey-milovidov). - Fix filling table
system.settings_profile_elements
. This PR fixes #18231. #18379 (Vitaly Baranov). - Fix possible crashes in aggregate functions with combinator
Distinct
, while using two-level aggregation. Fixes #17682. #18365 (Anton Popov). - Fixed issue when
clickhouse-odbc-bridge
process is unreachable by server on machines with dual IPv4/IPv6 stack; Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes of the odbc-bridge process; Possibly closes #14489. #18278 (Denis Glazachev). - Access control:
SELECT count() FROM table
now can be executed if the user has access to at least single column from a table. This PR fixes #10639. #18233 (Vitaly Baranov). - Access control:
SELECT JOIN
now requires theSELECT
privilege on each of the joined tables. This PR fixes #17654. #18232 (Vitaly Baranov). - Fix key comparison between Enum and Int types. This fixes #17989. #18214 (Amos Bird).
- Replication from MySQL (experimental feature). Fixes #18186 Fixes #16372 Fix unique key convert issue in MaterializeMySQL database engine. #18211 (Winter Zhang).
- Fix inconsistency for queries with both
WITH FILL
andWITH TIES
#17466. #18188 (hexiaoting). - Fix inserting a row with default value in case of parsing error in the last column. Fixes #17712. #18182 (Jianmei Zhang).
- Fix
Unknown setting profile
error on attempt to set settings profile. #18167 (tavplubix). - Fix error when query
MODIFY COLUMN ... REMOVE TTL
doesn't actually remove column TTL. #18130 (alesapin). - Fixed
std::out_of_range: basic_string
in S3 URL parsing. #18059 (Vladimir Chebotarev). - Fix comparison of
DateTime64
andDate
. Fixes #13804 and #11222. ... #18050 (Vasily Nemkov). - Replication from MySQL (experimental feature): Fixes #15187 Fixes #17912 support convert MySQL prefix index for MaterializeMySQL. #17944 (Winter Zhang).
- When server log rotation was configured using
logger.size
parameter with numeric value larger than 2^32, the logs were not rotated properly. This is fixed. #17905 (Alexander Kuzmenkov). - Trivial query optimization was producing wrong result if query contains ARRAY JOIN (so query is actually non trivial). #17887 (sundyli).
- Fix possible segfault in
topK
aggregate function. This closes #17404. #17845 (Maksim Kita). - WAL (experimental feature): Do not restore parts from WAL if
in_memory_parts_enable_wal
is disabled. #17802 (detailyang). - Exception message about max table size to drop was displayed incorrectly. #17764 (alexey-milovidov).
- Fixed possible segfault when there is not enough space when inserting into
Distributed
table. #17737 (tavplubix). - Fixed problem when ClickHouse fails to resume connection to MySQL servers. #17681 (Alexander Kazakov).
- Windows: Fixed
Function not implemented
error when executingRENAME
query inAtomic
database with ClickHouse running on Windows Subsystem for Linux. Fixes #17661. #17664 (tavplubix). - In might be determined incorrectly if cluster is circular- (cross-) replicated or not when executing
ON CLUSTER
query due to race condition whenpool_size
> 1. It's fixed. #17640 (tavplubix). - Fix empty
system.stack_trace
table when server is running in daemon mode. #17630 (Amos Bird). - Exception
fmt::v7::format_error
can be logged in background for MergeTree tables. This fixes #17613. #17615 (alexey-milovidov). - When clickhouse-client is used in interactive mode with multiline queries, single line comment was erronously extended till the end of query. This fixes #13654. #17565 (alexey-milovidov).
- Fix alter query hang when the corresponding mutation was killed on the different replica. Fixes #16953. #17499 (alesapin).
- Fix issue with memory accounting when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. #17496 (alesapin).
- Fix
ORDER BY
with enabled settingoptimize_redundant_functions_in_order_by
. #17471 (Anton Popov). - Fix duplicates after
DISTINCT
which were possible because of incorrect optimization. Fixes #17294. #17296 (li chengxiang). #17439 (Nikolai Kochetov). - Fixed high CPU usage in background tasks of *MergeTree tables. #17416 (tavplubix).
- Fix possible crash while reading from
JOIN
table withLowCardinality
types. Fixes #17228. #17397 (Nikolai Kochetov). - Replication from MySQL (experimental feature): Fixes #16835 try fix miss match header with MySQL SHOW statement. #17366 (Winter Zhang).
- Fix nondeterministic functions with predicate optimizer. This fixes #17244. #17273 (Winter Zhang).
- Fix possible
Unexpected packet Data received from client
error for Distributed queries withLIMIT
. #17254 (Azat Khuzhin). - Fix set index invalidation when there are const columns in the subquery. This fixes #17246. #17249 (Amos Bird).
- clickhouse-copier: Fix for non-partitioned tables #15235. #17248 (Qi Chen).
- Fixed possible not-working mutations for parts stored on S3 disk (experimental feature). #17227 (Pavel Kovalenko).
- Bug fix for funciton
fuzzBits
, related issue: #16980. #17051 (hexiaoting). - Fix
optimize_distributed_group_by_sharding_key
for query with OFFSET only. #16996 (Azat Khuzhin). - Fix queries from
Merge
tables overDistributed
tables with JOINs. #16993 (Azat Khuzhin). - Fix order by optimization with monotonic functions. Fixes #16107. #16956 (Anton Popov).
- Fix incorrect comparison of types
DateTime64
with different scales. Fixes #16655 ... #16952 (Vasily Nemkov). - Fix optimization of group by with enabled setting
optimize_aggregators_of_group_by_keys
and joins. Fixes #12604. #16951 (Anton Popov). - Minor fix in SHOW ACCESS query. #16866 (tavplubix).
- Fix the behaviour with enabled
optimize_trivial_count_query
setting with partition predicate. #16767 (Azat Khuzhin). - Return number of affected rows for INSERT queries via MySQL wire protocol. Previously ClickHouse used to always return 0, it's fixed. Fixes #16605. #16715 (Winter Zhang).
- Fix inconsistent behavior caused by
select_sequential_consistency
for optimized trivial count query and system tables. #16309 (Hao Chen). - Throw error when
REPLACE
column transformer operates on non existing column. #16183 (hexiaoting). - Throw exception in case of not equi-join ON expression in RIGH|FULL JOIN. #15162 (Artem Zuikov).
- Add simple integrity check for ClickHouse binary. It allows to detect corruption due to faulty hardware (bit rot on storage media or bit flips in RAM). #18811 (alexey-milovidov).
- Change
OpenSSL
toBoringSSL
. It allows to avoid issues with sanitizers. This fixes #12490. This fixes #17502. This fixes #12952. #18129 (alexey-milovidov). - Simplify
Sys/V
init script. It was not working on Ubuntu 12.04 or older. #17428 (alexey-milovidov). - Multiple improvements in
./clickhouse install
script. #17421 (alexey-milovidov). - Now ClickHouse can pretend to be a fake ZooKeeper. Currently, storage implementation is just stored in-memory hash-table, and server partially support ZooKeeper protocol. #16877 (alesapin).
- Fix dead list watches removal for TestKeeperStorage (a mock for ZooKeeper). #18065 (alesapin).
- Add
SYSTEM SUSPEND
command for fault injection. It can be used to faciliate failover tests. This closes #15979. #18850 (alexey-milovidov). - Generate build id when ClickHouse is linked with
lld
. It's appeared thatlld
does not generate it by default on my machine. Build id is used for crash reports and introspection. #18808 (alexey-milovidov). - Fix shellcheck errors in style check. #18566 (Ilya Yatsishin).
- Update timezones info to 2020e. #18531 (alesapin).
- Fix codespell warnings. Split style checks into separate parts. Update style checks docker image. #18463 (Ilya Yatsishin).
- Automated check for leftovers of conflict markers in docs. #18332 (alexey-milovidov).
- Enable Thread Fuzzer for stateless tests flaky check. #18299 (alesapin).
- Do not use non thread-safe function
strerror
. #18204 (alexey-milovidov). - Update
anchore/scan-action@main
workflow action (was moved frommaster
tomain
). #18192 (Stig Bakken). - Now
clickhouse-test
does DROP/CREATE databases with a timeout. #18098 (alesapin). - Enable experimental support for Pytest framework for stateless tests. #17902 (Ivan).
- Now we use the fresh docker daemon version in integration tests. #17671 (alesapin).
- Send info about official build, memory, cpu and free disk space to Sentry if it is enabled. Sentry is opt-in feature to help ClickHouse developers. This closes #17279. #17543 (alexey-milovidov).
- There was an uninitialized variable in the code of clickhouse-copier. #17363 (Nikita Mikhaylov).
- Fix one MSan report from #17309. #17344 (Nikita Mikhaylov).
- Fix for the issue with IPv6 in Arrow Flight library. See the comments for details. #16664 (Zhanna).
- Add a library that replaces some
libc
functions to traps that will terminate the process. #16366 (alexey-milovidov). - Provide diagnostics in server logs in case of stack overflow, send error message to clickhouse-client. This closes #14840. #16346 (alexey-milovidov).
- Now we can run almost all stateless functional tests in parallel. #15236 (alesapin).
- Fix corruption in
librdkafka
snappy decompression (was a problem only for gcc10 builds, but official builds uses clang already, so at least recent official releases are not affected). #18053 (Azat Khuzhin). - If server was terminated by OOM killer, print message in log. #13516 (alexey-milovidov).
- PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes #18525. #18526 (alexey-milovidov).
- Minor improvement for path concatenation of zookeeper paths inside DDLWorker. #17767 (Bharat Nallan).
- Allow to reload symbols from debug file. This PR also fixes a build-id issue. #17637 (Amos Bird).