ClickHouse/docs/changelogs/v23.10.1.1976-stable.md
Alexey Milovidov 26501178e6 Fix analyzer
2024-05-17 10:23:32 +02:00

77 KiB
Raw Permalink Blame History

sidebar_position sidebar_label
1 2023

2023 Changelog

ClickHouse release v23.10.1.1976-stable (13adae0e42) FIXME as compared to v23.9.1.1854-stable (8f9a227de1)

Backward Incompatible Change

  • Rewrited storage S3Queue completely: changed the way we keep information in zookeeper which allows to make less zookeeper requests, added caching of zookeeper state in cases when we know the state will not change, improved the polling from s3 process to make it less aggressive, changed the way ttl and max set for trached files is maintained, now it is a background process. Added system.s3queue and system.s3queue_log tables. Closes #54998. #54422 (Kseniia Sumarokova).
  • There is no longer an option to automatically remove broken data parts. This closes #55174. #55184 (Alexey Milovidov).
  • The obsolete in-memory data parts can no longer be read from the write-ahead log. If you have configured in-memory parts before, they have to be removed before the upgrade. #55186 (Alexey Milovidov).
  • Remove the integration with Meilisearch. Reason: it was compatible only with the old version 0.18. The recent version of Meilisearch changed the protocol and does not work anymore. Note: we would appreciate it if you help to return it back. #55189 (Alexey Milovidov).
  • Rename directory monitor concept into background INSERT. All settings *directory_monitor* had been renamed to distributed_background_insert*. Backward compatibility should be preserved (since old settings had been added as an alias). #55978 (Azat Khuzhin).
  • Do not mix-up send_timeout and receive_timeout. #56035 (Azat Khuzhin).
  • Comparison of time intervals with different units will throw an exception. This closes #55942. You might have occasionally rely on the previous behavior when the underlying numeric values were compared regardless of the units. #56090 (Alexey Milovidov).

New Feature

  • Add function "arrayFold(x1, ..., xn, accum -> expression, array1, ..., arrayn, init_accum)" which applies a lambda function to multiple arrays of the same cardinality and collects the result in an accumulator. #49794 (Lirikl).
  • Added aggregation function lttb which uses the Largest-Triangle-Three-Buckets algorithm for downsampling data for visualization. #53145 (Sinan).
  • QueryCHECK TABLE has better performance and usability (sends progress updates, cancellable). Support checking particular part with CHECK TABLE ... PART 'part_name'. #53404 (vdimir).
  • Added function jsonMergePatch. When working with JSON data as strings, it provides a way to merge these strings (of JSON objects) together to form a single string containing a single JSON object. #54364 (Memo).
  • Added a new SQL function, "arrayRandomSample(arr, k)" which returns a sample of k elements from the input array. Similar functionality could previously be achieved only with less convenient syntax, e.g. "SELECT arrayReduce('groupArraySample(3)', range(10))". #54391 (itayisraelov).
  • Added new function getHttpHeader to get HTTP request header value used for a request to ClickHouse server. Return empty string if the request is not done over HTTP protocol or there is no such header. #54813 (凌涛).
  • Introduce -ArgMin/-ArgMax aggregate combinators which allow to aggregate by min/max values only. One use case can be found in #54818. This PR also reorganize combinators into dedicated folder. #54947 (Amos Bird).
  • Allow to drop cache for Protobuf format with SYSTEM DROP SCHEMA FORMAT CACHE [FOR Protobuf]. #55064 (Aleksandr Musorin).
  • Add external HTTP Basic authenticator. #55199 (Aleksei Filatov).
  • Added function byteSwap which reverses the bytes of unsigned integers. This is particularly useful for reversing values of types which are represented as unsigned integers internally such as IPv4. #55211 (Priyansh Agrawal).
  • Added function formatQuery() which returns a formatted version (possibly spanning multiple lines) of a SQL query string. Also added function formatQuerySingleLine() which does the same but the returned string will not contain linebreaks. #55239 (Salvatore Mesoraca).
  • Added DWARF input format that reads debug symbols from an ELF executable/library/object file. #55450 (Michael Kolupaev).
  • Allow to save unparsed records and errors in RabbitMQ, NATS and FileLog engines. Add virtual columns _error and _raw_message(for NATS and RabbitMQ), _raw_record (for FileLog) that are filled when ClickHouse fails to parse new record. The behaviour is controlled under storage settings nats_handle_error_mode for NATS, rabbitmq_handle_error_mode for RabbitMQ, handle_error_mode for FileLog similar to kafka_handle_error_mode. If it's set to default, en exception will be thrown when ClickHouse fails to parse a record, if it's set to stream, erorr and raw record will be saved into virtual columns. Closes #36035. #55477 (Kruglov Pavel).
  • Keeper client improvement: add get_all_children_number command that returns number of all children nodes under a specific path. #55485 (guoxiaolong).
  • If a table has a space-filling curve in its key, e.g., ORDER BY mortonEncode(x, y), the conditions on its arguments, e.g., x >= 10 AND x <= 20 AND y >= 20 AND y <= 30 can be used for indexing. A setting analyze_index_with_space_filling_curves is added to enable or disable this analysis. This closes #41195. Continuation of #4538. Continuation of #6286. Continuation of #28130. Continuation of #41753. #55642 (Alexey Milovidov).
  • Add setting optimize_trivial_approximate_count_query to use count() approximation for storage EmbeddedRocksDB. Enable trivial count for StorageJoin. #55806 (Duc Canh Le).
  • Keeper client improvement: add get_direct_children_number command that returns number of direct children nodes under a path. #55898 (xuzifu666).
  • Add statement SHOW SETTING setting_name which is a simpler version of existing statement SHOW SETTINGS. #55979 (Maksim Kita).
  • This pr gives possibility to pass data in Npy format to Clickhouse. ``` SELECT * FROM file('example_array.npy', Npy). #55982 (Yarik Briukhovetskyi).
  • This PR impleents a new setting called force_optimize_projection_name, it takes a name of projection as an argument. If it's value set to a non-empty string, ClickHouse checks that this projection is used in the query at least once. Closes #55331. #56134 (Yarik Briukhovetskyi).

Performance Improvement

  • Add option query_plan_preserve_num_streams_after_window_functions to preserve the number of streams after evaluating window functions to allow parallel stream processing. #50771 (frinkr).
  • Release more num_streams if data is small. #53867 (Jiebin Sun).
  • RoaringBitmaps being optimized before serialization. #55044 (UnamedRus).
  • Posting lists in inverted indexes are now optimized to use the smallest possible representation for internal bitmaps. Depending on the repetitiveness of the data, this may significantly reduce the space consumption of inverted indexes. #55069 (Harry Lee).
  • Fix contention on Context lock, this significantly improves performance for a lot of short-running concurrent queries. #55121 (Maksim Kita).
  • Improved the performance of inverted index creation by 30%. This was achieved by replacing std::unordered_map with absl::flat_hash_map. #55210 (Harry Lee).
  • Support orc filter push down (rowgroup level). #55330 (李扬).
  • Improve performance of external aggregation with a lot of temporary files. #55489 (Maksim Kita).
  • Set a reasonable size for the marks cache for secondary indices by default to avoid loading the marks over and over again. #55654 (Alexey Milovidov).
  • Avoid unnecessary reconstruction of index granules when reading skip indexes. This addresses #55653#issuecomment-1763766009 . #55683 (Amos Bird).
  • Cache cast function in set during execution to improve the performance of function IN when set element type doesn't exactly match column type. #55712 (Duc Canh Le).
  • Performance improvement for ColumnVector::insertMany and ColumnVector::insertManyFrom. #55714 (frinkr).
  • ... Getting values from a map is widely used. In practice, the key structrues are usally the same in the same map column, we could try to predict the next row's key position and reduce the comparisons. #55929 (lgbo).
  • ... Fix an issue that struct field prune doesn't work in some cases. For example ```sql INSERT INTO FUNCTION file('test_parquet_struct', Parquet, 'x Tuple(a UInt32, b UInt32, c String)') SELECT tuple(number, rand(), concat('testxxxxxxx' toString(number))) FROM numbers(10);. #56117 (lgbo).

Improvement

  • This is the second part of Kusto Query Language dialect support. Phase 1 implementation has been merged. #42510 (larryluogit).
  • Op processors IDs are raw ptrs casted to UInt64. Print it in a prettier manner:. #48852 (Vlad Seliverstov).
  • Creating a direct dictionary with a lifetime field set will be rejected at create time. Fixes: #27861. #49043 (Rory Crispin).
  • Allow parameters in queries with partitions like ALTER TABLE t DROP PARTITION. Closes #49449. #49516 (Nikolay Degterinsky).
  • 1.Refactor the code about zookeeper_connection 2.Add a new column xid for zookeeper_connection. #50702 (helifu).
  • Add the ability to tune the number of parallel replicas used in a query execution based on the estimation of rows to read. #51692 (Raúl Marín).
  • Distributed queries executed in async_socket_for_remote mode (default) now respect max_threads limit. Previously, some queries could create excessive threads (up to max_distributed_connections), causing server performance issues. #53504 (filimonov).
  • Display the correct server settings after reload. #53774 (helifu).
  • Add support for mathematical minus character in queries, similar to -. #54100 (Alexey Milovidov).
  • Add replica groups to the Replicated database engine. Closes #53620. #54421 (Nikolay Degterinsky).
  • This PR will fix UBsan test error here. #54568 (JackyWoo).
  • Support asynchronous inserts with external data via native protocol. Previously it worked only if data is inlined into query. #54730 (Anton Popov).
  • It is better to retry retriable s3 errors than totally fail the query. Set bigger value to the s3_retry_attempts by default. #54770 (Sema Checherinda).
  • Optimised external aggregation memory consumption in case many temporary files were generated. #54798 (Nikita Taranov).
  • Add load balancing test_hostname_levenshtein_distance. #54826 (JackyWoo).
  • Caching skip-able entries while executing DDL from Zookeeper distributed DDL queue. #54828 (Duc Canh Le).
  • Improve hiding secrets in logs. #55089 (Vitaly Baranov).
  • Added fields substreams and filenames to the system.parts_columns table. #55108 (Anton Popov).
  • For now the projection analysis will be performed only on top of query plan. The setting query_plan_optimize_projection became obsolete (it was enabled by default long time ago). #55112 (Nikita Mikhaylov).
  • When function "untuple()" is now called on a tuple with named elements and itself has an alias (e.g. "select untuple(tuple(1)::Tuple(element_alias Int)) AS untuple_alias"), then the result column name is now generated from the untuple alias and the tuple element alias (in the example: "untuple_alias.element_alias"). #55123 (garcher22).
  • Added setting describe_include_virtual_columns, which allows to include virtual columns of table into result of DESCRIBE query. Added setting describe_compact_output. If it is set to true, DESCRIBE query returns only names and types of columns without extra information. #55129 (Anton Popov).
  • Sometimes OPTIMIZE with optimize_throw_if_noop=1 may fail with an error unknown reason while the real cause of it - different projections in different parts. This behavior is fixed. #55130 (Nikita Mikhaylov).
  • Allow to have several MaterializedPostgreSQL tables following the same Postgres table. By default this behaviour is not enabled (for compatibility, because it is backward-incompatible change), but can be turned on with setting materialized_postgresql_use_unique_replication_consumer_identifier. Closes #54918. #55145 (Kseniia Sumarokova).
  • Allow to parse negative DateTime64 and DateTime with fractional part from short strings. #55146 (Andrey Zvonov).
  • To improve compatibility with MySQL, 1. "information_schema.tables" now includes the new field "table_rows", and 2. "information_schema.columns" now includes the new field "extra". #55215 (Robert Schulze).
  • Clickhouse-client won't show "0 rows in set" if it is zero and if exception was thrown. #55240 (Salvatore Mesoraca).
  • Support rename table without keyword TABLE like RENAME db.t1 to db.t2. #55373 (凌涛).
  • Add internal_replication to system.clusters. #55377 (Konstantin Morozov).
  • Select remote proxy resolver based on request protocol, add proxy feature docs and remove DB::ProxyConfiguration::Protocol::ANY. #55430 (Arthur Passos).
  • Avoid retrying keeper operations on INSERT after table shutdown. #55519 (Azat Khuzhin).
  • Improved overall resilience for ClickHouse in case of many parts within partition (more than 1000). It might reduce the number of TOO_MANY_PARTS errors. #55526 (Nikita Mikhaylov).
  • Follow up https://github.com/ClickHouse/ClickHouse/pull/55184 to not to fall into UNKNOWN_SETTING error when the user uses the deleted MergeTree settings. #55557 (Jihyuk Bok).
  • Updated dashboard.html() added a if to check if there is one chart, then "maximize" and "drag" buttons are not shown. #55581 (bhavuk2002).
  • Functions toDayOfWeek() (MySQL alias: DAYOFWEEK()), toYearWeek() (YEARWEEK()) and toWeek() (WEEK()) now supports String arguments. This makes its behavior consistent with MySQL's behavior. #55589 (Robert Schulze).
  • Implement query parameters support for ALTER TABLE ... ACTION PARTITION [ID] {parameter_name:ParameterType}. Merges #49516. Closes #49449. #55604 (alesapin).
  • Inverted indexes do not store tokens with too many matches (i.e. row ids in the posting list). This saves space and avoids ineffective index lookups when sequential scans would be equally fast or faster. The previous heuristics (density parameter passed to the index definition) that controlled when tokens would not be stored was too confusing for users. A much simpler heuristics based on parameter max_rows_per_postings_list (default: 64k) is introduced which directly controls the maximum allowed number of row ids in a postings list. #55616 (Harry Lee).
  • SHOW COLUMNS now correctly reports type FixedString as BLOB if setting use_mysql_types_in_show_columns is on. Also added two new settings, mysql_map_string_to_text_in_show_columns and mysql_map_fixed_string_to_text_in_show_columns to switch the output for types String and FixedString as TEXT or BLOB. #55617 (Serge Klochkov).
  • During ReplicatedMergeTree tables startup clickhouse server checks set of parts for unexpected parts (exists locally, but not in zookeeper). All unexpected parts move to detached directory and instead of them server tries to restore some ancestor (covered) parts. Now server tries to restore closest ancestors instead of random covered parts. #55645 (alesapin).
  • The advanced dashboard now supports draggable charts on touch devices. This closes #54206. #55649 (Alexey Milovidov).
  • Introduced setting date_time_overflow_behavior with possible values ignore, throw, saturate that controls the overflow behavior when converting from Date, Date32, DateTime64, Integer or Float to Date, Date32, DateTime or DateTime64. #55696 (Andrey Zvonov).
  • Improve write performance to rocksdb. #55732 (Duc Canh Le).
  • Use the default query format if declared when outputting exception with http_write_exception_in_output_format. #55739 (Raúl Marín).
  • Use upstream repo for apache datasketches. #55787 (Nikita Taranov).
  • Add support for SHOW MERGES query. #55815 (megao).
  • Provide a better message for common MV pitfalls. #55826 (Raúl Marín).
  • Reduced memory consumption during loading of hierarchical dictionaries. #55838 (Nikita Taranov).
  • All dictionaries support setting dictionary_use_async_executor. #55839 (vdimir).
  • If you dropped the current database, you will still be able to run some queries in clickhouse-local and switch to another database. This makes the behavior consistent with clickhouse-client. This closes #55834. #55853 (Alexey Milovidov).
  • Functions (add|subtract)(Year|Quarter|Month|Week|Day|Hour|Minute|Second|Millisecond|Microsecond|Nanosecond) now support string-encoded date arguments, e.g. SELECT addDays('2023-10-22', 1). This increases compatibility with MySQL and is needed by Tableau Online. #55869 (Robert Schulze).
  • Introduce setting create_table_empty_primary_key_by_default for default ORDER BY (). #55899 (Srikanth Chekuri).
  • Prevent excesive memory usage when deserializing AggregateFunctionTopKGenericData. #55947 (Raúl Marín).
  • The setting apply_deleted_mask when disabled allows to read rows that where marked as deleted by lightweight DELETE queries. This is useful for debugging. #55952 (Alexander Gololobov).
  • Allow skip null values when serailize tuple to json objects, which makes it possible to keep compatiability with spark to_json function, which is also useful for gluten. #55956 (李扬).
  • Functions (add|sub)Date() now support string-encoded date arguments, e.g. SELECT addDate('2023-10-22 11:12:13', INTERVAL 5 MINUTE). The same support for string-encoded date arguments is added to the plus and minus operators, e.g. SELECT '2023-10-23' + INTERVAL 1 DAY. This increases compatibility with MySQL and is needed by Tableau Online. #55960 (Robert Schulze).
  • Allow unquoted strings with CR in CSV format. Closes #39930. #56046 (Kruglov Pavel).
  • On a Keeper with lots of watches AsyncMetrics threads can consume 100% of CPU for noticable time in DB::KeeperStorage::getSessionsWithWatchesCount(). The fix is to avoid traversing heavy watches and list_watches sets. #56054 (Alexander Gololobov).
  • Allow to run clickhouse-keeper using embedded config. #56086 (Maksim Kita).
  • Set limit of the maximum configuration value for queued.min.messages to avoid problem with start fetching data with Kafka. #56121 (Stas Morozov).
  • Fixed typo in SQL function minSampleSizeContinous (renamed minSampleSizeContinuous). Old name is preserved for backward compatibility. This closes: #56139. #56143 (Dorota Szeremeta).
  • Print corrupted part path on disk before shutdown server. Before this change if a part is corrupted on disk and server cannot start, it was almost impossible to understand which part is broken. This is fixed. #56181 (Duc Canh Le).

Build/Testing/Packaging Improvement

  • If the database is already initialized, it doesn't need to be initialized again upon subsequent launches. This can potentially fix the issue of infinite container restarts when the database fails to load within 1000 attempts (relevant for very large databases and multi-node setups). #50724 (Alexander Nikolaev).
  • Resource with source code including submodules is built in Darwin special build task. It may be used to build ClickHouse without checkouting submodules. #51435 (Ilya Yatsishin).
  • An error will occur when building ClickHouse with the avx series of instructions enabled. CMake command shell cmake .. -DCMAKE_BUILD_TYPE=Release -DENABLE_AVX=ON -DENABLE_AVX2=ON -DENABLE_AVX2_FOR_SPEC_OP=ON Failed message [1558/11630] Building CXX object contrib/snappy-cmake/CMakeFiles/_snappy.dir/__/snappy/snappy.cc.o FAILED: contrib/snappy-cmake/CMakeFiles/_snappy.dir/__/snappy/snappy.cc.o /usr/bin/ccache /usr/bin/clang++-17 --target=x86_64-linux-gnu --sysroot=/opt/ClickHouse/cmake/linux/../../contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc -DHAVE_CONFIG_H -DSTD_EXCEPTION_HAS_STACK_TRACE=1 -D_LIBCPP_ENABLE_THREAD_SAFETY_ANNOTATIONS -I/opt/ClickHouse/base/glibc-compatibility/memcpy -isystem /opt/ClickHouse/contrib/snappy -isystem /opt/ClickHouse/build/contrib/snappy-cmake -isystem /opt/ClickHouse/contrib/llvm-project/libcxx/include -isystem /opt/ClickHouse/contrib/llvm-project/libcxxabi/include -isystem /opt/ClickHouse/contrib/libunwind/include --gcc-toolchain=/opt/ClickHouse/cmake/linux/../../contrib/sysroot/linux-x86_64 -fdiagnostics-color=always -Wno-enum-constexpr-conversion -fsized-deallocation -pipe -mssse3 -msse4.1 -msse4.2 -mpclmul -mpopcnt -mavx -mavx2 -fasynchronous-unwind-tables -ffile-prefix-map=/opt/ClickHouse=. -falign-functions=32 -mbranches-within-32B-boundaries -fdiagnostics-absolute-paths -fstrict-vtable-pointers -w -O3 -DNDEBUG -D OS_LINUX -Werror -nostdinc++ -std=c++2b -MD -MT contrib/snappy-cmake/CMakeFiles/_snappy.dir/__/snappy/snappy.cc.o -MF contrib/snappy-cmake/CMakeFiles/_snappy.dir/__/snappy/snappy.cc.o.d -o contrib/snappy-cmake/CMakeFiles/_snappy.dir/__/snappy/snappy.cc.o -c /opt/ClickHouse/contrib/snappy/snappy.cc /opt/ClickHouse/contrib/snappy/snappy.cc:1061:3: error: unknown type name '__m256i' 1061 | __m256i data = _mm256_lddqu_si256(static_cast<const __m256i *>(src)); | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1061:55: error: unknown type name '__m256i' 1061 | __m256i data = _mm256_lddqu_si256(static_cast<const __m256i *>(src)); | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1061:18: error: use of undeclared identifier '_mm256_lddqu_si256'; did you mean '_mm_lddqu_si128'? 1061 | __m256i data = _mm256_lddqu_si256(static_cast<const __m256i *>(src)); | ^~~~~~~~~~~~~~~~~~ | _mm_lddqu_si128 /usr/lib/llvm-17/lib/clang/17/include/pmmintrin.h:38:1: note: '_mm_lddqu_si128' declared here 38 | _mm_lddqu_si128(__m128i_u const *__p) | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1061:66: error: cannot initialize a parameter of type 'const __m128i_u *' with an lvalue of type 'const void *' 1061 | __m256i data = _mm256_lddqu_si256(static_cast<const __m256i *>(src)); | ^~~ /usr/lib/llvm-17/lib/clang/17/include/pmmintrin.h:38:34: note: passing argument to parameter '__p' here 38 | _mm_lddqu_si128(__m128i_u const *__p) | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1062:40: error: unknown type name '__m256i' 1062 | _mm256_storeu_si256(reinterpret_cast<__m256i *>(dst), data); | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1065:49: error: unknown type name '__m256i' 1065 | data = _mm256_lddqu_si256(static_cast<const __m256i *>(src) + 1); | ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1065:65: error: arithmetic on a pointer to void 1065 | data = _mm256_lddqu_si256(static_cast<const __m256i *>(src) + 1); | ~~~ ^ /opt/ClickHouse/contrib/snappy/snappy.cc:1066:42: error: unknown type name '__m256i' 1066 | _mm256_storeu_si256(reinterpret_cast<__m256i *>(dst) + 1, data); | ^ 8 errors generated. [1567/11630] Building CXX object contrib/rocksdb-cma...rocksdb.dir/__/rocksdb/db/arena_wrapped_db_iter.cc.o ninja: build stopped: subcommand failed. The reason is that snappy does not enable SNAPPY_HAVE_X86_CRC32. #55049 (monchickey).
  • Add instance_env_variables option to integration tests. #55208 (Arthur Passos).
  • Solve issue with launching standalone clickhouse-keeper from clickhouse-server package. #55226 (Mikhail f. Shiryaev).
  • In tests RabbitMQ version is updated to 3.12.6. Improved logs collection for RabbitMQ tests. #55424 (Ilya Yatsishin).
  • Fix integration check python script to use gh api url - Add Readme for CI tests. #55476 (Max K.).
  • Fix integration check python script to use gh api url - Add Readme for CI tests. #55716 (Max K.).
  • Check sha512 for tgz; use a proper repository for keeper; write only filenames to TGZ.sha512 files for tarball packages. Prerequisite for #31473. #55717 (Mikhail f. Shiryaev).
  • Updated to get free port for azurite. #55796 (SmitaRKulkarni).
  • Reduce the ouput info. #55938 (helifu).
  • Modified the error message difference between openssl and boringssl to fix the functional test. #55975 (MeenaRenganathan22).
  • Changes to support the HDFS for s390x. #56128 (MeenaRenganathan22).
  • Fix flaky test of jbod balancer by relaxing the Gini coefficient and introducing more determinism in insertions. #56175 (Amos Bird).

Bug Fix (user-visible misbehavior in an official stable release)

NO CL ENTRY

  • NO CL ENTRY: 'Revert "Fix libssh+openssl3 & s390x (part 2)"'. #55188 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Support SAMPLE BY for VIEW"'. #55357 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Revert "refine error code of duplicated index in create query""'. #55467 (Han Fei).
  • NO CL ENTRY: 'Update mysql.md - Remove the Private Preview Note'. #55486 (Ryadh DAHIMENE).
  • NO CL ENTRY: 'Revert "Removed "maximize" and "drag" buttons from dashboard in case of single chart"'. #55623 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Fix filtering by virtual columns with OR filter in query"'. #55657 (Antonio Andelic).
  • NO CL ENTRY: 'Revert "Improve ColumnDecimal, ColumnVector getPermutation performance using pdqsort with RadixSort"'. #55682 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Integration check script fix ups"'. #55694 (alesapin).
  • NO CL ENTRY: 'Revert "Fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table"'. #56103 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Add function getHttpHeader"'. #56109 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Fix output/input of Arrow dictionary column"'. #56150 (Alexey Milovidov).

NOT FOR CHANGELOG / INSIGNIFICANT