mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
75 KiB
75 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2022 |
2022 Changelog
ClickHouse release v21.12.1.9017-prestable FIXME as compared to v21.11.1.8636-prestable
Backward Incompatible Change
- 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). - Return unquoted string in JSON_VALUE. Closes #27965. #31008 (Kseniia Sumarokova).
- Do not allow direct select for Kafka/RabbitMQ/FileLog. Can be enables by setting
stream_like_engine_allow_direct_select
. Direct select will be not allowed even if enabled by setting in case there is attached materialized view. For Kafka and RabbitMQ direct select 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
). cc @filimonov. #31053 (Kseniia Sumarokova). - A "leader election" mechanism is removed from
ReplicatedMergeTree
, because multiple leaders are supported since 20.6. If you are upgrading from older version and some replica with 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 (Alexander Tokmakov).
New Feature
- Support for Stream Processing. #8331 (vxider).
-
- Added CONSTRAINT ... ASSUME ... (without checking during INSERT) - Added query transformation to CNF (https://github.com/ClickHouse/ClickHouse/issues/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 - Added ability to use the index in queries. #18787 (Nikita Vasilev).
-
- Add Map combinator for
Map
type. * Rename oldsum-, min-, max- Map
for mapped arrays tosum-, min-, max- MappedArrays
. #24539 (Ildus Kurbangaliev).
- Add Map combinator for
- Support
JOIN ON 1 = 1
that have CROSS JOIN semantic, close #25578. #25894 (Vladimir C). - Adding function
getFuzzerData()
to easily fuzz particular functions. This closes #23227. #27526 (Alexey Boykov). - This closes #28774. #28965 (小路).
- We need to implement similar commands in clickhouse-keeper: https://zookeeper.apache.org/doc/r3.4.8/zookeeperAdmin.html#sc_zkCommands. #28981 (JackyWoo).
- Add option to compress logs before writing them to a file using LZ4. Closes #23860. #29219 (Nikolay Degterinsky).
- Introduced window functions: -
exponentialTimeDecayedSum
-exponentialTimeDecayedMax
-exponentialTimeDecayedCount
-exponentialTimeDecayedAvg
which are more effective thanexponentialMovingAverage
for bigger windows. Also more use-cases were covered. #29799 (Vladimir Chebotarev). - Support for PARTITION BY in File, URL, HDFS storages and with INSERT INTO table function. Closes #30273. #30690 (Kseniia Sumarokova).
- support bool data type. #31072 (kevin wan).
- Exposes all GlobalThreadPool configurations to the configuration files. #31285 (Tomáš Hromada).
- Aliyun OSS Storage support. #31286 (cfcz48).
- Allow to print/parse names and types of colums in CustomSeparated input/output format. Add formats CustomSeparatedWithNames/WithNamesAndTypes similar to TSVWithNames/WithNamesAndTypes. #31434 (Kruglov Pavel).
Performance Improvement
- ... Allow to split GraphiteMergeTree rollup rules for plain/tagged metrics (optional rule_type field). #25122 (Michail Safronov).
- Fixing query performance issue in
LiveView
tables. Fixes #30831. #31006 (vzakaznikov). - Improve performance of syncing data to block device. This closes #31181. #31229 (zhanglistar).
- Support parallel formatting for all text formats, except
JSONEachRowWithProgress
andPrettyCompactMonoBlock
. #31489 (Kruglov Pavel). - Improve performance of JSON and XML output formats. #31673 (Alexey Milovidov).
- Speedup avg and sumCount aggregate functions. #31694 (Raúl Marín).
- Speed up count over nullable columns. #31806 (Raúl Marín).
- Speedup query parsing. #31949 (Raúl Marín).
Improvement
- Enable clang
-fstrict-vtable-pointers
,-fwhole-program-vtables
compile options. #20151 (Maksim Kita). - Skipping mutations of different partitions in
StorageMergeTree
. #21326 (Vladimir Chebotarev). - Closes #12552. Allow versioning of aggregate function states. #24820 (Kseniia Sumarokova).
- Add optimizations for constant conditions in JOIN ON, ref #26928. #27021 (Vladimir C).
- Add support for
Identifier
table and database query parameters. Closes #27226. #28668 (Nikolay Degterinsky). - Allow to specify one or any number of PostgreSQL schemas for one MaterializedPostgreSQL database. Closes #28901. Closes #29324. #28933 (Kseniia Sumarokova).
- Make reading from HTTP retriable. Closes #29696. #29894 (Kseniia Sumarokova).
- Add support for parallel reading from multiple files and support globs in
FROM INFILE
clause. #30135 (Filatenkov Artur). -
- Refactor formats TSV, TSVRaw, CSV and JSONCompactEachRow, JSONCompactStringsEachRow, remove code duplication, add base interface for formats with -WithNames and -WithNamesAndTypes suffixes. - Add formats CSVWithNamesAndTypes, TSVRawWithNames, TSVRawWithNamesAndTypes, JSONCompactEachRowWIthNames, JSONCompactStringsEachRowWIthNames, RowBinaryWithNames - Support parallel parsing for formats TSVWithNamesAndTypes, TSVRaw(WithNames/WIthNamesAndTypes), CSVWithNamesAndTypes, JSONCompactEachRow(WithNames/WIthNamesAndTypes), JSONCompactStringsEachRow(WithNames/WIthNamesAndTypes). - Support columns mapping and types checking for RowBinaryWithNamesAndTypes format. - Add setting
input_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't 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 in TSVRaw format, make this format fully compatible with inserting into TSV. - Fix inserting NULLs in LowCardinality(Nullable) wheninput_format_null_as_default
is enabled (previously default values was inserted instead of actual NULLs). - Fix strings deserialization in JSONStringsEachRow/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 setting min_chunk_bytes_for_parallel_parsing is less than bytes in a single row. #30178 (Kruglov Pavel).
- Refactor formats TSV, TSVRaw, CSV and JSONCompactEachRow, JSONCompactStringsEachRow, remove code duplication, add base interface for formats with -WithNames and -WithNamesAndTypes suffixes. - Add formats CSVWithNamesAndTypes, TSVRawWithNames, TSVRawWithNamesAndTypes, JSONCompactEachRowWIthNames, JSONCompactStringsEachRowWIthNames, RowBinaryWithNames - Support parallel parsing for formats TSVWithNamesAndTypes, TSVRaw(WithNames/WIthNamesAndTypes), CSVWithNamesAndTypes, JSONCompactEachRow(WithNames/WIthNamesAndTypes), JSONCompactStringsEachRow(WithNames/WIthNamesAndTypes). - Support columns mapping and types checking for RowBinaryWithNamesAndTypes format. - Add setting
- Avro format works against Kafka. Setting
output_format_avro_rows_in_file
added. #30351 (Ilya Golshtein). - Implement the commands BACKUP and RESTORE for the Log family. #30688 (Vitaly Baranov).
- 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).
- 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).
- added \l, \d, \c aliases like in MySQL. #30876 (Pavel Medvedev).
- Fix
--verbose
option in clickhouse-local interactive mode and allow logging into file. #30881 (Kseniia Sumarokova). - Support
INTERVAL
type inSTEP
clause forWITH FILL
modifier. #30927 (Anton Popov). - 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 settings
merge_tree_min_rows_for_concurrent_read_for_remote_filesystem
andmerge_tree_min_bytes_for_concurrent_read_for_remote_filesystem
. #30970 (Kseniia Sumarokova). - Do not allow to drop a table or dictionary if some tables or dictionaries depend on it. #30977 (Alexander Tokmakov).
- Only grab AlterLock when we do alter command. Let's see if the assumption is correct. #31010 (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). - Cancel vertical merges when partition is dropped. This is a follow-up of https://github.com/ClickHouse/ClickHouse/pull/25684 and https://github.com/ClickHouse/ClickHouse/pull/30996. #31057 (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). - 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 https://github.com/ClickHouse/ClickHouse/pull/20174. Mark as improvements as there are no bug reports and the senario is somehow rare. #31095 (Amos Bird).
- Enable multiline editing in clickhouse-client by default. This addresses #31121 . #31123 (Amos Bird).
- Use DiskPtr instead of OS's file system API in class IDiskRemote in order to get more extendiability. Closes #31117. #31136 (Yangkuan Liu).
- Now every replica will send to client only incremental information about profile events counters. #31155 (Dmitry Novik).
-
- Syntax changed so now backup engine should be set explicitly:
BACKUP ... TO Disk('backups', 'path\')
- Changed the format of backup's metadata, now it's in XML - Backup of a whole database now works. #31178 (Vitaly Baranov).
- Syntax changed so now backup engine should be set explicitly:
- Improved backoff for background cleanup tasks in
MergeTree
. Settingsmerge_tree_clear_old_temporary_directories_interval_seconds
andmerge_tree_clear_old_parts_interval_seconds
moved form users settings to merge tree settings. #31180 (Alexander Tokmakov). - Optimize function
mapContains
to reading of subcolumnkey
with enabled settingsoptimize_functions_to_subcolumns
. #31218 (Anton Popov). - If some obsolete setting is changed show warning in
system.warnings
. #31252 (Alexander Tokmakov). - Optimize function
tupleElement
to reading of subcolumn with enabled settingoptimize_functions_to_subcolumns
. #31261 (Anton Popov). - Initial user's roles are used now to find row policies, see #31080. #31262 (Vitaly Baranov).
- Previously progress was shown only for
numbers
table function, not fornumbers_mt
. Now fornumbers_mt
it is also shown. #31318 (Kseniia Sumarokova). - return fake create query when executing
show create table
on system's tables. #31391 (SuperDJY). - MaterializedMySQL now handles
CREATE TABLE ... LIKE ...
DDL queries. #31410 (Stig Bakken). - Default value of
http_send_timeout
andhttp_receive_timeout
settings changed from 1800 (30 minutes) to 180 (3 minutes). #31450 (Alexander Tokmakov). - Throw an exception if there is some garbage after field in JSONCompactStrings(EachRow) format. #31455 (Kruglov Pavel).
- Fix waiting of the editor during interactive query edition (
waitpid()
returns -1 onSIGWINCH
andEDITOR
andclickhouse-local
/clickhouse-client
works concurrently). #31456 (Azat Khuzhin). - Add
--pager
support forclickhouse-local
. #31457 (Azat Khuzhin). - 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). - 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 (Alexander Tokmakov). - Better exception message when
users.xml
cannot be loaded due to bad password hash. This closes #24126. #31557 (Vitaly Baranov). - 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). - Add bindings for navigating through history (instead of lines/history). #31641 (Azat Khuzhin).
- 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).
- Allow to use named collections configuration for kafka and rabbitmq engines (the same way as for other intgration table engines). #31691 (Kseniia Sumarokova).
- ClickHouse dictionary source support named connections. Closes #31705. #31749 (Kseniia Sumarokova).
- MaterializedMySQL: Fix issue with table named 'table'. #31781 (Håvard Kvålen).
- Recreate system.*_log tables in case of different engine/partition_by. #31824 (Azat Khuzhin).
- Fix the issue that
LowCardinality
ofInt256
cannot be created. #31832 (Alexey Milovidov). - Support PostgreSQL style ALTER MODIFY COLUMN. #32003 (SuperDJY).
- 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). -
- Fix a bug that opentelemetry span log duration is zero at the query level if there's query exception. #32038 (Frank Chen).
- Added ClickHouse
exception
andexception_code
fields to opentelemetry span log. #32040 (Frank Chen). - Allow a user configured
hdfs_replication
parameter for DiskHdfs and StorageHdfs. Closes #32039. #32049 (leosunli). - Allow to write
+
before Float32/Float64 values. #32079 (Kruglov Pavel). -
- returns Content-Type as 'application/json' for
JSONEachRow
format ifoutput_format_json_array_of_rows
is enabled. #32112 (Frank Chen).
- returns Content-Type as 'application/json' for
-
- Set Content-Type in HTTP packets issued from URL engine. #32113 (Frank Chen).
- Now
clickhouse-keeper
refuse to start or apply configuration changes when they contain duplicated IDs or endpoints. Fixes #31339. #32121 (alesapin). - Added
update_field
support forRangeHashedDictionary
,ComplexKeyRangeHashedDictionary
. #32185 (Maksim Kita). - Improve skiping unknown fields with Quoted escaping rule in Template/CustomSeparated formats. Previously we could skip only quoted strings, now we can skip values with any type. #32204 (Kruglov Pavel).
- Use
Content-Type: application/x-ndjson
(http://ndjson.org/) for output formatJSONEachRow
. #32223 (Dmitriy Dorofeev). -
- Improve the operation name of an opentelemetry span. #32234 (Frank Chen).
- Support default expression for storage hdfs and optimize fetching when source is column oriented. #32256 (李扬).
Bug Fix
- Memory amount was incorrectly estimated when ClickHouse is run in containers with cgroup limits. #31157 (Pavel Medvedev).
- Fix SHOW GRANTS when partial revokes are used. This PR fixes #31138. #31249 (Vitaly Baranov).
- Quota limit was not reached, but the limit was exceeded. This PR fixes #31174. #31337 (sunny).
- 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).
Build/Testing/Packaging Improvement
- 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).
- 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). - Fix build snappy error in #30790 Update of contrib/snappy is in https://github.com/google/snappy/pull/145/files. #30796 (李扬).
- Drop support for using Ordinary databases with MaterializedMySQL. #31292 (Stig Bakken).
- Initial support for risc-v. See development/build-cross-riscv for quirks and build command that was tested. #31309 (Vladimir Smirnov).
- Remove hardcoded repository name from CI scripts. #31536 (Constantine Peresypkin).
- Avoid downloading toolchain tarballs for cross-compiling for FreeBSD. #31672 (Alexey Milovidov).
- The script for uploading packages to the artifactory is added. #31748 (Mikhail f. Shiryaev).
- Replaced default ports for clickhouse-keeper internal communication from 44444 to 9234. Fixes #30879. #31799 (alesapin).
- More correct setting up capabilities inside Docker. #31802 (Constantine Peresypkin).
- Revert changes from #28016: archive.ubuntu.com should be faster in general than RU mirror. #31822 (Mikhail f. Shiryaev).
- Remove filesystem path to the build directory from binaries to enable reproducible builds. This needed for #22113. #31838 (Alexey Milovidov).
- Make ClickHouse build fully reproducible (byte identical on different machines). This closes #22113. #31899 (Alexey Milovidov).
-
- Adjust artifactory pusher to a new bucket paths - Use only version or pull request number in bucket, no
0
- Create a function to read github event data. #31952 (Mikhail f. Shiryaev).
- Adjust artifactory pusher to a new bucket paths - Use only version or pull request number in bucket, no
- Build rpm and tgz packages in master and release branches workfolw. #32048 (Mikhail f. Shiryaev).
- Fix broken symlink for sysroot/linux-riscv64/usr/lib. #32071 (Mikhail f. Shiryaev).
Bug Fix (user-visible misbehaviour in official stable release)
- Fix some corner cases with intersect/except. Closes #30803. #30965 (Kseniia Sumarokova).
- Skip max_partition_size_to_drop check in case of ATTACH PARTITION ... FROM and MOVE PARTITION ... #30995 (Amr Alaa).
- Fix bug which broke select queries if they happened after dropping materialized view. Found in #30691. #30997 (Kseniia Sumarokova).
- Using
formatRow
function with not row formats led to segfault. Don't allow to use this function with such formats (because it doesn't make sense). #31001 (Kruglov Pavel). - Fix JSONValue/Query with quoted identifiers. This allows to have spaces in json path. Closes #30971. #31003 (Kseniia Sumarokova).
- Fix possible assert in
hdfs
table function/engine, add test. #31036 (Kruglov Pavel). - 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 StorageMerge with aliases and where (it did not work before at all). Closes #28802. #31044 (Kseniia Sumarokova).
- Rewrite right distributed table in local join. solves #25809. #31105 (abel-cheng).
- 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).
- Remove not like function into RPNElement. #31169 (sundyli).
- Resolve
nullptr
in STS credentials provider for S3. #31409 (Vladimir Chebotarev). - Fix bug with group by and positional arguments. Closes #31280#issuecomment-968696186. #31420 (Kseniia Sumarokova).
- Fix progress for short INSERT SELECT queries. #31510 (Azat Khuzhin).
-
- 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).
- Disable
- Fix invalid generated JSON when only column names contain invalid UTF-8 sequences. #31534 (Kevin Michel).
- All non-x86 builds were broken, because we don't have tests for them. This closes #31417. This closes #31524. #31574 (Alexey Milovidov).
- Fix sparkbars are not aligned, see: #26175#issuecomment-960353867, comment. #31624 (小路).
RENAME TABLE
query worked incorrectly on attempt to rename an DDL dictionary inOrdinary
database, it's fixed. #31638 (Alexander Tokmakov).- Fixed null pointer exception in
MATERIALIZE COLUMN
. #31679 (Vladimir Chebotarev). - 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 (Alexander Tokmakov). - Fix exception on some of the applications of
decrypt
function on Nullable columns. This closes #31662. This closes #31426. #31707 (Alexey Milovidov). - Fixed
there are no such cluster here
error on execution ofON CLUSTER
query if specified cluster name is name ofReplicated
database. #31723 (Alexander Tokmakov). - Fix race in JSONEachRowWithProgress output format when data and lines with progress are mixed in output. #31736 (Kruglov Pavel).
- Fixed rare segfault on concurrent
ATTACH PARTITION
queries. #31738 (Alexander Tokmakov). - 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). - 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 crash with empty result on odbc query. Closes #31465. #31766 (Kseniia Sumarokova).
- Fix crash when function
dictGet
with type is used for dictionary attribute when type isNullable
. Fixes #30980. #31800 (Maksim Kita). - Fix possible assertion
../src/IO/ReadBuffer.h:58: bool DB::ReadBuffer::next(): Assertion '!hasPendingData()' failed.
in TSKV format. #31804 (Kruglov Pavel). - Fix recursive user defined functions crash. Closes #30856. #31820 (Maksim Kita).
- Fix invalid cast of nullable type when nullable primary key is used. This fixes #31075. #31823 (Amos Bird).
- Fix reading from
MergeTree
tables with enableduse_uncompressed_cache
. #31826 (Anton Popov). - Fix a bug about function transform with decimal args. #31839 (Shuai li).
- Fix functions
empty
andnotEmpty
with arguments ofUUID
type. Fixes #31819. #31883 (Anton Popov). - 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 (Alexander Tokmakov). - Fix FileLog engine unnesessary create meta data directory when create table failed. Fix #31962. #31967 (flynn).
- MaterializedMySQL: Fix rare corruption of DECIMAL data. #31990 (Håvard Kvålen).
- Fixed
Directory ... already exists and is not empty
error when detaching part. #32063 (Alexander Tokmakov). - Fix CREATE TABLE of Join Storage with multiply settings contains persistency. Close #31680. #32066 (SuperDJY).
- Fix
CAST
fromNullable
withcast_keep_nullable
(PARAMETER_OUT_OF_BOUND
error before for i.e.toUInt32OrDefault(toNullable(toUInt32(1)))
). #32080 (Azat Khuzhin). - Dictionaries fix cases when
{condition}
does not work for custom database queries. #32117 (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 (Alexander Tokmakov). - XML dictionaries identifiers, used in table create query, can be qualified to
default_database
during upgrade to newer version. Closes #31963. #32187 (Maksim Kita). - Fix parsing error while NaN deserializing for
Nullable(Float)
forQuoted
escaping rule. #32190 (Kruglov Pavel). - Fix window view parser. #32232 (vxider).
- 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 (Alexander Tokmakov). - Fixed crash with SIGFPE in aggregate function
avgWeighted
withDecimal
argument. Fixes #32053. #32303 (Alexander Tokmakov). - 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 the behavior when mutations that have nothing to do are stuck (with enabled setting
empty_result_for_aggregation_by_empty_set
). #32358 (Nikita Mikhaylov).
Build
- support compile in arm machine with parameter "-DENABLE_TESTS=OFF". #31007 (zhanghuajie).
Improvement (changelog entry is not required)
- Make remote_filesystem_read_method=threadpool by default. #31291 (Kseniia Sumarokova).
Imrovement (changelog entry is not required)
- Rename setting value
read_threadpool
tothreadpool
for settingremote_filesystem_read_method
. #31224 (Kseniia Sumarokova).
NO CL ENTRY
- NO CL ENTRY: 'Update permissions-for-queries.md of cn'. #30902 (Laurie Li).
- NO CL ENTRY: 'Make use of untuple alias for untupled columns names prefix'. #30984 (qieqieplus).
- NO CL ENTRY: 'Add banner block for index,company,careers pages'. #31647 (Tom Risse).
- NO CL ENTRY: 'Revert "Fixed null pointer exception in
MATERIALIZE COLUMN
"'. #31692 (Alexander Tokmakov). - NO CL ENTRY: 'Check time limit sending data for global in.'. #31805 (Nikolai Kochetov).
- NO CL ENTRY: 'Fix syntax error: drop comma'. #32095 (Federico Ceratto).
- NO CL ENTRY: 'Revert "Add a test with 20000 mutations in one query"'. #32326 (Nikolai Kochetov).
- NO CL ENTRY: 'Revert "Revert "Add a test with 20000 mutations in one query""'. #32327 (Nikolai Kochetov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Refactor pipeline executor #19587 (Nikolai Kochetov).
- Support NULLS FIRST in KeyCondition #29528 (DimasKovas).
- Integration test improvements #29806 (Ilya Yatsishin).
- Do not allow zero-length reads #30190 (Azat Khuzhin).
- Fix test_backward_compatibility #30950 (Ilya Yatsishin).
- Add stress test to github actions #30952 (alesapin).
- Try smaller blacklist of non parallel integration tests #30963 (Ilya Yatsishin).
- Fix flacky test #30967 (Kseniia Sumarokova).
- Move access-rights source code #30973 (Vitaly Baranov).
- Set output_format_avro_rows_in_file default to 1 #30990 (Kruglov Pavel).
- Remove remaining usages of Y_IGNORE #30993 (Yuriy Chernyshov).
- Return back accidentally removed code #30996 (Nikita Mikhaylov).
- Rename AccessControlManager #30998 (Vitaly Baranov).
- Add fuzzer check to actions #31002 (alesapin).
- Small refactoring in formats #31004 (Kruglov Pavel).
- Changelog for version 21.11 #31014 (Rich Raposa).
- perf: add missing DROP TABLE queries #31028 (Azat Khuzhin).
- Tune perf tests configs/scripts #31029 (Azat Khuzhin).
- Remove metric_log/part_log overrides in tests (enabled by default) #31030 (Azat Khuzhin).
- improve error message while using OFFSET FETCH clause without ORDER BY #31031 (SuperDJY).
- Log size of remapped memory (remap_executable) #31033 (Azat Khuzhin).
- Separate option for enabling fuse syntax for sum, avg, count #31035 (Vladimir C).
- Add integration test on top of github actions #31045 (alesapin).
- Fix intersecting parts in
parts_to_do
2 #31060 (Alexander Tokmakov). - perf: switch *_log tables to Memory engine (attempt to reduce cache misses) #31063 (Azat Khuzhin).
- Add new employee photos to team #31084 (Cody Baker).
- Update button widths for responsive sizing #31085 (Cody Baker).
- Add overflow to benchmark tables #31086 (Cody Baker).
- Remove padding below greenhouse iframe #31087 (Cody Baker).
- Crb update case study cards #31088 (Cody Baker).
- Fix threadpool read for remote disks #31112 (Kseniia Sumarokova).
- Doc build: remove single.md #31118 (lehasm).
- remove unnecessary assert in StorageFileLog #31119 (nauta).
- Add lambda for approve #31139 (alesapin).
- Do not include unnecessary experimental/type_traits #31147 (Yuriy Chernyshov).
- Disable optimize_syntax_fuse_functions by default #31149 (Vladimir C).
- Add pvs studio to actions #31156 (alesapin).
- Add cherry-pick on github actions #31158 (alesapin).
- correct disk space calculations #31159 (Alexandre Snarskii).
- Fix typo #31164 (Kseniia Sumarokova).
- Update text on thank you page #31166 (Cody Baker).
- Add unit tests to CI #31175 (alesapin).
- Debug cherry-pick CI #31177 (alesapin).
- Update docker_compose_postgres.yml #31179 (Kseniia Sumarokova).
- Fix integration tests #31223 (Kseniia Sumarokova).
- Relax test 02026_storage_filelog_largefile.sh #31225 (Kseniia Sumarokova).
- Fix typo in USE_MYSQL check #31226 (Yuriy Chernyshov).
- Download ccache if not exists in builds #31227 (alesapin).
- Disable fuzzer builds in CI #31244 (Nikita Mikhaylov).
- Add flaky check to new CI #31248 (alesapin).
- Fix test #31250 (Kseniia Sumarokova).
- move InputCreatorFunc to InputCreator #31258 (Alex Cao).
- Print warning during old directories cleanup in MergeTree only if it is old #31259 (Azat Khuzhin).
- Cleanup extern ProfileEvents/CurrentMetrics and add a style check #31260 (Azat Khuzhin).
- Fix and refactor WriteBiffer-s a little #31265 (Kruglov Pavel).
- Followup to ccache build #31287 (alesapin).
- Add compatibility check #31294 (alesapin).
- Add assertions to ZooKeeperLock #31295 (Alexander Tokmakov).
- Add split build check #31299 (alesapin).
- Remove strange code from mutations #31300 (Alexander Tokmakov).
- Disable integration tests in new CI #31301 (alesapin).
- Merging #31081 #31305 (Alexander Tokmakov).
- Debug rabbitmq tests #31316 (Kseniia Sumarokova).
- Fix possible data-race in case of query cancellation with async_socket_for_remote #31317 (Azat Khuzhin).
- Improve fuzzer report in case of fuzzer killed #31324 (Azat Khuzhin).
- Fix check in async buffer #31325 (Kseniia Sumarokova).
- Add Blog Post for 21.11 Release Update #31326 (Cody Baker).
- Add blog post for Moscow meetup #31327 (Cody Baker).
- Do not try to resolve temporary tables from global context #31333 (Azat Khuzhin).
- BloomFilter index check fix #31334 (Maksim Kita).
- Remove some copypaste from CI #31340 (alesapin).
- Fix test_kafka_insert_avro by pinning avro version #31387 (Azat Khuzhin).
- Fix QueryScope in MaterializedMySQLSyncThread #31392 (Azat Khuzhin).
- Check stderr is writable before reopining it (to avoid losing errors) #31393 (Azat Khuzhin).
- Remove thread_local std::string #31400 (Alexey Milovidov).
- Fix client #31403 (Kseniia Sumarokova).
- Remove excessive debug info from the log message in DDLWorker #31406 (Alexander Tokmakov).
- Turn on more CI checks #31413 (alesapin).
- Update description for webinar calendar links #31433 (Cody Baker).
- Trying to debug integration tests #31443 (alesapin).
- Try increase
snapshot_distance
for functional tests #31448 (Alexander Tokmakov). - Minor improvement for test_replicated_fetches_bandwidth #31451 (Alexander Tokmakov).
- Merging #18787 (Constraints) #31476 (Anton Popov).
- Crb support page #31490 (Cody Baker).
- Add new team members to company page #31491 (Cody Baker).
- [ci] whitelist codyrobert #31492 (Ivan Blinkov).
- Reapply style changes to hardware page #31506 (Cody Baker).
- Split row policy and quota headers #31509 (Vitaly Baranov).
- Do not clean iptables rules in session-scope fixture #31527 (Alexander Tokmakov).
- Push tests results to CI database #31540 (alesapin).
- Remove strange multimap for mutations in StorageMergeTree #31542 (Alexander Tokmakov).
- Remove duplicated not compressed logs from CI (actions) #31544 (Azat Khuzhin).
- Fix 02010_lc_native flakiness (Query with id = 123456 is already running) #31556 (Azat Khuzhin).
- Make 01114_database_atomic more stable in debug builds #31564 (Azat Khuzhin).
- Fix MySQLWire format (this will also fix performance tests) #31565 (Azat Khuzhin).
- get Build ID via Section headers first #31566 (Ilya Golshtein).
- Try to debug expired sessions #31584 (Alexander Tokmakov).
- Fix cleanup in integration tests #31605 (Vitaly Baranov).
- Stop all periodic reloading of all the configuration files on shutdown earlier #31607 (Vitaly Baranov).
- Fix build requirements for unit tests #31617 (alesapin).
- Add workflow run for release branches #31618 (alesapin).
- Other event for release PR #31619 (alesapin).
- Trying push event again #31623 (alesapin).
- Add github actions for master #31629 (alesapin).
- Fix master yml #31630 (alesapin).
- fix kerberized_hadoop image #31637 (Constantine Peresypkin).
- [ci] add flickerbox-tom to whitelist #31651 (Ivan Blinkov).
- Try to fix possible data race in RemoteQueryExecutorReadContext #31652 (Kruglov Pavel).
- Integration tests flaky check and small fixes #31654 (alesapin).
- Update base64 library #31677 (Alexey Milovidov).
- fix typo #31678 (flynn).
- Try fix OOMs with TSAN #31685 (Alexander Tokmakov).
- Fix workflow in master #31688 (alesapin).
- Add perf test for writing valid UTF8 #31695 (Kruglov Pavel).
- hdfs disable stderr logging #31703 (Kseniia Sumarokova).
- Fix possible Logical error: Cannot write to finalized buffer #31712 (Kruglov Pavel).
- 02050: Use CLICKHOUSE_TMP and delete files when finished #31713 (Raúl Marín).
- Make 02112_with_fill_interval independent of the server timezone #31714 (Raúl Marín).
- 02010_lc_native: Generate a new id for each query #31720 (Raúl Marín).
- 00623_replicated_truncate_table_zookeeper_long: Wait for truncate in replicas #31721 (Raúl Marín).
- Try to push data into another ci database. #31724 (Nikolai Kochetov).
- Remove OpenCL completely #31744 (Timur Magomedov).
- Fix diff for backports. #31765 (alesapin).
- Fail fasttest, builds and functional checks if some tests was not successful #31767 (alesapin).
- Improve how queries are output in the performance dashboard #31780 (Raúl Marín).
- Use version from git describe in builds #31782 (alesapin).
- Fix stylecheck for tests/ci/push_to_artifactory.py #31798 (Mikhail f. Shiryaev).
- Shorter stress and fuzzer tests #31803 (alesapin).
- Fix oss-fuzz build #31818 (Nikita Mikhaylov).
- Fix check in approve lambda #31821 (alesapin).
- Cover query_views_log #31825 (Azat Khuzhin).
- Forbid files that differ only by character case #31834 (Alexey Milovidov).
- Added ubsan suppression for libprotobuf-mutator #31835 (Nikita Mikhaylov).
- Update 01155_rename_move_materialized_view.sql #31849 (Alexander Tokmakov).
- Fix style. #31850 (Nikolai Kochetov).
- Minor improvements to DUMP macro #31858 (Vasily Nemkov).
- Get rid of build numbers and simplify builds paths in S3 #31861 (alesapin).
- Support toUInt8/toInt8 for if constant condition optimization. #31866 (Nikolai Kochetov).
- Added -no-sanitize=unsigned-integer-overflow build flag #31881 (Nikita Mikhaylov).
- Fix typos #31886 (Anton Popov).
- Try to fix flacky test. #31889 (Nikolai Kochetov).
- Reduce the files that depend on parser headers #31896 (Raúl Marín).
- Fix magic_enum for debug helpers (fixes build w/ USE_DEBUG_HELPERS) #31922 (Azat Khuzhin).
- Remove some trash from build #31923 (Alexey Milovidov).
- Add json type to changes in documentation #31926 (alesapin).
- fix some broken links #31948 (Ramazan Polat).
- Kill container in integration tests if it's already running #31950 (alesapin).
- Drop libc-headers #31951 (Raúl Marín).
- Give some love to macOS platform #31957 (Denis Glazachev).
- Fix segfault in MaterializedMySQL #31960 (Alexander Tokmakov).
- Fix docs check #31980 (alesapin).
- Make stress tests slightly more stable #31985 (alesapin).
- Add rest functional tests to CI #31987 (alesapin).
- Add special builds to CI #31991 (alesapin).
- Run less tests for backport branches #31992 (alesapin).
- Fix race in ParallelFormattingOutputFormat constructor #32004 (Nikita Mikhaylov).
- Fix build on master #32009 (alesapin).
- Fix style-check for ProfileEvents checking #32033 (Azat Khuzhin).
- Provide clickhouse binary w/o debug symbols (stripped) in fasttest #32036 (Azat Khuzhin).
- Minor fixes for
StorageMergeTree
#32037 (Vladimir Chebotarev). - Remove most of old checks #32041 (alesapin).
- Remove fast test from master #32042 (alesapin).
- Better scripts for runners #32043 (alesapin).
- Fix force tests label #32044 (alesapin).
- Don't run checks for label event #32046 (alesapin).
- Fix flaky test 00925 #32050 (Alexander Tokmakov).
- Cancel redundant checks with lambda #32051 (alesapin).
- Fix flaky integration test for MaterializedMySQL CREATE TABLE LIKE #32052 (Stig Bakken).
- Use functional test group for tests with thread sanitizer #32062 (alesapin).
- Add ability for lightweight checks rerun #32064 (alesapin).
- Increase length of random database in clickhouse-test #32094 (Azat Khuzhin).
- make looping in H3 funcs uniform #32110 (Bharat Nallan).
- Remove PVS check from master #32114 (alesapin).
- Fix flaky keeper whitelist test #32115 (alesapin).
- Fix flacky test test_executable_storage_input #32118 (Maksim Kita).
- Fix data race in
removePartAndEnqueueFetch(...)
#32119 (Alexander Tokmakov). - Move fuzzers and unit tests to another group #32120 (alesapin).
- Add a test with 20000 mutations in one query #32122 (Nikolai Kochetov).
- Change test 02117_custom_separated_with_names_and_types #32123 (Kruglov Pavel).
- Use seq_cst semantic for MergeTreeBackgroundExecutor mertic. #32125 (Nikolai Kochetov).
- Remove 02116_global_in_time_limit. #32126 (Nikolai Kochetov).
- fixing postgres tests #32129 (Kseniia Sumarokova).
- Small improvements in lambda code #32155 (alesapin).
- Update featured image for 21.11 release blog post #32156 (Cody Baker).
- Fix CI #32159 (alesapin).
- tests/ci: do not compress logs twice #32162 (Azat Khuzhin).
- clickhouse-test: do not use random generator with shared state #32163 (Azat Khuzhin).
- Fix stress tests #32164 (Azat Khuzhin).
- Fix QueryProfiler (query_profiler_{cpu,real}_time_period_ns) reset #32165 (Azat Khuzhin).
- MaterializedMySQL support VARBINARY type #32173 (zzsmdfj).
- perf: fix waiting of the server after running tests #32174 (Azat Khuzhin).
- Better output for some actions #32175 (alesapin).
- Use ccache in fast test #32177 (alesapin).
- Fix window view tests #32178 (Kseniia Sumarokova).
- Function accurateCastOrDefault remove separate branch #32184 (Maksim Kita).
- Add test for #32186 #32203 (Raúl Marín).
- Fix uncaught exception in DatabaseLazy #32206 (Alexander Tokmakov).
- Update ASTCreateQuery.cpp #32208 (Kseniia Sumarokova).
- Fix flacky fileLog test (probably) #32209 (Kseniia Sumarokova).
- Fix jemalloc under osx #32219 (Azat Khuzhin).
- Add missing timezones to some tests #32222 (Kseniia Sumarokova).
- Fix versioning of aggregate functions (fixes performance tests) #32236 (Azat Khuzhin).
- Disable window view tests temporarily because still flacky #32257 (Kseniia Sumarokova).
- Fix typo in tupleToNameValuePairs doc #32262 (Vladimir C).
- Fix possible Pipeline stuck in case of StrictResize processor. #32270 (Nikolai Kochetov).
- Fix possible crash in DataTypeAggregateFunction #32287 (Nikita Mikhaylov).
- Update backport.py #32323 (Kseniia Sumarokova).
- Fix graphite-bench build #32351 (Nikita Mikhaylov).
- Revert "graphite: split tagged/plain rollup rules (for merges perfoma... #32376 (Mikhail f. Shiryaev).
- Another attempt to fix unit test Executor::RemoveTasksStress #32390 (Nikita Mikhaylov).