mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 17:44:23 +00:00
55 KiB
55 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2022 |
2022 Changelog
ClickHouse release v21.4.2.10-prestable FIXME as compared to v21.3.1.6185-prestable
Backward Incompatible Change
- 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). - 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). - 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 and 22:00:00..23:59:59. This behaviour is more suited for practical needs. This closes #9510. #22060 (Alexey Milovidov).
New Feature
- Extended range of
DateTime64
to properly support dates from year 1925 to 2283. Improved support ofDateTime
around zero date (1970-01-01
). ... #9404 (Vasily Nemkov). -
- Added support of Kerberos authentication for preconfigured users and HTTP requests (GSS-SPNEGO). #14995 (Denis Glazachev).
- Zero-copy replication for ReplicatedMergeTree over S3 storage. #16240 (ianton-ru).
- Support
dictHas
function forRangeHashedDictionary
. Fixes #6680. #19816 (Maksim Kita). - Supports implicit key type conversion for JOIN. Closes #18567. #19885 (Vladimir C).
- Allow customizing timeouts for http connections used for replication independently from other http timeouts. #20088 (nvartolomei).
- Added async update in
ComplexKeyCache
,SSDCache
,SSDComplexKeyCache
dictionaries. Added support for Nullable type inCache
,ComplexKeyCache
,SSDCache
,SSDComplexKeyCache
dictionaries. Added support for multiple attributes fetch withdictGet
,dictGetOrDefault
functions. Fixes #21517. #20595 (Maksim Kita). - Added
Grant,
Revoke
andSystem
values ofquery_kind
column for corresponding queries insystem.query_log
... #21102 (Vasily Nemkov). - Added new SQL command ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'. #21142 (Pavel Kovalenko).
- Added ExecutablePool dictionary source. Close #14528. #21321 (Maksim Kita).
- 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 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 option
--backslash
for clickhouse-format, which can add a backslash at the end of each line of the formatted query. #21494 (flynn). - Add new optional clause GRANTEES for CREATE/ALTER USER commands:. #21641 (Vitaly Baranov).
- Add
ctime
option tozookeeper-dump-tree
. It allows to dump node creation time. #21842 (Ilya). - Functions 'dictGet', 'dictHas' use current database name if it is not specified for dictionaries created with DDL. Closes #21632. #21859 (Maksim Kita).
- Support
Nullable
type forPolygonDictionary
attribute. #21890 (Maksim Kita). - Added table function
dictionary
. It works the same way asDictionary
engine. Closes #21560. #21910 (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
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).
Performance Improvement
- Support parallel parsing for
CSVWithNames
andTSVWithNames
formats. This closes #21085. #21149 (Nikita Mikhaylov). - Improved performance by replacing
memcpy
to another implementation. This closes #18583. #21520 (Alexey Milovidov). - Supported parallel formatting in clickhouse-local and everywhere else. #21630 (Nikita Mikhaylov).
- 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). - Faster
GROUP BY
with smallmax_rows_to_group_by
andgroup_by_overflow_mode='any'
. #21856 (Nikolai Kochetov). - 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). - 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). - 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).
Improvement
- 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). - Improve performance of aggregation in order of sorting key (with enabled setting
optimize_aggregation_in_order
). #19401 (Anton Popov). - MaterializeMySQL: add minmax skipping index for _version column. #20382 (Stig Bakken).
- Do not create empty parts on INSERT when
optimize_on_insert
setting enabled. Fixes #20304. #20387 (Kruglov Pavel). -
- Support more cases to rewrite
CROSS JOIN
toINNER JOIN
. #20392 (Vladimir C).
- Support more cases to rewrite
- MaterializeMySQL: Attempt to reconnect to MySQL if the connection is lost. #20961 (Håvard Kvålen).
- Improve support of integer keys in data type
Map
. #21157 (Anton Popov). - 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 format
ASTInsertQuery
with data . #21311 (flynn). - Age and Precision 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).
- Add setting
optimize_skip_unused_shards_limit
to limit the number of sharding key values foroptimize_skip_unused_shards
. #21512 (Azat Khuzhin). - Add
last_error_time
/last_error_message
/last_error_stacktrace
/remote
columns forsystem.errors
. #21529 (Azat Khuzhin). - 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).
-
- 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 (Alexander Tokmakov).
- Propagate query and session settings for distributed DDL queries. Set
- Update clusters only if their configurations were updated. #21685 (Kruglov Pavel).
- Support replicas priority for postgres dictionary source. #21710 (Kseniia Sumarokova).
- Closes #21701. Support non-default table schema for postgres storage/table-function. #21711 (Kseniia Sumarokova).
- Better formatting for
Array
andMap
data types in Web UI. #21798 (Alexey Milovidov). - 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).
- Add connection pool for PostgreSQL table/database engine and dictionary source. Should fix #21444. #21839 (Kseniia Sumarokova).
- Add profile event HedgedRequestsChangeReplica, change read data timeout from sec to ms. #21886 (Kruglov Pavel).
- Support
RANGE OFFSET
frame 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). - Show path to data directory of
EmbeddedRocksDB
tables in system tables. #21903 (Alexander Tokmakov). - Supported
replication_alter_partitions_sync=1
setting for moving partitions from helping table to destination. Decreased default timeouts. Fixes #21911. #21912 (jasong). - 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). -
- Add option strict_increase to windowFunnel function to calculate each event once (resolve #21835). #22025 (Vladimir C).
- Added case insensitive aliases for
CONNECTION_ID()
andVERSION()
functions. This fixes #22028. #22042 (Eugene Klimov). - Update used version of simdjson to 0.9.1. This fixes #21984. #22057 (Vitaly Baranov).
- Convert
system.errors.stack_trace
fromString
intoArray(UInt64)
(This should decrease overhead for the errors collecting). #22058 (Azat Khuzhin). - 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). - Added possibility to migrate existing S3 disk to the schema with backup-restore capabilities. #22070 (Pavel Kovalenko).
- Add case-insensitive history search/navigation and subword movement features to clickhouse-client. #22105 (Amos Bird).
- Add
current_database
column tosystem.processes
table. It contains the current database of the query. #22365 (Alexander Kuzmenkov). - Fix MSan report for function
range
withUInt256
argument (support for large integers is experimental). This closes #22157. #22387 (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). - 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).
Bug Fix
- Fixed open behavior of remote host filter in case when there is
remote_url_allow_hosts
section in configuration but no entries there. ⚠️ please add a note about potential issue when upgrading - @alexey-milovidov. #20058 (Vladimir Chebotarev). force_drop_table
flag didn't work forMATERIALIZED VIEW
, it's fixed. Fixes #18943. #20626 (Alexander Tokmakov).- fix official website documents which introduced cluster secret feature. #21331 (Chao Ma).
- Fix receive and send timeouts and non-blocking read in secure socket. #21429 (Kruglov Pavel).
- Fix Avro format parsing for Kafka. Fixes #21437. #21438 (Ilya Golshtein).
- Fixed race on SSL object inside SecureSocket in Poco. #21456 (Nikita Mikhaylov).
- Fix that S3 table holds old credentials after config update. #21457 (Pervakov Grigorii).
- Fix table function
clusterAllReplicas
returns wrong_shard_num
. close #21481. #21498 (flynn). - The
::poll()
returnrc == 1
, it could be a request or it could be a response. #21544 (小路). - 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 possible error
Cannot find column
whenoptimize_skip_unused_shards
is enabled and zero shards are used. #21579 (Azat Khuzhin). std::terminate
was called if there is an error writing data into s3. #21624 (Vladimir C).- 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 C).
- Fix fsync_part_directory for horizontal merge. #21642 (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). - Add type conversion for StorageJoin (previously led to SIGSEGV). #21646 (Azat Khuzhin).
- Start accepting connections after DDLWorker and dictionaries initialization. #21676 (Azat Khuzhin).
- Fix SIGSEGV on not existing attributes from ip_trie with access_to_key_from_attributes. #21692 (Azat Khuzhin).
- Fix function
arrayElement
with typeMap
for constant integer arguments. #21699 (Anton Popov). - Fix concurrent
OPTIMIZE
andDROP
forReplicatedMergeTree
. #21716 (Azat Khuzhin). - Fix bug for ReplicatedMerge table engines when
ALTER MODIFY COLUMN
query doesn't change the type of decimal column if its size (32 bit or 64 bit) doesn't change. #21728 (alesapin). - Reverted S3 connection pools. #21737 (Vladimir Chebotarev).
- Fix adding of parts with already existing in destination table names in query
MOVE PARTITION TO TABLE
with non-replicatedMergeTree
tables. #21760 (ygrek). - Fix scalar subquery index analysis. This fixes #21717 , which was introduced in https://github.com/ClickHouse/ClickHouse/pull/18896 . #21766 (Amos Bird).
- Fix possible crashes in aggregate functions with combinator Distinct, while using two-level aggregation. This is a follow-up fix of https://github.com/ClickHouse/ClickHouse/pull/18365 . Can only reproduced in production env. No test case available yet. cc @CurtizJ. #21818 (Amos Bird).
- Better error handling and logging in WriteBufferFromS3. #21836 (Pavel Kovalenko).
- Fix incorrect query result (and possible crash) which could happen when
WHERE
orHAVING
condition is pushed beforeGROUP BY
. Fixes #21773. #21841 (Nikolai Kochetov). - Fix deadlock in first catboost model execution. Closes #13832. #21844 (Kruglov Pavel).
- Backported in #22625: Fix 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). - 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 reading the HTTP POST request with "multipart/form-data" content type. #21936 (Ivan).
- Prevent hedged connections overlaps (
Unknown packet 9 from server
error). #21941 (Azat Khuzhin). - Reverted #15454 that may cause significant increase in memory usage while loading external dictionaries of hashed type. This closes #21935. #21948 (Maksim Kita).
- 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). - The function
decrypt
was lacking a check for the minimal size of data encrypted in AEAD mode. This closes #21897. #22064 (Alexey Milovidov). - Docker entrypoint: avoid chown of
.
in case whenLOG_PATH
is empty. Closes #22100. #22102 (filimonov). - Disable
async_socket_for_remote
/use_hedged_requests
for buggy linux kernels. #22109 (Azat Khuzhin). - Fix waiting for
OPTIMIZE
andALTER
queries forReplicatedMergeTree
table engines. Now the query will not hang when the table was detached or restarted. #22118 (alesapin). - Fix the background thread pool name. #22122 (fastio).
- Fix error
Invalid number of rows in Chunk
inJOIN
withTOTALS
andarrayJoin
. Closes #19303. #22129 (Vladimir C). - Fix docker entrypoint in case
http_port
is not in the config. #22132 (Ewout). - Fix uncaught exception in InterserverIOHTTPHandler. #22146 (Azat Khuzhin).
- Use finalize() over next() for nested writers. #22147 (Azat Khuzhin).
- Fix query cancellation with
use_hedged_requests=0
andasync_socket_for_remote=1
. #22183 (Azat Khuzhin). - Fix exception which may happen when
SELECT
has constantWHERE
condition and source table has columns which names are digits. #22270 (LiuNeng). - Now clickhouse will not throw
LOGICAL_ERROR
exception when we try to mutate the already covered part. Fixes #22013. #22291 (alesapin). - Backported in #22541: Do not limit HTTP chunk size. Fixes #21907. #22322 (Ivan).
- Fixed bug in S3 zero-copy replication for hybrid storage. #22378 (ianton-ru).
- Backported in #22532: 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). - 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).
- Backported in #22519: Remove socket from epoll before cancelling packet receiver in HedgedConnections to prevent possible race. I hope it fixes #22161. #22443 (Kruglov Pavel).
- Backported in #22617: Fix ClickHouseDictionarySource configuration loop. Closes #14314. #22479 (Maksim Kita).
- Backported in #22558: Fix bug in partial merge join with
LowCardinality
. Close #22386, close #22388. #22510 (Vladimir C). - Backported in #22559: Fix 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). - Backported in #22577: Fix UB by unlocking the rwlock of the TinyLog from the same thread. #22560 (Azat Khuzhin).
- Backported in #22649: Avoid UB in *Log engines for rwlock unlock due to unlock from another thread. #22583 (Azat Khuzhin).
- Backported in #22652: Try flush write buffer only if it is initialized. Fixes segfault when client closes connection very early #22579. #22591 (nvartolomei).
- Backported in #22680: Fix LOGICAL_ERROR for Log with nested types w/o columns in the SELECT clause. #22654 (Azat Khuzhin).
Build/Testing/Packaging Improvement
- Fix macOS shared lib build. #20184 (nvartolomei).
- Fixing LDAP authentication performance test by removing assertion. #21507 (vzakaznikov).
- Updating docker/test/testflows/runner/dockerd-entrypoint.sh to use Yandex dockerhub-proxy. #21551 (vzakaznikov).
- 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 TestFlows to 1.6.74. #21673 (vzakaznikov).
- Add jepsen tests for NuKeeper. #21677 (alesapin).
- remove decode method with python3. #21832 (kevin wan).
- Allow to use clang-tidy with release builds by enabling assertions if it is used. #21914 (Alexey Milovidov).
- Introduce 2 arguments for clickhouse-server image Dockerfile: deb_location & single_binary_location. #21977 (filimonov).
- Add
tzdata
to Docker containers because reading ORC formats requires it. This closes #14156. #22000 (Alexey Milovidov). - Enable status check for SQLancer CI run. #22015 (Ilya Yatsishin).
- try fix #22289 https://clickhouse-test-reports.s3.yandex.net/22289/c71da4a5c8e655f4bdfaa33b92ab022b97dfdf1a/integration_tests_(asan).html#fail1 MySQL is started only once with MaterializeMySQL integration test. #22341 (Winter Zhang).
- Fixed compiling on ppc64le and use the correct instruction pointer register on ppc64le. #22430 (Kfir Itzhak).
- Fix ClickHouse's config embedding and cctz's timezone embedding on ppc64le. ... #22445 (Kfir Itzhak).
- Enable building with jemalloc on ppc64le ... #22447 (Kfir Itzhak).
- Fix Fedora\RHEL\CentOS not finding libclang_rt.builtins on ppc64le ... #22458 (Kfir Itzhak).
- Fix CMake error about internal CMake variable CMAKE_ASM_COMPILE_OBJECT not set on ppc64le ... #22469 (Kfir Itzhak).
- Fix compiling boost on ppc64le ... #22474 (Kfir Itzhak).
- Enable compiling on ppc64le with Clang ... #22476 (Kfir Itzhak).
- Re-enable the S3 (AWS) library on aarch64 ... #22484 (Kfir Itzhak).
Other
- Update tests for hedged requests. #21998 (Kruglov Pavel).
- Don't set the same timeouts in ReadBufferFromPocoSocket/WriteBufferFromPocoSocket in nextImpl because it causes a race. #22343 (Kruglov Pavel).
NO CL ENTRY
- NO CL ENTRY: 'Improve the translation of
query_log.md
in Chinese documents'. #21729 (Pysaoke). - NO CL ENTRY: 'Update gui.md: add SeekTable'. #21768 (Vitaliy Fedorchenko).
- NO CL ENTRY: 'Flatten libcpuid PEERDIRs'. #22078 (Yuriy Chernyshov).
- NO CL ENTRY: 'Revert "quick fix for broken resolution of apt.llvm.org on Yandex infra"'. #22374 (alesapin).
NOT FOR CHANGELOG / INSIGNIFICANT
- Add soft task timeout for Intergation tests #16608 (Alexander Tokmakov).
- Merging geometry functions #19257 (Nikita Mikhaylov).
- Remove outdated suppressions, part 2 #19496 (Alexey Milovidov).
- Move some CI-related scripts to github #20946 (Alexander Tokmakov).
- fix #21170 #21182 (Tachikoma).
- Add more tests for quota consumption by the SHOW statement #21190 (Vitaly Baranov).
- Save packed keys for GROUP BY with multiple fixed size keys #21196 (Nikolai Kochetov).
- Refine MergeTreeData::loadDataParts to not parse format version file and detached directory #21351 (Fuwang Hu).
- Persistent nukeeper snapshot storage #21425 (alesapin).
- Fix logging for optimize_aggregation_in_order=1 (with small max_block_size) #21436 (Azat Khuzhin).
- Adjust prewhere_with_row_level_filter performance test #21442 (Denis Glazachev).
- Refactor actions dag #21459 (Nikolai Kochetov).
- add query formatting idempotence check to fuzzer #21466 (Alexander Kuzmenkov).
- Fix heap-buffer-overflow in highlighting multi-line comments #21492 (Azat Khuzhin).
- Fix global stop merges in test #21508 (alesapin).
- DirectDictionary updated #21513 (Maksim Kita).
- Avoid processing optimize_skip_unused_shards twice #21526 (Azat Khuzhin).
- DOCSUP-7197: Escaped Unicode replaced with symbols #21530 (lehasm).
- Pod array left pad not multiple of element crash fix #21532 (Maksim Kita).
- ShellCommand waitpid eintr signal fix #21546 (Maksim Kita).
- Refactoring of data types serialization #21562 (Anton Popov).
- fix a typo in window functions frame #21572 (Alexander Kuzmenkov).
- Added specialized CacheDictionaryStorage #21573 (Maksim Kita).
- [RFC] Union merge for arcadia_skip_list.txt to avoid frequent conflicts #21580 (Azat Khuzhin).
- Enable ipv6 in NuRaft #21593 (alesapin).
- add an article about ast-based fuzzer #21608 (Alexander Kuzmenkov).
- Changelog 21.3 #21618 (Alexey Milovidov).
- redefine some errorcode #21629 (flynn).
- Fix ambigous column error in joins_in_memory #21658 (Vladimir C).
- Add test for path as a query parameter in system.zookeeper #21661 (Kruglov Pavel).
- ExecutablePool fix default max execution time setting #21662 (Maksim Kita).
- DictionaryStructure fix non unique attribute names #21668 (Maksim Kita).
- Fix flaky test_replace_partition #21674 (Azat Khuzhin).
- DOC Fix ORDER BY syntax #21675 (Michael Monashev).
- PODArray swap fix #21678 (Maksim Kita).
- LibraryDictionarySource fix possible leak #21686 (Maksim Kita).
- Run three nodes with Replicated database and NuKeeper in CI #21690 (Alexander Tokmakov).
- Fix error message in clickhouse-test #21691 (Azat Khuzhin).
- Set SOCK_CLOEXEC for sockets (hardcoded via poco update) #21695 (Azat Khuzhin).
- Tests fixes (that was found by stress tests) #21696 (Azat Khuzhin).
- Fix log_comment for *.sh in clickhouse-test #21700 (Azat Khuzhin).
- Remove useless CMake option #21712 (Alexey Milovidov).
- Fix UBSan report in modulo by constant #21713 (Alexey Milovidov).
- Add more variants for memcpy benchmark #21715 (Alexey Milovidov).
- Do not overlap zookeeper path for ReplicatedMergeTree in stateless *.sh tests #21724 (Azat Khuzhin).
- make the fuzzer use sources from the CI #21754 (Alexander Kuzmenkov).
- Add one more variant to memcpy benchmark #21759 (Alexey Milovidov).
- fix incorrect number of rows for Chunks with no columns in PartialSor… #21761 (Alexander Kuzmenkov).
- docs(fix): typo #21775 (Ali Demirci).
- DDLWorker.cpp: fixed exceeded amount of tries typo #21807 (Eldar Nasyrov).
- fix integration MaterializeMySQL test #21819 (TCeason).
- more robust error handling in perf test #21846 (Alexander Kuzmenkov).
- test for #17302 #21848 (Denny Crane).
- Add bash completion support for clickhouse utils #21853 (Azat Khuzhin).
- LRUCache fix exception unsafe element insertion #21891 (Maksim Kita).
- fix fuzzer failure in tupleElement formatting #21896 (Alexander Kuzmenkov).
- Fix possibly dangling reference to Context #21913 (Alexander Tokmakov).
- Add stress test for distributed queries #21944 (Azat Khuzhin).
- Fix misleading log in WriteBufferFromS3 #21954 (flynn).
- Add a test for #21991 #21995 (Alexey Milovidov).
- Add a test for #11720 #21997 (Alexey Milovidov).
- Add a test for #15784 #22002 (Alexey Milovidov).
- prevent accidental reinterpret_cast in Field::get<> #22003 (Alexander Kuzmenkov).
- Fix UBSan report in addMonths #22006 (Alexey Milovidov).
- Add a test for #7963 #22007 (Alexey Milovidov).
- Fix UBSan report in intDiv #21769 #22009 (Alexey Milovidov).
- Cast to enum nullable fix #22026 (Maksim Kita).
- Small simplification in ExternalLoader. #22027 (Vitaly Baranov).
- Add test for #21760 #22036 (Anton Popov).
- quick fix for broken resolution of apt.llvm.org on Yandex infra #22055 (Alexander Kuzmenkov).
- Darwin cmake disable memcpy benchmark #22056 (Maksim Kita).
- Fix UBSan report in TransformDateTime64 #22062 (Alexey Milovidov).
- Fix UBSan report in mapPopulateSeries. #22099 (Alexey Milovidov).
- Fix bar with invalid float value #22114 (flynn).
- remove useless code #22117 (flynn).
- stable formatting for negate() #22133 (Alexander Kuzmenkov).
- adjust perf test thresholds #22148 (Alexander Kuzmenkov).
- Fix sleep_in_send_tables_status_ms/sleep_in_send_data_ms in integration tests #22151 (Azat Khuzhin).
- Update requirements.txt #22153 (Ilya Yatsishin).
- Fix some flaky order dependent integration tests. #22170 (alesapin).
- Prevent busy waiting in hedged requests when async_socket_for_remote=0 #22172 (Kruglov Pavel).
- less flaky functional tests #22181 (Alexander Kuzmenkov).
- test for #10489 #22219 (Denny Crane).
- CachedCompressedReadBuffer fix cache usage #22225 (Maksim Kita).
- Fix MSan report in
quantileDeterministic
#22235 (Alexey Milovidov). - Remove harmful default parameters #22238 (Alexey Milovidov).
- Fix build error #22243 (hexiaoting).
- Rename NuKeeper and TestKeeper to Keeper in all places #22274 (alesapin).
- Update materialize-mysql.md #22275 (曲正鹏).
- Fix native macOS build for ALL_BUILD (Xcode/AppleClang) #22289 (Denis Glazachev).
- Add suffixes for dockerfile arguments #22301 (filimonov).
- More coarse test for DateLUT #22320 (Alexey Milovidov).
- Remove useless code #22328 (Anton Popov).
- Maybe fix false MSan report in GRPC #22338 (Alexey Milovidov).
- Remove old MSan suppressions (part 3) #22357 (Alexey Milovidov).
- Remove old MSan suppressions (part 4) #22358 (Alexey Milovidov).
- Fix flaky tests test_row_policy* and test_quota* #22371 (Vitaly Baranov).
- Try fix flaky rabbitmq test #22380 (Kseniia Sumarokova).
- Fix UBSan report in mapOp #22389 (Alexey Milovidov).
- Remove recursive submodules from Arrow #22390 (Alexey Milovidov).
- Fix some OOMs in stress tests #22396 (Alexey Milovidov).
- Correctly place debug helpers #22407 (Amos Bird).
- fix error message for invalid window frame start #22412 (Alexander Kuzmenkov).
- Lower scale of a test #22446 (Alexey Milovidov).
- Remove TestFlows (2) #22480 (Alexey Milovidov).