From da9a539cf79edb3f8daf8521ea7cac14eb0271a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Apr 2023 15:36:23 +0200 Subject: [PATCH 01/34] Reduce the usage of Arena.h --- .../AggregateFunctionDistinct.h | 1 + .../AggregateFunctionGroupUniqArray.h | 1 + .../AggregateFunctionHistogram.h | 2 +- .../AggregateFunctionIntervalLengthSum.h | 1 - .../AggregateFunctionKolmogorovSmirnovTest.h | 1 - .../AggregateFunctionMannWhitney.h | 1 - .../AggregateFunctionRankCorrelation.h | 2 -- .../AggregateFunctionRetention.h | 1 - .../AggregateFunctionTopK.h | 1 + .../AggregateFunctionWindowFunnel.h | 1 - src/Common/TLDListsHolder.h | 2 +- src/Dictionaries/DictionaryHelpers.h | 3 +- src/Dictionaries/IPAddressDictionary.h | 3 +- src/Dictionaries/RegExpTreeDictionary.h | 1 - src/IO/ReadHelpers.h | 17 ---------- src/IO/ReadHelpersArena.h | 31 +++++++++++++++++++ src/Interpreters/AggregationCommon.h | 3 +- src/Interpreters/Aggregator.h | 5 ++- .../Impl/ParallelFormattingOutputFormat.h | 1 - .../Transforms/TotalsHavingTransform.h | 4 --- 20 files changed, 46 insertions(+), 36 deletions(-) create mode 100644 src/IO/ReadHelpersArena.h diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.h b/src/AggregateFunctions/AggregateFunctionDistinct.h index e09e0ef621d..4338dcff5c0 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/AggregateFunctionDistinct.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index bc7ccb08267..4cd7a7932b0 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -4,6 +4,7 @@ #include #include +#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 3a98737f199..967bc9bb517 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -2,7 +2,6 @@ #include -#include #include #include @@ -29,6 +28,7 @@ namespace DB { struct Settings; +class Arena; namespace ErrorCodes { diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index e31c62802f1..16e9388d4bb 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -6,7 +6,6 @@ #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.h b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.h index 51e0950f782..737f37b1fba 100644 --- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.h +++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.h b/src/AggregateFunctions/AggregateFunctionMannWhitney.h index 86075440169..ac6ce0d0ca9 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.h +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h index 4f9ca55f9f5..4f7d04100cf 100644 --- a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h +++ b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { struct Settings; diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index 7ecb9509dd5..63ff5921540 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionTopK.h b/src/AggregateFunctions/AggregateFunctionTopK.h index f1e57608195..89985c0ea6b 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/src/AggregateFunctions/AggregateFunctionTopK.h @@ -2,6 +2,7 @@ #include #include +#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index c4a9fa1b936..e83c5277d26 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/TLDListsHolder.h b/src/Common/TLDListsHolder.h index 5ea8c5afe9f..be399843c08 100644 --- a/src/Common/TLDListsHolder.h +++ b/src/Common/TLDListsHolder.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -11,6 +10,7 @@ namespace DB { +class Arena; enum TLDType { diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 4fc080f2960..1de7be0bf4f 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -29,6 +28,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +class Arena; + /** Simple helper for getting default. * Initialized with default value and default values column. * If default values column is not null default value is taken from column. diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 67827c6524e..40dc5dd6782 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +17,8 @@ namespace DB { +class Arena; + class IPAddressDictionary final : public IDictionary { public: diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 17a0c6bbef3..4e8e20bba2d 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -10,7 +10,6 @@ #include #include -#include #include #include #include diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9c0c9525773..f1f001dd2c5 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -27,7 +27,6 @@ #include #include #include -#include #include #include @@ -142,22 +141,6 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin buf.readStrict(s.data(), size); } - -inline StringRef readStringBinaryInto(Arena & arena, ReadBuffer & buf) -{ - size_t size = 0; - readVarUInt(size, buf); - - if (unlikely(size > DEFAULT_MAX_STRING_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size."); - - char * data = arena.alloc(size); - buf.readStrict(data, size); - - return StringRef(data, size); -} - - template void readVectorBinary(std::vector & v, ReadBuffer & buf) { diff --git a/src/IO/ReadHelpersArena.h b/src/IO/ReadHelpersArena.h new file mode 100644 index 00000000000..3ff65817e33 --- /dev/null +++ b/src/IO/ReadHelpersArena.h @@ -0,0 +1,31 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int TOO_LARGE_STRING_SIZE; +} + +inline StringRef readStringBinaryInto(Arena & arena, ReadBuffer & buf) +{ + size_t size = 0; + readVarUInt(size, buf); + + if (unlikely(size > DEFAULT_MAX_STRING_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size."); + + char * data = arena.alloc(size); + buf.readStrict(data, size); + + return StringRef(data, size); +} + +} diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 32b01ee0416..2e6da40ff1f 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -25,6 +24,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class Arena; + using Sizes = std::vector; /// When packing the values of nullable columns at a given row, we have to diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 5fb94c5f4e8..92d0c27cac4 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -7,7 +7,6 @@ #include -#include #include #include #include @@ -47,6 +46,10 @@ namespace ErrorCodes extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; } +class Arena; +using ArenaPtr = std::shared_ptr; +using Arenas = std::vector; + /** Different data structures that can be used for aggregation * For efficiency, the aggregation data itself is put into the pool. * Data and pool ownership (states of aggregate functions) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 4e5aaab5dcb..fddcd059be5 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -2,7 +2,6 @@ #include -#include #include #include #include diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index f252d683b9a..350956c9c6b 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -2,14 +2,10 @@ #include #include -#include namespace DB { -class Arena; -using ArenaPtr = std::shared_ptr; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; From 2b70e08f23b72f75a763dcf11bde4ad9348faca4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Apr 2023 17:09:05 +0200 Subject: [PATCH 02/34] Don't count unreserved bytes in Arenas as read_bytes --- src/Columns/ColumnAggregateFunction.cpp | 9 ++-- src/Common/Arena.h | 17 +++++-- src/Common/ArenaWithFreeLists.h | 9 ++-- src/Common/examples/arena_with_free_lists.cpp | 6 +-- src/Coordination/SnapshotableHashTable.h | 5 +- src/Dictionaries/CacheDictionaryStorage.h | 2 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/HashedArrayDictionary.cpp | 2 +- src/Dictionaries/HashedDictionary.cpp | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.h | 2 +- src/Interpreters/HashJoin.cpp | 2 +- .../Algorithms/AggregatingSortedAlgorithm.cpp | 6 +-- .../Algorithms/SummingSortedAlgorithm.cpp | 6 +-- ...714_read_bytes_aggregateFunction.reference | 6 +++ .../02714_read_bytes_aggregateFunction.sql | 49 +++++++++++++++++++ 16 files changed, 93 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02714_read_bytes_aggregateFunction.reference create mode 100644 tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index fd46b38ada8..b52f078c315 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -385,8 +385,7 @@ void ColumnAggregateFunction::updateHashFast(SipHash & hash) const /// threads, so we can't know the size of these data. size_t ColumnAggregateFunction::byteSize() const { - return data.size() * sizeof(data[0]) - + (my_arena ? my_arena->size() : 0); + return data.size() * sizeof(data[0]) + (my_arena ? my_arena->reservedBytes() : 0); } size_t ColumnAggregateFunction::byteSizeAt(size_t) const @@ -395,11 +394,11 @@ size_t ColumnAggregateFunction::byteSizeAt(size_t) const return sizeof(data[0]) + func->sizeOfData(); } -/// Like in byteSize(), the size is underestimated. +/// Similar to byteSize() the size is underestimated. +/// In this case it's also overestimated at the same time as it counts all the bytes allocated by the arena, used or not size_t ColumnAggregateFunction::allocatedBytes() const { - return data.allocated_bytes() - + (my_arena ? my_arena->size() : 0); + return data.allocated_bytes() + (my_arena ? my_arena->allocatedBytes() : 0); } void ColumnAggregateFunction::protect() diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 5772dff6bca..4b358795972 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -299,10 +299,21 @@ public: return res; } - /// Size of MemoryChunks in bytes. - size_t size() const + /// Size of all MemoryChunks in bytes. + size_t allocatedBytes() const { return size_in_bytes; } + + /// Total reserved space of all MemoryChunks in bytes. + size_t reservedBytes() const { - return size_in_bytes; + size_t used_bytes = 0; + auto * current = head; + while (current) + { + used_bytes += current->pos - current->begin; + current = current->prev; + } + + return used_bytes; } /// Bad method, don't use it -- the MemoryChunks are not your business, the entire diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index 53a59c98299..76760a20320 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -107,10 +107,7 @@ public: } /// Size of the allocated pool in bytes - size_t size() const - { - return pool.size(); - } + size_t allocatedBytes() const { return pool.allocatedBytes(); } }; class SynchronizedArenaWithFreeLists : private ArenaWithFreeLists @@ -135,10 +132,10 @@ public: } /// Size of the allocated pool in bytes - size_t size() const + size_t allocatedBytes() const { std::lock_guard lock{mutex}; - return ArenaWithFreeLists::size(); + return ArenaWithFreeLists::allocatedBytes(); } private: mutable std::mutex mutex; diff --git a/src/Common/examples/arena_with_free_lists.cpp b/src/Common/examples/arena_with_free_lists.cpp index 4f209ccb5b2..3f1b3e88328 100644 --- a/src/Common/examples/arena_with_free_lists.cpp +++ b/src/Common/examples/arena_with_free_lists.cpp @@ -270,7 +270,7 @@ int main(int argc, char ** argv) watch.stop(); std::cerr - << "Insert info arena. Bytes: " << arena.size() + << "Insert info arena. Bytes: " << arena.allocatedBytes() << ", elapsed: " << watch.elapsedSeconds() << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.," << " " << sum_strings_size / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)" @@ -298,7 +298,7 @@ int main(int argc, char ** argv) watch.stop(); std::cerr - << "Randomly remove and insert elements. Bytes: " << arena.size() + << "Randomly remove and insert elements. Bytes: " << arena.allocatedBytes() << ", elapsed: " << watch.elapsedSeconds() << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.," << " " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)" @@ -331,7 +331,7 @@ int main(int argc, char ** argv) watch.stop(); std::cerr - << "Filling cache. Bytes: " << arena.size() + << "Filling cache. Bytes: " << arena.allocatedBytes() << ", elapsed: " << watch.elapsedSeconds() << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.," << " " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)" diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index cfa3098b4a1..7db546bd4c8 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -333,10 +333,7 @@ public: } } - uint64_t keyArenaSize() const - { - return arena.size(); - } + uint64_t keyArenaSize() const { return arena.allocatedBytes(); } iterator begin() { return list.begin(); } const_iterator begin() const { return list.cbegin(); } diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index 5b52fbde00d..ba17cebebba 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -157,7 +157,7 @@ public: }); } - return arena.size() + sizeof(Cell) * configuration.max_size_in_cells + attributes_size_in_bytes; + return arena.allocatedBytes() + sizeof(Cell) * configuration.max_size_in_cells + attributes_size_in_bytes; } private: diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index cc345b97abe..d3699a150c4 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -505,7 +505,7 @@ void FlatDictionary::calculateBytesAllocated() bytes_allocated += hierarchical_index_bytes_allocated; } - bytes_allocated += string_arena.size(); + bytes_allocated += string_arena.allocatedBytes(); } FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 9e6ce0597cb..880f68cea95 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -797,7 +797,7 @@ void HashedArrayDictionary::calculateBytesAllocated() bytes_allocated += hierarchical_index_bytes_allocated; } - bytes_allocated += string_arena.size(); + bytes_allocated += string_arena.allocatedBytes(); } template diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 5cfac20e572..b27a0b5160e 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1022,7 +1022,7 @@ void HashedDictionary::calculateBytesAlloc } for (const auto & arena : string_arenas) - bytes_allocated += arena->size(); + bytes_allocated += arena->allocatedBytes(); } template diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index ff1c784750b..6bb06de7506 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -541,7 +541,7 @@ template <> void IPAddressDictionary::addAttributeSize(const Attribute & attribute) { addAttributeSize(attribute); - bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); + bytes_allocated += sizeof(Arena) + attribute.string_arena->allocatedBytes(); } void IPAddressDictionary::calculateBytesAllocated() diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index d6bb510542e..600530a3e70 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -726,7 +726,7 @@ void RangeHashedDictionary::calculateBytesAllocated() if (update_field_loaded_block) bytes_allocated += update_field_loaded_block->allocatedBytes(); - bytes_allocated += string_arena.size(); + bytes_allocated += string_arena.allocatedBytes(); } template diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fe0244ff314..68bde0d3edb 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -517,7 +517,7 @@ size_t HashJoin::getTotalByteCount() const res += data->blocks_allocated_size; res += data->blocks_nullmaps_allocated_size; - res += data->pool.size(); + res += data->pool.allocatedBytes(); if (data->type != Type::CROSS) { diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index ef103eb508c..74cccdb08dd 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -168,7 +168,7 @@ AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData( if (def.allocates_memory_in_arena) { arena = std::make_unique(); - arena_size = arena->size(); + arena_size = arena->allocatedBytes(); } } @@ -194,10 +194,10 @@ void AggregatingSortedAlgorithm::AggregatingMergedData::startGroup(const ColumnR /// To avoid this, reset arena if and only if: /// - arena is required (i.e. SimpleAggregateFunction(any, String) in PK), /// - arena was used in the previous groups. - if (def.allocates_memory_in_arena && arena->size() > arena_size) + if (def.allocates_memory_in_arena && arena->allocatedBytes() > arena_size) { arena = std::make_unique(); - arena_size = arena->size(); + arena_size = arena->allocatedBytes(); } is_group_started = true; diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index d8e95e6b950..7f4dcfba6c2 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -505,7 +505,7 @@ SummingSortedAlgorithm::SummingMergedData::SummingMergedData( if (def.allocates_memory_in_arena) { arena = std::make_unique(); - arena_size = arena->size(); + arena_size = arena->allocatedBytes(); } } @@ -519,10 +519,10 @@ void SummingSortedAlgorithm::SummingMergedData::startGroup(ColumnRawPtrs & raw_c for (auto & desc : def.columns_to_aggregate) desc.createState(); - if (def.allocates_memory_in_arena && arena->size() > arena_size) + if (def.allocates_memory_in_arena && arena->allocatedBytes() > arena_size) { arena = std::make_unique(); - arena_size = arena->size(); + arena_size = arena->allocatedBytes(); } if (def.maps_to_sum.empty()) diff --git a/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.reference b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.reference new file mode 100644 index 00000000000..d315d85a11e --- /dev/null +++ b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.reference @@ -0,0 +1,6 @@ +UInt64 1 8 +UInt64 10 80 +UInt64 1000 8000 +AggregateFunction(argMax, String, DateTime) 1 80 +AggregateFunction(argMax, String, DateTime) 10 800 +AggregateFunction(argMax, String, DateTime) 1000 80000 diff --git a/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql new file mode 100644 index 00000000000..f66857ddee6 --- /dev/null +++ b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql @@ -0,0 +1,49 @@ +CREATE TABLE test (id UInt64, `amax` AggregateFunction(argMax, String, DateTime)) +ENGINE=MergeTree() +ORDER BY id +SETTINGS ratio_of_defaults_for_sparse_serialization=1 -- Sparse columns will take more bytes for a single row +AS + SELECT number, argMaxState(number::String, '2023-04-12 16:23:01'::DateTime) + FROM numbers(1) + GROUP BY number; + +SELECT sum(id) FROM test FORMAT Null; +SELECT argMaxMerge(amax) FROM test FORMAT Null; + +INSERT INTO test + SELECT number, argMaxState(number::String, '2023-04-12 16:23:01'::DateTime) + FROM numbers(9) + GROUP BY number; + +SELECT sum(id) FROM test FORMAT Null; +SELECT argMaxMerge(amax) FROM test FORMAT Null; + +INSERT INTO test +SELECT number, argMaxState(number::String, '2023-04-12 16:23:01'::DateTime) +FROM numbers(990) +GROUP BY number; + +SELECT sum(id) FROM test FORMAT Null; +SELECT argMaxMerge(amax) FROM test FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT 'UInt64', + read_rows, + read_bytes +FROM system.query_log +WHERE + current_database = currentDatabase() AND + query = 'SELECT sum(id) FROM test FORMAT Null;' AND + type = 2 AND event_date >= yesterday() +ORDER BY event_time_microseconds; + +SELECT 'AggregateFunction(argMax, String, DateTime)', + read_rows, + read_bytes +FROM system.query_log +WHERE + current_database = currentDatabase() AND + query = 'SELECT argMaxMerge(amax) FROM test FORMAT Null;' AND + type = 2 AND event_date >= yesterday() +ORDER BY event_time_microseconds; From e239df4c25e89bc5391ea37dc9ec3b37b68807d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Apr 2023 13:41:44 +0200 Subject: [PATCH 03/34] Keep count to avoid iterating over lists --- src/Common/Arena.h | 32 ++++++++----------- .../02714_read_bytes_aggregateFunction.sql | 10 ++++++ 2 files changed, 24 insertions(+), 18 deletions(-) diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 4b358795972..829d0687e0d 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -80,7 +80,8 @@ private: /// Last contiguous MemoryChunk of memory. MemoryChunk * head; - size_t size_in_bytes; + size_t allocated_bytes; + size_t reserved_bytes; size_t page_size; static size_t roundUpToPageSize(size_t s, size_t page_size) @@ -119,7 +120,7 @@ private: void NO_INLINE addMemoryChunk(size_t min_size) { head = new MemoryChunk(nextSize(min_size + pad_right), head); - size_in_bytes += head->size(); + allocated_bytes += head->size(); } friend class ArenaAllocator; @@ -127,9 +128,12 @@ private: public: explicit Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024) - : growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_), - head(new MemoryChunk(initial_size_, nullptr)), size_in_bytes(head->size()), - page_size(static_cast(::getPageSize())) + : growth_factor(growth_factor_) + , linear_growth_threshold(linear_growth_threshold_) + , head(new MemoryChunk(initial_size_, nullptr)) + , allocated_bytes(head->size()) + , reserved_bytes(0) + , page_size(static_cast(::getPageSize())) { } @@ -141,6 +145,7 @@ public: /// Get piece of memory, without alignment. char * alloc(size_t size) { + reserved_bytes += size; if (unlikely(static_cast(size) > head->end - head->pos)) addMemoryChunk(size); @@ -153,6 +158,7 @@ public: /// Get piece of memory with alignment char * alignedAlloc(size_t size, size_t alignment) { + reserved_bytes += size; do { void * head_pos = head->pos; @@ -184,6 +190,7 @@ public: */ void * rollback(size_t size) { + reserved_bytes -= size; head->pos -= size; ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right); return head->pos; @@ -300,21 +307,10 @@ public: } /// Size of all MemoryChunks in bytes. - size_t allocatedBytes() const { return size_in_bytes; } + size_t allocatedBytes() const { return allocated_bytes; } /// Total reserved space of all MemoryChunks in bytes. - size_t reservedBytes() const - { - size_t used_bytes = 0; - auto * current = head; - while (current) - { - used_bytes += current->pos - current->begin; - current = current->prev; - } - - return used_bytes; - } + size_t reservedBytes() const { return reserved_bytes; } /// Bad method, don't use it -- the MemoryChunks are not your business, the entire /// purpose of the arena code is to manage them for you, so if you find diff --git a/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql index f66857ddee6..26bc9ebe62b 100644 --- a/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql +++ b/tests/queries/0_stateless/02714_read_bytes_aggregateFunction.sql @@ -38,6 +38,16 @@ WHERE type = 2 AND event_date >= yesterday() ORDER BY event_time_microseconds; +-- Size of ColumnAggregateFunction: Number of pointers * pointer size + arena size +-- 1 * 8 + AggregateFunction(argMax, String, DateTime) +-- +-- Size of AggregateFunction(argMax, String, DateTime): +-- SingleValueDataString() + SingleValueDataFixed(DateTime) +-- SingleValueDataString = 64B for small strings, 64B + string size + 1 for larger +-- SingleValueDataFixed(DateTime) = 1 + 4. With padding = 8 +-- SingleValueDataString Total: 72B +-- +-- ColumnAggregateFunction total: 8 + 72 = 80 SELECT 'AggregateFunction(argMax, String, DateTime)', read_rows, read_bytes From 2c20850cde6e5ce62f55da8845df560a4657d019 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Apr 2023 10:35:33 +0200 Subject: [PATCH 04/34] Style --- src/IO/ReadHelpersArena.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadHelpersArena.h b/src/IO/ReadHelpersArena.h index 3ff65817e33..b88d5c037d4 100644 --- a/src/IO/ReadHelpersArena.h +++ b/src/IO/ReadHelpersArena.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include From db71bbb08da0832d1f08bbdad62c6379944c85ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Apr 2023 10:59:40 +0200 Subject: [PATCH 05/34] Fix examples build --- src/Interpreters/examples/string_hash_map.cpp | 2 +- src/Interpreters/examples/string_hash_set.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/examples/string_hash_map.cpp b/src/Interpreters/examples/string_hash_map.cpp index 15df8c399de..f55ed983fbc 100644 --- a/src/Interpreters/examples/string_hash_map.cpp +++ b/src/Interpreters/examples/string_hash_map.cpp @@ -156,7 +156,7 @@ void NO_INLINE bench(const std::vector & data, DB::Arena &, const cha } watch.stop(); - std::cerr << "arena-memory " << pool.size() + map.getBufferSizeInBytes() << std::endl; + std::cerr << "arena-memory " << pool.allocatedBytes() + map.getBufferSizeInBytes() << std::endl; std::cerr << "single-run " << std::setprecision(3) << watch.elapsedSeconds() << std::endl; } diff --git a/src/Interpreters/examples/string_hash_set.cpp b/src/Interpreters/examples/string_hash_set.cpp index 355789e97ec..527ada1579d 100644 --- a/src/Interpreters/examples/string_hash_set.cpp +++ b/src/Interpreters/examples/string_hash_set.cpp @@ -34,7 +34,7 @@ void NO_INLINE bench(const std::vector & data, DB::Arena & pool, cons } watch.stop(); - std::cerr << "arena-memory " << pool.size() + set.getBufferSizeInBytes() << std::endl; + std::cerr << "arena-memory " << pool.allocatedBytes() + set.getBufferSizeInBytes() << std::endl; std::cerr << "single-run " << std::setprecision(3) << watch.elapsedSeconds() << std::endl; } From 33bedc0291d9410c1c0ff0d9c1e1faee797aacf5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 14 Apr 2023 10:03:38 +0800 Subject: [PATCH 06/34] improve jsonpath support in json functions --- .../Parsers/ParserJSONPathMemberAccess.cpp | 50 ++++++++++++++---- ...arserJSONPathMemberSquareBracketAccess.cpp | 51 +++++++++++++++++++ .../ParserJSONPathMemberSquareBracketAccess.h | 17 +++++++ .../JSONPath/Parsers/ParserJSONPathQuery.cpp | 3 ++ .../01889_sql_json_functions.reference | 20 ++++++++ .../0_stateless/01889_sql_json_functions.sql | 10 ++++ 6 files changed, 142 insertions(+), 9 deletions(-) create mode 100644 src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp create mode 100644 src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp index f0ed178e1c2..8a415816798 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp @@ -1,9 +1,11 @@ #include #include +#include #include #include #include +#include namespace DB { @@ -16,18 +18,48 @@ namespace DB */ bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (pos->type != TokenType::Dot) + // There's a specical case, that a path member can begin with number + if (pos->type != TokenType::Dot && pos->type != TokenType::Number) return false; + if (pos->type != TokenType::Number) + ++pos; - ++pos; - - if (pos->type != TokenType::BareWord && pos->type !=TokenType::QuotedIdentifier) - return false; - - ParserIdentifier name_p; ASTPtr member_name; - if (!name_p.parse(pos, member_name, expected)) - return false; + + if (pos->type == TokenType::Number)[[unlikely]] + { + for (const auto * c = pos->begin; c != pos->end; ++c) + { + if (*c == '.' && c == pos->begin) + continue; + if (!isNumericASCII(*c)) + { + return false; + } + } + const auto * last_begin = *pos->begin == '.' ? pos->begin + 1 : pos->begin; + const auto * last_end = pos->end; + ++pos; + + if (pos.isValid() && pos->type == TokenType::BareWord && pos->begin == last_end) + { + member_name = std::make_shared(String(last_begin, pos->end)); + ++pos; + } + else + { + return false; + } + } + else + { + if (pos->type != TokenType::BareWord && pos->type != TokenType::QuotedIdentifier) + return false; + + ParserIdentifier name_p; + if (!name_p.parse(pos, member_name, expected)) + return false; + } auto member_access = std::make_shared(); node = member_access; diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp new file mode 100644 index 00000000000..b3059fcfd94 --- /dev/null +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp @@ -0,0 +1,51 @@ +#include "ParserJSONPathMemberSquareBracketAccess.h" +#include +#include +#include +#include +#include +#include + +namespace DB +{ +bool ParserJSONPathMemberSquareBracketAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (pos->type != TokenType::OpeningSquareBracket) + return false; + ++pos; + ASTPtr member_name; + if (pos->type == TokenType::BareWord || pos->type == TokenType::QuotedIdentifier) + { + ParserIdentifier name_p; + if (!name_p.parse(pos, member_name, expected)) + return false; + } + else if (pos->type == TokenType::StringLiteral) + { + try + { + ReadBufferFromMemory in(pos->begin, pos->size()); + String name; + readQuotedStringWithSQLStyle(name, in); + member_name = std::make_shared(name); + ++pos; + } + catch (const Exception &) + { + return false; + } + } + else + { + return false; + } + if (pos->type != TokenType::ClosingSquareBracket) + { + return false; + } + ++pos; + auto member_access = std::make_shared(); + node = member_access; + return tryGetIdentifierNameInto(member_name, member_access->member_name); +} +} diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h new file mode 100644 index 00000000000..b682ec5bb96 --- /dev/null +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h @@ -0,0 +1,17 @@ +#pragma once +#include +// cases +// - [ident] +// - ['ident'] +// - ["ident"] +namespace DB +{ +class ParserJSONPathMemberSquareBracketAccess : public IParserBase +{ +private: + const char * getName() const override { return "ParserJSONPathMemberSquareBracketAccess"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +public: + explicit ParserJSONPathMemberSquareBracketAccess() = default; +}; +} diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathQuery.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathQuery.cpp index c18b2ad9b31..d8d633a1ec9 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathQuery.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -19,6 +20,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect { query = std::make_shared(); ParserJSONPathMemberAccess parser_jsonpath_member_access; + ParserJSONPathMemberSquareBracketAccess parser_jsonpath_member_square_bracket_access; ParserJSONPathRange parser_jsonpath_range; ParserJSONPathStar parser_jsonpath_star; ParserJSONPathRoot parser_jsonpath_root; @@ -32,6 +34,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect ASTPtr accessor; while (parser_jsonpath_member_access.parse(pos, accessor, expected) + || parser_jsonpath_member_square_bracket_access.parse(pos, accessor, expected) || parser_jsonpath_range.parse(pos, accessor, expected) || parser_jsonpath_star.parse(pos, accessor, expected)) { diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 5ac1ff501e5..23630bffb93 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -37,6 +37,16 @@ select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_v {"world":"!"} SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello') settings function_json_value_return_type_allow_complex=true; ["world","world2"] +SELECT JSON_VALUE('{"1key":1}', '$.1key'); +1 +SELECT JSON_VALUE('{"hello":1}', '$[hello]'); +1 +SELECT JSON_VALUE('{"hello":1}', '$["hello"]'); +1 +SELECT JSON_VALUE('{"hello":1}', '$[\'hello\']'); +1 +SELECT JSON_VALUE('{"hello 1":1}', '$["hello 1"]'); +1 SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); @@ -61,6 +71,16 @@ SELECT JSON_QUERY('', '$.hello'); SELECT JSON_QUERY('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); [0, 1, 4, 0, -1, -4] +SELECT JSON_QUERY('{"1key":1}', '$.1key'); +[1] +SELECT JSON_QUERY('{"hello":1}', '$[hello]'); +[1] +SELECT JSON_QUERY('{"hello":1}', '$["hello"]'); +[1] +SELECT JSON_QUERY('{"hello":1}', '$[\'hello\']'); +[1] +SELECT JSON_QUERY('{"hello 1":1}', '$["hello 1"]'); +[1] SELECT '--JSON_EXISTS--'; --JSON_EXISTS-- SELECT JSON_EXISTS('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index f174d04933c..7f417f3f228 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -20,6 +20,11 @@ select JSON_VALUE('{"a":"\\u263a"}', '$.a'); select JSON_VALUE('{"hello":"world"}', '$.b') settings function_json_value_return_type_allow_nullable=true; select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello') settings function_json_value_return_type_allow_complex=true; +SELECT JSON_VALUE('{"1key":1}', '$.1key'); +SELECT JSON_VALUE('{"hello":1}', '$[hello]'); +SELECT JSON_VALUE('{"hello":1}', '$["hello"]'); +SELECT JSON_VALUE('{"hello":1}', '$[\'hello\']'); +SELECT JSON_VALUE('{"hello 1":1}', '$["hello 1"]'); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); @@ -33,6 +38,11 @@ SELECT JSON_QUERY('{"hello":{"world":"!"}}', '$.hello'); SELECT JSON_QUERY( '{hello:{"world":"!"}}}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_QUERY('', '$.hello'); SELECT JSON_QUERY('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); +SELECT JSON_QUERY('{"1key":1}', '$.1key'); +SELECT JSON_QUERY('{"hello":1}', '$[hello]'); +SELECT JSON_QUERY('{"hello":1}', '$["hello"]'); +SELECT JSON_QUERY('{"hello":1}', '$[\'hello\']'); +SELECT JSON_QUERY('{"hello 1":1}', '$["hello 1"]'); SELECT '--JSON_EXISTS--'; SELECT JSON_EXISTS('{"hello":1}', '$'); From d8d7639ba7501158f4bf0c0af22fb964914926a8 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 14 Apr 2023 14:42:59 +0800 Subject: [PATCH 07/34] fixed typos & style --- src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp index 8a415816798..460f1d6a649 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp @@ -18,7 +18,7 @@ namespace DB */ bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - // There's a specical case, that a path member can begin with number + // There's a special case, that a path member can begin with number if (pos->type != TokenType::Dot && pos->type != TokenType::Number) return false; if (pos->type != TokenType::Number) From ac87a1c23ae3cd7a4863d93c9da97bec24a91db9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sun, 23 Apr 2023 13:10:41 +0800 Subject: [PATCH 08/34] update --- .../Parsers/ParserJSONPathMemberAccess.cpp | 12 ++++++++++++ .../ParserJSONPathMemberSquareBracketAccess.cpp | 17 +++++------------ .../01889_sql_json_functions.reference | 8 ++++++++ .../0_stateless/01889_sql_json_functions.sql | 8 ++++++++ 4 files changed, 33 insertions(+), 12 deletions(-) diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp index 460f1d6a649..709ef89dd3c 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp @@ -19,10 +19,22 @@ namespace DB bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { // There's a special case, that a path member can begin with number + // some invalid cases as following + // - ".123" is parsed as a number, not a dot and a number + // - ".123abc" is parsed as two parts, a number ".123" and a token "abc" + // - ".abc" is parsed as two parts. a dot and a token "abc" + // "$..123abc" is parsed into three parts, ".", ".123" and "abc" if (pos->type != TokenType::Dot && pos->type != TokenType::Number) return false; if (pos->type != TokenType::Number) + { ++pos; + // Check the case "$..123abc" + if (pos->type == TokenType::Number) + { + return false; + } + } ASTPtr member_name; diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp index b3059fcfd94..93e0639ccfe 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.cpp @@ -22,18 +22,11 @@ bool ParserJSONPathMemberSquareBracketAccess::parseImpl(Pos & pos, ASTPtr & node } else if (pos->type == TokenType::StringLiteral) { - try - { - ReadBufferFromMemory in(pos->begin, pos->size()); - String name; - readQuotedStringWithSQLStyle(name, in); - member_name = std::make_shared(name); - ++pos; - } - catch (const Exception &) - { - return false; - } + ReadBufferFromMemory in(pos->begin, pos->size()); + String name; + readQuotedStringWithSQLStyle(name, in); + member_name = std::make_shared(name); + ++pos; } else { diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 23630bffb93..cb8e19ea2a0 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -47,6 +47,10 @@ SELECT JSON_VALUE('{"hello":1}', '$[\'hello\']'); 1 SELECT JSON_VALUE('{"hello 1":1}', '$["hello 1"]'); 1 +SELECT JSON_VALUE('{"1key":1}', '$..1key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$1key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$.[key]'); -- { serverError 36 } SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); @@ -81,6 +85,10 @@ SELECT JSON_QUERY('{"hello":1}', '$[\'hello\']'); [1] SELECT JSON_QUERY('{"hello 1":1}', '$["hello 1"]'); [1] +SELECT JSON_QUERY('{"1key":1}', '$..1key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$1key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$.[key]'); -- { serverError 36 } SELECT '--JSON_EXISTS--'; --JSON_EXISTS-- SELECT JSON_EXISTS('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 7f417f3f228..947b0171ec6 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -25,6 +25,10 @@ SELECT JSON_VALUE('{"hello":1}', '$[hello]'); SELECT JSON_VALUE('{"hello":1}', '$["hello"]'); SELECT JSON_VALUE('{"hello":1}', '$[\'hello\']'); SELECT JSON_VALUE('{"hello 1":1}', '$["hello 1"]'); +SELECT JSON_VALUE('{"1key":1}', '$..1key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$1key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$key'); -- { serverError 36 } +SELECT JSON_VALUE('{"1key":1}', '$.[key]'); -- { serverError 36 } SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); @@ -43,6 +47,10 @@ SELECT JSON_QUERY('{"hello":1}', '$[hello]'); SELECT JSON_QUERY('{"hello":1}', '$["hello"]'); SELECT JSON_QUERY('{"hello":1}', '$[\'hello\']'); SELECT JSON_QUERY('{"hello 1":1}', '$["hello 1"]'); +SELECT JSON_QUERY('{"1key":1}', '$..1key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$1key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$key'); -- { serverError 36 } +SELECT JSON_QUERY('{"1key":1}', '$.[key]'); -- { serverError 36 } SELECT '--JSON_EXISTS--'; SELECT JSON_EXISTS('{"hello":1}', '$'); From 868b4e41fd1fd9fff3480b85cb1b9631a1323963 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 24 Apr 2023 10:46:23 +0200 Subject: [PATCH 09/34] Arena used bytes --- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Common/Arena.h | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index b52f078c315..e521262acd2 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -385,7 +385,7 @@ void ColumnAggregateFunction::updateHashFast(SipHash & hash) const /// threads, so we can't know the size of these data. size_t ColumnAggregateFunction::byteSize() const { - return data.size() * sizeof(data[0]) + (my_arena ? my_arena->reservedBytes() : 0); + return data.size() * sizeof(data[0]) + (my_arena ? my_arena->usedBytes() : 0); } size_t ColumnAggregateFunction::byteSizeAt(size_t) const diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 829d0687e0d..0cf343809e8 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -81,7 +81,7 @@ private: /// Last contiguous MemoryChunk of memory. MemoryChunk * head; size_t allocated_bytes; - size_t reserved_bytes; + size_t used_bytes; size_t page_size; static size_t roundUpToPageSize(size_t s, size_t page_size) @@ -132,7 +132,7 @@ public: , linear_growth_threshold(linear_growth_threshold_) , head(new MemoryChunk(initial_size_, nullptr)) , allocated_bytes(head->size()) - , reserved_bytes(0) + , used_bytes(0) , page_size(static_cast(::getPageSize())) { } @@ -145,7 +145,7 @@ public: /// Get piece of memory, without alignment. char * alloc(size_t size) { - reserved_bytes += size; + used_bytes += size; if (unlikely(static_cast(size) > head->end - head->pos)) addMemoryChunk(size); @@ -158,7 +158,7 @@ public: /// Get piece of memory with alignment char * alignedAlloc(size_t size, size_t alignment) { - reserved_bytes += size; + used_bytes += size; do { void * head_pos = head->pos; @@ -190,7 +190,7 @@ public: */ void * rollback(size_t size) { - reserved_bytes -= size; + used_bytes -= size; head->pos -= size; ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right); return head->pos; @@ -309,8 +309,8 @@ public: /// Size of all MemoryChunks in bytes. size_t allocatedBytes() const { return allocated_bytes; } - /// Total reserved space of all MemoryChunks in bytes. - size_t reservedBytes() const { return reserved_bytes; } + /// Total space actually used (not counting padding or space unused by caller allocations) in all MemoryChunks in bytes. + size_t usedBytes() const { return used_bytes; } /// Bad method, don't use it -- the MemoryChunks are not your business, the entire /// purpose of the arena code is to manage them for you, so if you find From 277393fd8dd3159d288d3809d5c916f7772b6fe6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 Apr 2023 14:45:41 +0200 Subject: [PATCH 10/34] Fix --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 4 ++++ src/Databases/PostgreSQL/DatabasePostgreSQL.h | 1 + src/Storages/StoragePostgreSQL.cpp | 2 +- .../test_postgresql_database_engine/test.py | 11 +++++++++++ 4 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 8c267ea929b..d9116f74c52 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -51,6 +51,7 @@ DatabasePostgreSQL::DatabasePostgreSQL( , configuration(configuration_) , pool(std::move(pool_)) , cache_tables(cache_tables_) + , log(&Poco::Logger::get("DatabasePostgreSQL(" + dbname_ + ")")) { cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); cleaner_task->deactivate(); @@ -192,7 +193,10 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict); if (cache_tables) + { + LOG_TEST(log, "Cached table `{}`", table_name); cached_tables[table_name] = storage; + } return storage; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 08583f4b6d9..31fa036c0ee 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -73,6 +73,7 @@ private: mutable Tables cached_tables; std::unordered_set detached_or_dropped; BackgroundSchedulePool::TaskHolder cleaner_task; + Poco::Logger * log; String getTableNameForLogs(const String & table_name) const; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 8e1a799fa07..8548d558fd3 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -397,7 +397,7 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult required_arguments.insert("table"); validateNamedCollection>( - named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port"}); + named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port", "use_tables_cache"}); configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index de6c9ad2cf9..63e85afb1d4 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -327,6 +327,17 @@ def test_predefined_connection_configuration(started_cluster): node1.query(f"SELECT count() FROM postgres_database.test_table").rstrip() == "100" ) + node1.query( + """ + DROP DATABASE postgres_database; + CREATE DATABASE postgres_database ENGINE = PostgreSQL(postgres1, use_tables_cache=1); + """ + ) + assert ( + node1.query(f"SELECT count() FROM postgres_database.test_table").rstrip() + == "100" + ) + assert node1.contains_in_log("Cached table `test_table`") node1.query("DROP DATABASE postgres_database") cursor.execute(f"DROP TABLE test_table ") From 50dd8365869071dee74569fc85f184d1e20873c1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 16:37:54 +0200 Subject: [PATCH 11/34] Fix flaky integration test test_async_query_sending --- tests/integration/test_hedged_requests/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 88371f6908d..2ca37fbb7ee 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -128,12 +128,12 @@ def check_changing_replica_events(expected_count): assert int(result) >= expected_count -def check_if_query_sending_was_suspended(minimum_count): +def check_if_query_sending_was_suspended(): result = NODES["node"].query( "SELECT value FROM system.events WHERE event='SuspendSendingQueryToShard'" ) - assert int(result) >= minimum_count + assert int(result) >= 1 def check_if_query_sending_was_not_suspended(): @@ -381,7 +381,7 @@ def test_async_connect(started_cluster): "SELECT hostName(), id FROM distributed_connect ORDER BY id LIMIT 1 SETTINGS prefer_localhost_replica = 0, connect_timeout_with_failover_ms=5000, async_query_sending_for_remote=1, max_threads=1" ) check_changing_replica_events(2) - check_if_query_sending_was_suspended(2) + check_if_query_sending_was_suspended() NODES["node"].query("DROP TABLE distributed_connect") @@ -406,7 +406,7 @@ def test_async_query_sending(started_cluster): NODES["node"].query("DROP TABLE IF EXISTS tmp") NODES["node"].query( "CREATE TEMPORARY TABLE tmp (number UInt64, s String) " - "as select number, randomString(number % 1000) from numbers(1000000)" + "as select number, randomString(number % 1000) from numbers(10000000)" ) NODES["node"].query( @@ -419,6 +419,6 @@ def test_async_query_sending(started_cluster): "SELECT hostName(), id FROM distributed_query_sending ORDER BY id LIMIT 1 SETTINGS" " prefer_localhost_replica = 0, async_query_sending_for_remote=1, max_threads = 1" ) - check_if_query_sending_was_suspended(3) + check_if_query_sending_was_suspended() NODES["node"].query("DROP TABLE distributed_query_sending") From 8e4056218f78e4181d088cf756fc49dc5b6ed327 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 20 Apr 2023 14:01:31 -0700 Subject: [PATCH 12/34] Correct functional test to reflect interoperability --- tests/queries/0_stateless/00322_disable_checksumming.sh | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/00322_disable_checksumming.sh b/tests/queries/0_stateless/00322_disable_checksumming.sh index c044a5c6650..e04ec076f80 100755 --- a/tests/queries/0_stateless/00322_disable_checksumming.sh +++ b/tests/queries/0_stateless/00322_disable_checksumming.sh @@ -4,10 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# use big-endian version of binary data for s390x -if [[ $(uname -a | grep s390x) ]]; then -echo -ne '\xdb\x8a\xe9\x59\xf2\x32\x74\x50\x39\xc4\x22\xfb\xa7\x4a\xc6\x37''\x82\x13\x00\x00\x00\x09\x00\x00\x00''\x90SELECT 1\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&decompress=1" --data-binary @- -else echo -ne '\x50\x74\x32\xf2\x59\xe9\x8a\xdb\x37\xc6\x4a\xa7\xfb\x22\xc4\x39''\x82\x13\x00\x00\x00\x09\x00\x00\x00''\x90SELECT 1\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&decompress=1" --data-binary @- -fi echo -ne 'xxxxxxxxxxxxxxxx''\x82\x13\x00\x00\x00\x09\x00\x00\x00''\x90SELECT 1\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&decompress=1&http_native_compression_disable_checksumming_on_decompress=1" --data-binary @- From 4048756bdb12b6b63718c719c74cff314ef1fc55 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 Apr 2023 16:50:07 +0200 Subject: [PATCH 13/34] Fix --- src/Interpreters/Cache/FileCache.cpp | 15 ++++++--- src/Interpreters/Cache/FileSegment.cpp | 33 ++++++++++++++++--- src/Interpreters/Cache/IFileCachePriority.h | 2 +- .../Cache/LRUFileCachePriority.cpp | 16 ++++++--- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- 5 files changed, 53 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 24391ed0b7f..046e388430f 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -719,7 +719,7 @@ bool FileCache::tryReserve(const Key & key, size_t offset, size_t size, std::loc { auto queue_iterator = cell_for_reserve->queue_iterator; if (queue_iterator) - queue_iterator->incrementSize(size, cache_lock); + queue_iterator->updateSize(size, cache_lock); else cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, cache_lock); } @@ -836,7 +836,7 @@ bool FileCache::tryReserveForMainList( /// If queue iterator already exists, we need to update the size after each space reservation. auto queue_iterator = cell_for_reserve->queue_iterator; if (queue_iterator) - queue_iterator->incrementSize(size, cache_lock); + queue_iterator->updateSize(size, cache_lock); else cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, cache_lock); } @@ -1152,7 +1152,14 @@ void FileCache::reduceSizeToDownloaded( cell->file_segment = std::make_shared( offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED, create_settings); - assert(file_segment->reserved_size == downloaded_size); + chassert(cell->queue_iterator); + chassert(cell->queue_iterator->size() >= downloaded_size); + const ssize_t diff = cell->queue_iterator->size() - downloaded_size; + if (diff > 0) + cell->queue_iterator->updateSize(-diff, cache_lock); + + chassert(file_segment->reserved_size == downloaded_size); + chassert(file_segment->reserved_size == cell->queue_iterator->size()); } bool FileCache::isLastFileSegmentHolder( @@ -1450,7 +1457,7 @@ void FileCache::QueryContext::reserve(const Key & key, size_t offset, size_t siz auto queue_iter = priority->add(key, offset, 0, cache_lock); record = records.insert({{key, offset}, queue_iter}).first; } - record->second->incrementSize(size, cache_lock); + record->second->updateSize(size, cache_lock); } cache_size += size; } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 4a11ade5b42..6ae25c681d4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -11,6 +11,8 @@ #include +namespace fs = std::filesystem; + namespace CurrentMetrics { extern const Metric CacheDetachedFileSegments; @@ -63,7 +65,7 @@ FileSegment::FileSegment( { reserved_size = downloaded_size = size_; is_downloaded = true; - chassert(std::filesystem::file_size(getPathInLocalCache()) == size_); + chassert(fs::file_size(getPathInLocalCache()) == size_); break; } case (State::SKIP_CACHE): @@ -385,6 +387,28 @@ void FileSegment::write(const char * from, size_t size, size_t offset) chassert(std::filesystem::file_size(getPathInLocalCache()) == downloaded_size); } + catch (ErrnoException & e) + { + std::unique_lock segment_lock(mutex); + + wrapWithCacheInfo(e, "while writing into cache", segment_lock); + + int code = e.getErrno(); + if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + { + const auto file_size = fs::file_size(getPathInLocalCache()); + chassert(downloaded_size <= file_size); + chassert(reserved_size >= file_size); + if (downloaded_size != file_size) + downloaded_size = file_size; + } + + setDownloadFailedUnlocked(segment_lock); + + cv.notify_all(); + throw; + + } catch (Exception & e) { std::unique_lock segment_lock(mutex); @@ -394,7 +418,6 @@ void FileSegment::write(const char * from, size_t size, size_t offset) setDownloadFailedUnlocked(segment_lock); cv.notify_all(); - throw; } @@ -504,7 +527,7 @@ void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock 0); - assert(std::filesystem::file_size(getPathInLocalCache()) > 0); + assert(fs::file_size(getPathInLocalCache()) > 0); } void FileSegment::setDownloadFailedUnlocked(std::unique_lock & segment_lock) @@ -633,7 +656,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach case State::DOWNLOADED: { chassert(getDownloadedSizeUnlocked(segment_lock) == range().size()); - chassert(getDownloadedSizeUnlocked(segment_lock) == std::filesystem::file_size(getPathInLocalCache())); + chassert(getDownloadedSizeUnlocked(segment_lock) == fs::file_size(getPathInLocalCache())); chassert(is_downloaded); chassert(!cache_writer); break; @@ -746,7 +769,7 @@ void FileSegment::assertCorrectnessUnlocked(std::unique_lock & segme auto current_downloader = getDownloaderUnlocked(segment_lock); chassert(current_downloader.empty() == (download_state != FileSegment::State::DOWNLOADING)); chassert(!current_downloader.empty() == (download_state == FileSegment::State::DOWNLOADING)); - chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0); + chassert(download_state != FileSegment::State::DOWNLOADED || fs::file_size(getPathInLocalCache()) > 0); } void FileSegment::throwIfDetachedUnlocked(std::unique_lock & segment_lock) const diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index de5e3f1428c..af73fadebbf 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -61,7 +61,7 @@ public: /// the iterator should automatically point to the next record. virtual void removeAndGetNext(std::lock_guard &) = 0; - virtual void incrementSize(size_t, std::lock_guard &) = 0; + virtual void updateSize(ssize_t, std::lock_guard &) = 0; }; public: diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 4987d143f52..5623b8443f7 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -94,11 +94,19 @@ void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guar queue_iter = cache_priority->queue.erase(queue_iter); } -void LRUFileCachePriority::LRUFileCacheIterator::incrementSize(size_t size_increment, std::lock_guard &) +void LRUFileCachePriority::LRUFileCacheIterator::updateSize(ssize_t size, std::lock_guard &) { - cache_priority->cache_size += size_increment; - CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size_increment); - queue_iter->size += size_increment; + cache_priority->cache_size += size; + + if (size > 0) + CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); + else + CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, size); + + queue_iter->size += size; + + chassert(queue_iter->size > 0); + chassert(cache_priority->cache_size >= 0); } void LRUFileCachePriority::LRUFileCacheIterator::use(std::lock_guard &) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 2a3c4f3c801..949e83c2a28 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -54,7 +54,7 @@ public: void removeAndGetNext(std::lock_guard &) override; - void incrementSize(size_t size_increment, std::lock_guard &) override; + void updateSize(ssize_t size, std::lock_guard &) override; void use(std::lock_guard &) override; From 38e4148c0a4472a8de7dde31f73b476bbd454128 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 24 Apr 2023 15:44:32 +0200 Subject: [PATCH 14/34] Disable in debug --- .../02581_share_big_sets_between_mutation_tasks_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql index 97cf979e80a..21ff453cd8e 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-tsan, no-asan, no-ubsan, no-msan +-- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan DROP TABLE IF EXISTS 02581_trips; From 125736dbccc32af850414be26f042e6294c7fbc2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:06:30 +0200 Subject: [PATCH 15/34] More cases from long test to short test --- ..._big_sets_between_mutation_tasks.reference | 39 +++++++++++++++ ..._share_big_sets_between_mutation_tasks.sql | 49 +++++++++++++++++-- 2 files changed, 83 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.reference b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.reference index 18e83d1244a..452e0e0801e 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.reference +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.reference @@ -17,3 +17,42 @@ SELECT count(), _part FROM 02581_trips WHERE description = '' GROUP BY _part ORD 8000 all_2_2_0_6 8000 all_3_3_0_6 8000 all_4_4_0_6 +-- Run mutation with `id 'IN big subquery' +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +28000 +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10 + 1)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2, max_rows_in_set=1000; +SELECT count() from 02581_trips WHERE description = ''; +28000 +-- Run mutation with func(`id`) IN big subquery +ALTER TABLE 02581_trips UPDATE description='b' WHERE id::UInt64 IN (SELECT (number*10 + 2)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +28000 +-- Run mutation with non-PK `id2` IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +24000 +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) OR + (id2 IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +20000 +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) OR + (id2::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +16000 +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id::UInt32 IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) OR + ((id2+1)::String IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; +12000 diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql index fc90582d20e..7b52a89b16f 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS 02581_trips; -CREATE TABLE 02581_trips(id UInt32, description String) ENGINE=MergeTree ORDER BY id; +CREATE TABLE 02581_trips(id UInt32, id2 UInt32, description String) ENGINE=MergeTree ORDER BY id; -- Make multiple parts -INSERT INTO 02581_trips SELECT number, '' FROM numbers(10000); -INSERT INTO 02581_trips SELECT number+10000, '' FROM numbers(10000); -INSERT INTO 02581_trips SELECT number+20000, '' FROM numbers(10000); -INSERT INTO 02581_trips SELECT number+30000, '' FROM numbers(10000); +INSERT INTO 02581_trips SELECT number, number, '' FROM numbers(10000); +INSERT INTO 02581_trips SELECT number+10000, number+10000, '' FROM numbers(10000); +INSERT INTO 02581_trips SELECT number+20000, number+20000, '' FROM numbers(10000); +INSERT INTO 02581_trips SELECT number+30000, number+30000, '' FROM numbers(10000); -- { echoOn } SELECT count(), _part FROM 02581_trips GROUP BY _part ORDER BY _part; @@ -16,6 +16,45 @@ ALTER TABLE 02581_trips UPDATE description='1' WHERE id IN (SELECT (number*10+1) SELECT count(), _part FROM 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part; ALTER TABLE 02581_trips UPDATE description='2' WHERE id IN (SELECT (number*10+2)::UInt32 FROM numbers(10000)) SETTINGS mutations_sync=2; SELECT count(), _part FROM 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part; + +-- Run mutation with `id 'IN big subquery' +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; + +ALTER TABLE 02581_trips UPDATE description='a' WHERE id IN (SELECT (number*10 + 1)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2, max_rows_in_set=1000; +SELECT count() from 02581_trips WHERE description = ''; + +-- Run mutation with func(`id`) IN big subquery +ALTER TABLE 02581_trips UPDATE description='b' WHERE id::UInt64 IN (SELECT (number*10 + 2)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; + +-- Run mutation with non-PK `id2` IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' WHERE id2 IN (SELECT (number*10 + 3)::UInt32 FROM numbers(10000000)) SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; + +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) OR + (id2 IN (SELECT (number*10 + 4)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; + +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) OR + (id2::UInt64 IN (SELECT (number*10 + 5)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; + +-- Run mutation with PK and non-PK IN big subquery +ALTER TABLE 02581_trips UPDATE description='c' +WHERE + (id::UInt32 IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) OR + ((id2+1)::String IN (SELECT (number*10 + 6)::UInt32 FROM numbers(10000000))) +SETTINGS mutations_sync=2; +SELECT count() from 02581_trips WHERE description = ''; -- { echoOff } DROP TABLE 02581_trips; From aa81937eeea7f17000ec2cd7682d723907584fcb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Apr 2023 00:31:55 +0000 Subject: [PATCH 16/34] add field with number of rows to async insert log --- src/Interpreters/AsynchronousInsertLog.cpp | 2 ++ src/Interpreters/AsynchronousInsertLog.h | 1 + src/Interpreters/AsynchronousInsertQueue.cpp | 4 +++- .../0_stateless/02456_async_inserts_logs.reference | 12 ++++++------ .../queries/0_stateless/02456_async_inserts_logs.sh | 2 +- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertLog.cpp b/src/Interpreters/AsynchronousInsertLog.cpp index 916ec8f3d56..eeccd9ad92e 100644 --- a/src/Interpreters/AsynchronousInsertLog.cpp +++ b/src/Interpreters/AsynchronousInsertLog.cpp @@ -36,6 +36,7 @@ NamesAndTypesList AsynchronousInsertLogElement::getNamesAndTypes() {"format", std::make_shared(std::make_shared())}, {"query_id", std::make_shared()}, {"bytes", std::make_shared()}, + {"rows", std::make_shared()}, {"exception", std::make_shared()}, {"status", type_status}, @@ -71,6 +72,7 @@ void AsynchronousInsertLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(insert_query.format); columns[i++]->insert(query_id); columns[i++]->insert(bytes); + columns[i++]->insert(rows); columns[i++]->insert(exception); columns[i++]->insert(status); diff --git a/src/Interpreters/AsynchronousInsertLog.h b/src/Interpreters/AsynchronousInsertLog.h index e2fdd4c90a0..f1e39db1ded 100644 --- a/src/Interpreters/AsynchronousInsertLog.h +++ b/src/Interpreters/AsynchronousInsertLog.h @@ -24,6 +24,7 @@ struct AsynchronousInsertLogElement ASTPtr query; String query_id; UInt64 bytes{}; + UInt64 rows{}; String exception; Status status{}; diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index b8de0246ae2..76f956341fd 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -444,7 +444,8 @@ try { auto buffer = std::make_unique(entry->bytes); current_entry = entry; - total_rows += executor.execute(*buffer); + size_t num_rows = executor.execute(*buffer); + total_rows += num_rows; chunk_info->offsets.push_back(total_rows); /// Keep buffer, because it still can be used @@ -459,6 +460,7 @@ try elem.query = key.query; elem.query_id = entry->query_id; elem.bytes = entry->bytes.size(); + elem.rows = num_rows; elem.exception = current_exception; current_exception.clear(); diff --git a/tests/queries/0_stateless/02456_async_inserts_logs.reference b/tests/queries/0_stateless/02456_async_inserts_logs.reference index efd8a88eca4..79f7ea458a7 100644 --- a/tests/queries/0_stateless/02456_async_inserts_logs.reference +++ b/tests/queries/0_stateless/02456_async_inserts_logs.reference @@ -1,7 +1,7 @@ 5 - Values 21 1 Ok 1 -t_async_inserts_logs JSONEachRow 39 1 Ok 1 -t_async_inserts_logs Values 8 1 Ok 1 -t_async_inserts_logs JSONEachRow 6 0 ParsingError 1 -t_async_inserts_logs Values 6 0 ParsingError 1 -t_async_inserts_logs Values 8 0 FlushError 1 + Values 21 2 1 Ok 1 +t_async_inserts_logs JSONEachRow 39 2 1 Ok 1 +t_async_inserts_logs Values 8 1 1 Ok 1 +t_async_inserts_logs JSONEachRow 6 0 0 ParsingError 1 +t_async_inserts_logs Values 6 0 0 ParsingError 1 +t_async_inserts_logs Values 8 1 0 FlushError 1 diff --git a/tests/queries/0_stateless/02456_async_inserts_logs.sh b/tests/queries/0_stateless/02456_async_inserts_logs.sh index 006455e2d42..2e4db67c069 100755 --- a/tests/queries/0_stateless/02456_async_inserts_logs.sh +++ b/tests/queries/0_stateless/02456_async_inserts_logs.sh @@ -30,7 +30,7 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_async_inserts_logs" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -q " - SELECT table, format, bytes, empty(exception), status, + SELECT table, format, bytes, rows, empty(exception), status, status = 'ParsingError' ? flush_time_microseconds = 0 : flush_time_microseconds > event_time_microseconds AS time_ok FROM system.asynchronous_insert_log WHERE database = '$CLICKHOUSE_DATABASE' OR query ILIKE 'INSERT INTO FUNCTION%$CLICKHOUSE_DATABASE%' From cc5acfbe62a52c2b651794db9c8173e0d8be0ea5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Apr 2023 01:37:37 +0000 Subject: [PATCH 17/34] add profile event --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/AsynchronousInsertQueue.cpp | 2 ++ tests/queries/0_stateless/02456_async_inserts_logs.reference | 3 +++ tests/queries/0_stateless/02456_async_inserts_logs.sh | 5 +++++ 4 files changed, 11 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index da096085d5b..166db25e14c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -10,6 +10,7 @@ M(InsertQuery, "Same as Query, but only for INSERT queries.") \ M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \ M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \ + M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \ M(AsyncInsertCacheHits, "Number of times a duplicate hash id has been found in asynchronous INSERT hash id cache.") \ M(FailedQuery, "Number of failed queries.") \ M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 76f956341fd..88233f58a54 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -40,6 +40,7 @@ namespace ProfileEvents { extern const Event AsyncInsertQuery; extern const Event AsyncInsertBytes; + extern const Event AsyncInsertRows; extern const Event FailedAsyncInsertQuery; } @@ -481,6 +482,7 @@ try format->addBuffer(std::move(last_buffer)); auto insert_query_id = insert_context->getCurrentQueryId(); + ProfileEvents::increment(ProfileEvents::AsyncInsertRows, total_rows); auto finish_entries = [&] { diff --git a/tests/queries/0_stateless/02456_async_inserts_logs.reference b/tests/queries/0_stateless/02456_async_inserts_logs.reference index 79f7ea458a7..ba1b19fb184 100644 --- a/tests/queries/0_stateless/02456_async_inserts_logs.reference +++ b/tests/queries/0_stateless/02456_async_inserts_logs.reference @@ -5,3 +5,6 @@ t_async_inserts_logs Values 8 1 1 Ok 1 t_async_inserts_logs JSONEachRow 6 0 0 ParsingError 1 t_async_inserts_logs Values 6 0 0 ParsingError 1 t_async_inserts_logs Values 8 1 0 FlushError 1 +AsyncInsertBytes 1 +AsyncInsertQuery 1 +AsyncInsertRows 1 diff --git a/tests/queries/0_stateless/02456_async_inserts_logs.sh b/tests/queries/0_stateless/02456_async_inserts_logs.sh index 2e4db67c069..43cd73d7231 100755 --- a/tests/queries/0_stateless/02456_async_inserts_logs.sh +++ b/tests/queries/0_stateless/02456_async_inserts_logs.sh @@ -37,3 +37,8 @@ ${CLICKHOUSE_CLIENT} -q " ORDER BY table, status, format" ${CLICKHOUSE_CLIENT} -q "DROP TABLE t_async_inserts_logs" + +${CLICKHOUSE_CLIENT} -q " +SELECT event, value > 0 FROM system.events +WHERE event IN ('AsyncInsertQuery', 'AsyncInsertBytes', 'AsyncInsertRows') +ORDER BY event" From 7ec2de0d3fdd9c4cd654873e32ff06a5b717ef9a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Apr 2023 08:41:40 +0200 Subject: [PATCH 18/34] Fix building iceberg without avro Signed-off-by: Azat Khuzhin --- src/Storages/DataLakes/StorageIceberg.h | 4 ++-- src/Storages/DataLakes/registerDataLakes.cpp | 4 ++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/DataLakes/StorageIceberg.h b/src/Storages/DataLakes/StorageIceberg.h index 6e93c732338..f1c9c485ef0 100644 --- a/src/Storages/DataLakes/StorageIceberg.h +++ b/src/Storages/DataLakes/StorageIceberg.h @@ -5,7 +5,7 @@ #include #include "config.h" -#if USE_AWS_S3 +#if USE_AWS_S3 && USE_AVRO #include #include #endif @@ -18,7 +18,7 @@ struct StorageIcebergName static constexpr auto name = "Iceberg"; }; -#if USE_AWS_S3 +#if USE_AWS_S3 && USE_AVRO using StorageIcebergS3 = IStorageDataLake>; #endif diff --git a/src/Storages/DataLakes/registerDataLakes.cpp b/src/Storages/DataLakes/registerDataLakes.cpp index 508cf9c8d80..1447a4777c5 100644 --- a/src/Storages/DataLakes/registerDataLakes.cpp +++ b/src/Storages/DataLakes/registerDataLakes.cpp @@ -31,11 +31,15 @@ void registerStorageDeltaLake(StorageFactory & factory) } #endif +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. + void registerStorageIceberg(StorageFactory & factory) { REGISTER_DATA_LAKE_STORAGE(StorageIcebergS3, StorageIcebergName::name) } +#endif + void registerStorageHudi(StorageFactory & factory) { REGISTER_DATA_LAKE_STORAGE(StorageHudiS3, StorageHudiName::name) From a0791e3187d52c70da80ca00bcd9823d6cbe1d86 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 25 Apr 2023 11:53:59 +0200 Subject: [PATCH 19/34] Update DatabasePostgreSQL.cpp --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index d9116f74c52..4b42d799661 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; From 6032bacd0fbc6178023290a832fe4c70f4d36902 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 25 Apr 2023 12:36:41 +0200 Subject: [PATCH 20/34] Get unanonymized logs in workflow_jobs_lambda --- tests/ci/workflow_jobs_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index 9436e01ad53..6ac75bcb8e5 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -284,10 +284,10 @@ def handler(event: dict, _: Any) -> dict: wf_job["runner_group_name"] or "", # nullable repo["full_name"], ) + logging.info("Got the next event: %s", workflow_job) if repo["private"]: workflow_job.anonimyze() - logging.info("Got the next event: %s", workflow_job) send_event_workflow_job(workflow_job) return { From 2ab313e6b4272f5888bc7d3b54533b3f8ba86fba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 25 Apr 2023 12:42:06 +0200 Subject: [PATCH 21/34] Update the style_checker upscale speed --- tests/ci/autoscale_runners_lambda/app.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 1fcdbc40155..cbc9f4f8901 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -64,7 +64,10 @@ def get_scales(runner_type: str) -> Tuple[int, int]: if runner_type == "style-checker": # the style checkers have so many noise, so it scales up too quickly scale_down = 1 - scale_up = 10 + # The 5 was too quick, there are complainings regarding too slow with + # 10. I am trying 7 now. + # UPDATE THE COMMENT ON CHANGES + scale_up = 7 return scale_down, scale_up From d7f759ce381bd8d5c732d8b1f2f99e2272d76e69 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 25 Apr 2023 11:31:53 +0000 Subject: [PATCH 22/34] Docs: Make caption of processors_profile_log page consistent with other pages --- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/processors_profile_log.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 66a24ac3fca..4cef8d724c9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -890,7 +890,7 @@ Write time that processor spent during execution/waiting for data to `system.pro See also: -- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#system-processors_profile_log) +- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#processors_profile_log) - [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) ## max_insert_block_size {#settings-max_insert_block_size} diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index e849525e495..0532acd4288 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -1,4 +1,4 @@ -# system.processors_profile_log {#system-processors_profile_log} +# processors_profile_log {#processors_profile_log} This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)). @@ -73,4 +73,4 @@ Here you can see: **See Also** -- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) \ No newline at end of file +- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) From 3255f937f27aee295a4363e5676a5558a98b61c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 25 Apr 2023 11:34:10 +0000 Subject: [PATCH 23/34] Remove anchor --- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/processors_profile_log.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4cef8d724c9..c6fdcf317c3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -890,7 +890,7 @@ Write time that processor spent during execution/waiting for data to `system.pro See also: -- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#processors_profile_log) +- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md) - [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) ## max_insert_block_size {#settings-max_insert_block_size} diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index 0532acd4288..a6ff15642a1 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -1,4 +1,4 @@ -# processors_profile_log {#processors_profile_log} +# processors_profile_log This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)). From 6fecdef0fedfa72efd8fdd3afaced80b396bbb1e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Apr 2023 15:35:28 +0200 Subject: [PATCH 24/34] Fix glibc compatibility check --- base/glibc-compatibility/glibc-compatibility.c | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/base/glibc-compatibility/glibc-compatibility.c b/base/glibc-compatibility/glibc-compatibility.c index 7e8ea5051d7..49bb81a58be 100644 --- a/base/glibc-compatibility/glibc-compatibility.c +++ b/base/glibc-compatibility/glibc-compatibility.c @@ -235,6 +235,17 @@ ssize_t getrandom(void *buf, size_t buflen, unsigned flags) return syscall(SYS_getrandom, buf, buflen, flags); } +/* Structure for scatter/gather I/O. */ +struct iovec +{ + void *iov_base; /* Pointer to data. */ + size_t iov_len; /* Length of data. */ +}; + +ssize_t preadv(int __fd, const struct iovec *__iovec, int __count, __off_t __offset) +{ + return syscall(SYS_preadv, __fd, __iovec, __count, (long)(__offset), (long)(__offset>>32)); +} #include #include From bb2146114503258c4b1b6dd2f8ef34c45110a498 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Apr 2023 15:45:23 +0200 Subject: [PATCH 25/34] Try to fix with pie --- CMakeLists.txt | 7 +++++-- cmake/sanitize.cmake | 3 +-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index cc1a64a9e96..a59ecacd89d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -421,8 +421,11 @@ endif () set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") -set (CMAKE_POSITION_INDEPENDENT_CODE OFF) -if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X)) +if (NOT SANITIZE) + set (CMAKE_POSITION_INDEPENDENT_CODE OFF) +endif + +if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X) AND NOT SANITIZE) # Slightly more efficient code can be generated # It's disabled for ARM because otherwise ClickHouse cannot run on Android. set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 13aaa414b93..fc9793d8f35 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -33,8 +33,7 @@ if (SANITIZE) # RelWithDebInfo, and downgrade optimizations to -O1 but not to -Og, to # keep the binary size down. # TODO: try compiling with -Og and with ld.gold. - set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt") - + set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") From 0fcf19efd31a01b14ed5d5e1b13f340e390e8392 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Apr 2023 15:47:47 +0200 Subject: [PATCH 26/34] Missed brackets --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index a59ecacd89d..0554403cce5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -423,7 +423,7 @@ set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") if (NOT SANITIZE) set (CMAKE_POSITION_INDEPENDENT_CODE OFF) -endif +endif() if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X) AND NOT SANITIZE) # Slightly more efficient code can be generated From 0caa34f55e4e3d2de8b03b003e895d2434fceb16 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 25 Apr 2023 15:50:13 +0200 Subject: [PATCH 27/34] Do not randomize prefetch settings for debug build (#49134) * Do not randomize prefetch settings for debug build * Update tests/clickhouse-test --------- Co-authored-by: Alexander Tokmakov Co-authored-by: Alexander Tokmakov --- tests/clickhouse-test | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index eb5faa1ffb5..7c492a9b467 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -579,10 +579,17 @@ class SettingsRandomizer: } @staticmethod - def get_random_settings(): + def get_random_settings(args): random_settings = [] + is_debug = BuildFlags.DEBUG in args.build_flags for setting, generator in SettingsRandomizer.settings.items(): - random_settings.append(f"{setting}={generator()}") + if ( + is_debug + and setting == "allow_prefetched_read_pool_for_remote_filesystem" + ): + random_settings.append(f"{setting}=0") + else: + random_settings.append(f"{setting}={generator()}") return random_settings @@ -817,7 +824,7 @@ class TestCase: ) if self.randomize_settings: - self.random_settings = SettingsRandomizer.get_random_settings() + self.random_settings = SettingsRandomizer.get_random_settings(args) if self.randomize_merge_tree_settings: self.merge_tree_random_settings = ( From 2495e30b7d4a181107d07961cb7ae731e8a72f32 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Apr 2023 18:15:07 +0200 Subject: [PATCH 28/34] Bump sysroot --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index f0081b2649b..e0d1b64da66 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit f0081b2649b94837855f3bc7d05ef326b100bad8 +Subproject commit e0d1b64da666afbfaa6f1ee0487c33f3fd2cd5cb From ed8ca018bd38e469cede86d1929f860079be8160 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 25 Apr 2023 19:11:20 +0200 Subject: [PATCH 29/34] Add a note regarding private/public repo to logs --- tests/ci/workflow_jobs_lambda/app.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index 6ac75bcb8e5..49d475d11dc 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -284,7 +284,9 @@ def handler(event: dict, _: Any) -> dict: wf_job["runner_group_name"] or "", # nullable repo["full_name"], ) - logging.info("Got the next event: %s", workflow_job) + logging.info( + "Got the next event (private_repo=%s): %s", repo["private"], workflow_job + ) if repo["private"]: workflow_job.anonimyze() From 9b0e4835cd583f96809a3553d50d39aab7358704 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Apr 2023 22:57:34 +0200 Subject: [PATCH 30/34] Add changelog for 23.4 --- CHANGELOG.md | 150 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 150 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 47320208f02..482ca3cec73 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.4, 2023-04-26](#234)**
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**
**[ClickHouse release v23.2, 2023-02-23](#232)**
**[ClickHouse release v23.1, 2023-01-25](#231)**
@@ -6,6 +7,155 @@ # 2023 Changelog +### ClickHouse release 23.4 LTS, 2023-04-26 + +#### Backward Incompatible Change +* Formatter '%M' in function formatDateTime() now prints the month name instead of the minutes. This makes the behavior consistent with MySQL. The previous behavior can be restored using setting "formatdatetime_parsedatetime_m_is_month_name = 0". [#47246](https://github.com/ClickHouse/ClickHouse/pull/47246) ([Robert Schulze](https://github.com/rschu1ze)). +* This change makes sense only if you are using the virtual filesystem cache. If `path` in the virtual filesystem cache configuration is not empty and is not an absolute path, then it will be put in `/caches/`. [#48784](https://github.com/ClickHouse/ClickHouse/pull/48784) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Primary/secondary indices and sorting keys with identical expressions are now rejected. This behavior can be disabled using setting `allow_suspicious_indices`. [#48536](https://github.com/ClickHouse/ClickHouse/pull/48536) ([凌涛](https://github.com/lingtaolf)). + +#### New Feature +* Support new aggregate function `quantileGK`/`quantilesGK`, like [approx_percentile](https://spark.apache.org/docs/latest/api/sql/index.html#approx_percentile) in spark. Greenwald-Khanna algorithm refer to http://infolab.stanford.edu/~datar/courses/cs361a/papers/quantiles.pdf. [#46428](https://github.com/ClickHouse/ClickHouse/pull/46428) ([李扬](https://github.com/taiyang-li)). +* Add a statement `SHOW COLUMNS` which shows distilled information from system.columns. [#48017](https://github.com/ClickHouse/ClickHouse/pull/48017) ([Robert Schulze](https://github.com/rschu1ze)). +* Added `LIGHTWEIGHT` and `PULL` modifiers for `SYSTEM SYNC REPLICA` query. `LIGHTWEIGHT` version waits for fetches and drop-ranges only (merges and mutations are ignored). `PULL` version pulls new entries from ZooKeeper and does not wait for them. Fixes [#47794](https://github.com/ClickHouse/ClickHouse/issues/47794). [#48085](https://github.com/ClickHouse/ClickHouse/pull/48085) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add `kafkaMurmurHash` function for compatibility with Kafka DefaultPartitioner. Closes [#47834](https://github.com/ClickHouse/ClickHouse/issues/47834). [#48185](https://github.com/ClickHouse/ClickHouse/pull/48185) ([Nikolay Degterinsky](https://github.com/evillique)). +* Allow to easily create a user with the same grants as the current user by using `GRANT CURRENT GRANTS`. [#48262](https://github.com/ClickHouse/ClickHouse/pull/48262) ([pufit](https://github.com/pufit)). +* Add statistical aggregate function `kolmogorovSmirnovTest`. Close [#48228](https://github.com/ClickHouse/ClickHouse/issues/48228). [#48325](https://github.com/ClickHouse/ClickHouse/pull/48325) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added a `lost_part_count` column to the `system.replicas` table. The column value shows the total number of lost parts in the corresponding table. Value is stored in zookeeper and can be used instead of not persistent `ReplicatedDataLoss` profile event for monitoring. [#48526](https://github.com/ClickHouse/ClickHouse/pull/48526) ([Sergei Trifonov](https://github.com/serxa)). +* Add `soundex` function for compatibility. Closes [#39880](https://github.com/ClickHouse/ClickHouse/issues/39880). [#48567](https://github.com/ClickHouse/ClickHouse/pull/48567) ([FriendLey](https://github.com/FriendLey)). +* Support `Map` type for JSONExtract. [#48629](https://github.com/ClickHouse/ClickHouse/pull/48629) ([李扬](https://github.com/taiyang-li)). +* Add `PrettyJSONEachRow` format to output pretty JSON with new line delimieters and 4 space indents. [#48898](https://github.com/ClickHouse/ClickHouse/pull/48898) ([Kruglov Pavel](https://github.com/Avogar)). +* Add `ParquetMetadata` input format to read Parquet file metadata. [#48911](https://github.com/ClickHouse/ClickHouse/pull/48911) ([Kruglov Pavel](https://github.com/Avogar)). +* Add `extractKeyValuePairs` function to extract key value pairs from strings. Input strings might contain noise (i.e log files / do not need to be 100% formatted in key-value-pair format), the algorithm will look for key value pairs matching the arguments passed to the function. As of now, function accepts the following arguments: `data_column` (mandatory), `key_value_pair_delimiter` (defaults to `:`), `pair_delimiters` (defaults to `\space \, \;`) and `quoting_character` (defaults to double quotes). [#43606](https://github.com/ClickHouse/ClickHouse/pull/43606) ([Arthur Passos](https://github.com/arthurpassos)). +* Functions replaceOne(), replaceAll(), replaceRegexpOne() and replaceRegexpAll() can now be called with non-const pattern and replacement arguments. [#46589](https://github.com/ClickHouse/ClickHouse/pull/46589) ([Robert Schulze](https://github.com/rschu1ze)). +* Added functions to work with columns of type `Map`: `mapConcat`, `mapSort`, `mapExists`. [#48071](https://github.com/ClickHouse/ClickHouse/pull/48071) ([Anton Popov](https://github.com/CurtizJ)). + +#### Performance Improvement +* Reading files in `Parquet` format is now much faster. IO and decoding are parallelized (controlled by `max_threads` setting), and only required data ranges are read. [#47964](https://github.com/ClickHouse/ClickHouse/pull/47964) ([Michael Kolupaev](https://github.com/al13n321)). +* If we run a mutation with IN (subquery) like this: `ALTER TABLE t UPDATE col='new value' WHERE id IN (SELECT id FROM huge_table)` and the table `t` has multiple parts than for each part a set for subquery `SELECT id FROM huge_table` is built in memory. And if there are many parts then this might consume a lot of memory (and lead to an OOM) and CPU. The solution is to introduce a short-lived cache of sets that are currently being built by mutation tasks. If another task of the same mutation is executed concurrently it can lookup the set in the cache, wait for it to be built and reuse it. [#46835](https://github.com/ClickHouse/ClickHouse/pull/46835) ([Alexander Gololobov](https://github.com/davenger)). +* Only check dependencies if necessary when applying `ALTER TABLE` queries. [#48062](https://github.com/ClickHouse/ClickHouse/pull/48062) ([Raúl Marín](https://github.com/Algunenano)). +* Optimize function `mapUpdate`. [#48118](https://github.com/ClickHouse/ClickHouse/pull/48118) ([Anton Popov](https://github.com/CurtizJ)). +* Now an internal query to local replica is sent explicitly and data from it received through loopback interface. Setting `prefer_localhost_replica` is not respected for parallel replicas. This is needed for better scheduling and makes the code cleaner: the initiator is only responsible for coordinating of the reading process and merging results, continiously answering for requests while all the secondary queries read the data. Note: Using loopback interface is not so performant, otherwise some replicas could starve for tasks which could lead to even slower query execution and not utilizing all possible resources. The initialization of the coordinator is now even more lazy. All incoming requests contain the information about the reading algorithm we initialize the coordinator with it when first request comes. If any replica will decide to read with different algorithm - an exception will be thrown and a query will be aborted. [#48246](https://github.com/ClickHouse/ClickHouse/pull/48246) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Do not build set for the right side of `IN` clause with subquery when it is used only for analysis of skip indexes and they are disabled by setting (`use_skip_indexes=0`). Previously it might affect the performance of queries. [#48299](https://github.com/ClickHouse/ClickHouse/pull/48299) ([Anton Popov](https://github.com/CurtizJ)). +* Query processing is parallelized right after reading `FROM file(...)`. Related to [#38755](https://github.com/ClickHouse/ClickHouse/issues/38755). [#48525](https://github.com/ClickHouse/ClickHouse/pull/48525) ([Igor Nikonov](https://github.com/devcrafter)). +* Query processing is parallelized right after reading from a data source. Affected data sources are mostly simple or external storages like table functions `url`, `file`. [#48727](https://github.com/ClickHouse/ClickHouse/pull/48727) ([Igor Nikonov](https://github.com/devcrafter)). +* Lowered contention of ThreadPool mutex (may increase performance for a huge amount of small jobs). [#48750](https://github.com/ClickHouse/ClickHouse/pull/48750) ([Sergei Trifonov](https://github.com/serxa)). +* Reduce memory usage for multiple `ALTER DELETE` mutations. [#48522](https://github.com/ClickHouse/ClickHouse/pull/48522) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove the excessive connection attempts if the `skip_unavailable_shards` setting is enabled. [#48771](https://github.com/ClickHouse/ClickHouse/pull/48771) ([Azat Khuzhin](https://github.com/azat)). + +#### Experimental Feature +* Entries in the query cache are now squashed to max_block_size and compressed. [#45912](https://github.com/ClickHouse/ClickHouse/pull/45912) ([Robert Schulze](https://github.com/rschu1ze)). +* It is now possible to define per-user quotas in the query cache. [#48284](https://github.com/ClickHouse/ClickHouse/pull/48284) ([Robert Schulze](https://github.com/rschu1ze)). +* Some fixes for parallel replicas [#48433](https://github.com/ClickHouse/ClickHouse/pull/48433) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Implement zero-copy-replication (an experimental feature) on encrypted disks. [#48741](https://github.com/ClickHouse/ClickHouse/pull/48741) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Improvement +* Increase default value for `connect_timeout_with_failover_ms` to 1000 ms (because of adding async connections in https://github.com/ClickHouse/ClickHouse/pull/47229) . Closes [#5188](https://github.com/ClickHouse/ClickHouse/issues/5188). [#49009](https://github.com/ClickHouse/ClickHouse/pull/49009) ([Kruglov Pavel](https://github.com/Avogar)). +* Several improvements around data lakes: - Make `Iceberg` work with non-partitioned data. - Support `Iceberg` format version v2 (previously only v1 was supported) - Support reading partitioned data for `DeltaLake`/`Hudi` - Faster reading of `DeltaLake` metadata by using Delta's checkpoint files - Fixed incorrect `Hudi` reads: previously it incorrectly chose which data to read and therefore was able to read correctly only small size tables - Made these engines to pickup updates of changed data (previously the state was set on table creation) - Make proper testing for `Iceberg`/`DeltaLake`/`Hudi` using spark. [#47307](https://github.com/ClickHouse/ClickHouse/pull/47307) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add async connection to socket and async writing to socket. Make creating connections and sending query/external tables async across shards. Refactor code with fibers. Closes [#46931](https://github.com/ClickHouse/ClickHouse/issues/46931). We will be able to increase `connect_timeout_with_failover_ms` by default after this PR (https://github.com/ClickHouse/ClickHouse/issues/5188). [#47229](https://github.com/ClickHouse/ClickHouse/pull/47229) ([Kruglov Pavel](https://github.com/Avogar)). +* Support config sections `keeper`/`keeper_server` as an alternative to `zookeeper`. Close [#34766](https://github.com/ClickHouse/ClickHouse/issues/34766) , [#34767](https://github.com/ClickHouse/ClickHouse/issues/34767). [#35113](https://github.com/ClickHouse/ClickHouse/pull/35113) ([李扬](https://github.com/taiyang-li)). +* It is possible to set _secure_ flag in named_collections for a dictionary with a ClickHouse table source. Addresses [#38450](https://github.com/ClickHouse/ClickHouse/issues/38450) . [#46323](https://github.com/ClickHouse/ClickHouse/pull/46323) ([Ilya Golshtein](https://github.com/ilejn)). +* `bitCount` function support `FixedString` and `String` data type. [#49044](https://github.com/ClickHouse/ClickHouse/pull/49044) ([flynn](https://github.com/ucasfl)). +* Added configurable retries for all operations with [Zoo]Keeper for Backup queries. [#47224](https://github.com/ClickHouse/ClickHouse/pull/47224) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Enable `use_environment_credentials` for S3 by default, so the entire provider chain is constructed by default. [#47397](https://github.com/ClickHouse/ClickHouse/pull/47397) ([Antonio Andelic](https://github.com/antonio2368)). +* Currently, the JSON_VALUE function is similar as spark's get_json_object function, which support to get value from json string by a path like '$.key'. But still has something different - 1. in spark's get_json_object will return null while the path is not exist, but in JSON_VALUE will return empty string; - 2. in spark's get_json_object will return a complext type value, such as a json object/array value, but in JSON_VALUE will return empty string. [#47494](https://github.com/ClickHouse/ClickHouse/pull/47494) ([KevinyhZou](https://github.com/KevinyhZou)). +* For `use_structure_from_insertion_table_in_table_functions` more flexible insert table structure propagation to table function. Fixed an issue with name mapping and using virtual columns. No more need for 'auto' setting. [#47962](https://github.com/ClickHouse/ClickHouse/pull/47962) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Do not continue retrying to connect to ZK if the query is killed or over limits. [#47985](https://github.com/ClickHouse/ClickHouse/pull/47985) ([Raúl Marín](https://github.com/Algunenano)). +* Support Enum output/input in `BSONEachRow`, allow all map key types and avoid extra calculations on output. [#48122](https://github.com/ClickHouse/ClickHouse/pull/48122) ([Kruglov Pavel](https://github.com/Avogar)). +* Support more ClickHouse types in `ORC`/`Arrow`/`Parquet` formats: Enum(8|16), (U)Int(128|256), Decimal256 (for ORC), allow reading IPv4 from Int32 values (ORC outputs IPv4 as Int32 and we couldn't read it back), fix reading Nullable(IPv6) from binary data for `ORC`. [#48126](https://github.com/ClickHouse/ClickHouse/pull/48126) ([Kruglov Pavel](https://github.com/Avogar)). +* Add columns `perform_ttl_move_on_insert`, `load_balancing` for table `system.storage_policies`, modify column `volume_type` type to `Enum8`. [#48167](https://github.com/ClickHouse/ClickHouse/pull/48167) ([lizhuoyu5](https://github.com/lzydmxy)). +* Added support for `BACKUP ALL` command which backups all tables and databases, including temporary and system ones. [#48189](https://github.com/ClickHouse/ClickHouse/pull/48189) ([Vitaly Baranov](https://github.com/vitlibar)). +* Function mapFromArrays supports `Map` type as an input. [#48207](https://github.com/ClickHouse/ClickHouse/pull/48207) ([李扬](https://github.com/taiyang-li)). +* The output of some SHOW PROCESSLIST is now sorted. [#48241](https://github.com/ClickHouse/ClickHouse/pull/48241) ([Robert Schulze](https://github.com/rschu1ze)). +* Per-query/per-server throttling for remote IO/local IO/BACKUPs (server settings: `max_remote_read_network_bandwidth_for_server`, `max_remote_write_network_bandwidth_for_server`, `max_local_read_bandwidth_for_server`, `max_local_write_bandwidth_for_server`, `max_backup_bandwidth_for_server`, settings: `max_remote_read_network_bandwidth`, `max_remote_write_network_bandwidth`, `max_local_read_bandwidth`, `max_local_write_bandwidth`, `max_backup_bandwidth`). [#48242](https://github.com/ClickHouse/ClickHouse/pull/48242) ([Azat Khuzhin](https://github.com/azat)). +* Support more types in `CapnProto` format: Map, (U)Int(128|256), Decimal(128|256). Allow integer conversions during input/output. [#48257](https://github.com/ClickHouse/ClickHouse/pull/48257) ([Kruglov Pavel](https://github.com/Avogar)). +* Don't throw CURRENT_WRITE_BUFFER_IS_EXHAUSTED for normal behaviour. [#48288](https://github.com/ClickHouse/ClickHouse/pull/48288) ([Raúl Marín](https://github.com/Algunenano)). +* Add new setting `keeper_map_strict_mode` which enforces extra guarantees on operations made on top of `KeeperMap` tables. [#48293](https://github.com/ClickHouse/ClickHouse/pull/48293) ([Antonio Andelic](https://github.com/antonio2368)). +* Check primary key type for simple dictionary is native unsigned integer type Add setting `check_dictionary_primary_key ` for compatibility(set `check_dictionary_primary_key =false` to disable checking). [#48335](https://github.com/ClickHouse/ClickHouse/pull/48335) ([lizhuoyu5](https://github.com/lzydmxy)). +* Don't replicate mutations for `KeeperMap` because it's unnecessary. [#48354](https://github.com/ClickHouse/ClickHouse/pull/48354) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow write/read unnamed tuple as nested Message in Protobuf format. Tuple elements and Message fields are mathced by position. [#48390](https://github.com/ClickHouse/ClickHouse/pull/48390) ([Kruglov Pavel](https://github.com/Avogar)). +* Support `additional_table_filters` and `additional_result_filter` settings in the new planner. Also, add a documentation entry for `additional_result_filter`. [#48405](https://github.com/ClickHouse/ClickHouse/pull/48405) ([Dmitry Novik](https://github.com/novikd)). +* `parseDateTime` now understands format string '%f' (fractional seconds). [#48420](https://github.com/ClickHouse/ClickHouse/pull/48420) ([Robert Schulze](https://github.com/rschu1ze)). +* Format string "%f" in formatDateTime() now prints "000000" if the formatted value has no fractional seconds, the previous behavior (single zero) can be restored using setting "formatdatetime_f_prints_single_zero = 1". [#48422](https://github.com/ClickHouse/ClickHouse/pull/48422) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't replicate DELETE and TRUNCATE for KeeperMap. [#48434](https://github.com/ClickHouse/ClickHouse/pull/48434) ([Antonio Andelic](https://github.com/antonio2368)). +* Generate valid Decimals and Bools in generateRandom function. [#48436](https://github.com/ClickHouse/ClickHouse/pull/48436) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow trailing commas in expression list of SELECT query, for example `SELECT a, b, c, FROM table`. Closes [#37802](https://github.com/ClickHouse/ClickHouse/issues/37802). [#48438](https://github.com/ClickHouse/ClickHouse/pull/48438) ([Nikolay Degterinsky](https://github.com/evillique)). +* Override `CLICKHOUSE_USER` and `CLICKHOUSE_PASSWORD` environment variables with `--user` and `--password` client parameters. Closes [#38909](https://github.com/ClickHouse/ClickHouse/issues/38909). [#48440](https://github.com/ClickHouse/ClickHouse/pull/48440) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added retries to loading of data parts in `MergeTree` tables in case of retryable errors. [#48442](https://github.com/ClickHouse/ClickHouse/pull/48442) ([Anton Popov](https://github.com/CurtizJ)). +* Add support for `Date`, `Date32`, `DateTime`, `DateTime64` data types to `arrayMin`, `arrayMax`, `arrayDifference` functions. Closes [#21645](https://github.com/ClickHouse/ClickHouse/issues/21645). [#48445](https://github.com/ClickHouse/ClickHouse/pull/48445) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add support for `{server_uuid}` macro. It is useful for identifying replicas in autoscaled clusters when new replicas are constantly added and removed in runtime. This closes [#48554](https://github.com/ClickHouse/ClickHouse/issues/48554). [#48563](https://github.com/ClickHouse/ClickHouse/pull/48563) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The installation script will create a hard link instead of copying if it is possible. [#48578](https://github.com/ClickHouse/ClickHouse/pull/48578) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support `SHOW TABLE` syntax meaning the same as `SHOW CREATE TABLE`. Closes [#48580](https://github.com/ClickHouse/ClickHouse/issues/48580). [#48591](https://github.com/ClickHouse/ClickHouse/pull/48591) ([flynn](https://github.com/ucasfl)). +* HTTP temporary buffers now support working by evicting data from the virtual filesystem cache. [#48664](https://github.com/ClickHouse/ClickHouse/pull/48664) ([Vladimir C](https://github.com/vdimir)). +* Make Schema inference works for `CREATE AS SELECT`. Closes [#47599](https://github.com/ClickHouse/ClickHouse/issues/47599). [#48679](https://github.com/ClickHouse/ClickHouse/pull/48679) ([flynn](https://github.com/ucasfl)). +* Added a `replicated_max_mutations_in_one_entry` setting for `ReplicatedMergeTree` that allows limiting the number of mutation commands per one `MUTATE_PART` entry (default is 10000). [#48731](https://github.com/ClickHouse/ClickHouse/pull/48731) ([Alexander Tokmakov](https://github.com/tavplubix)). +* In AggregateFunction types, don't count unused arena bytes as `read_bytes`. [#48745](https://github.com/ClickHouse/ClickHouse/pull/48745) ([Raúl Marín](https://github.com/Algunenano)). +* Fix some MySQL-related settings not being handled with the MySQL dictionary source + named collection. Closes [#48402](https://github.com/ClickHouse/ClickHouse/issues/48402). [#48759](https://github.com/ClickHouse/ClickHouse/pull/48759) ([Kseniia Sumarokova](https://github.com/kssenii)). +* If a user set `max_single_part_upload_size` to a very large value, it can lead to a crash due to a bug in the AWS S3 SDK. This fixes [#47679](https://github.com/ClickHouse/ClickHouse/issues/47679). [#48816](https://github.com/ClickHouse/ClickHouse/pull/48816) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data race in `RabbitMQ` ([report](https://pastila.nl/?004f7100/de1505289ab5bb355e67ebe6c7cc8707)), refactor the code. [#48845](https://github.com/ClickHouse/ClickHouse/pull/48845) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add aliases `name` and `part_name` form `system.parts` and `system.part_log`. Closes [#48718](https://github.com/ClickHouse/ClickHouse/issues/48718). [#48850](https://github.com/ClickHouse/ClickHouse/pull/48850) ([sichenzhao](https://github.com/sichenzhao)). +* Functions "arrayDifferenceSupport()", "arrayCumSum()" and "arrayCumSumNonNegative()" now support input arrays of wide integer types (U)Int128/256. [#48866](https://github.com/ClickHouse/ClickHouse/pull/48866) ([cluster](https://github.com/infdahai)). +* Multi-line history in clickhouse-client is now no longer padded. This makes pasting more natural. [#48870](https://github.com/ClickHouse/ClickHouse/pull/48870) ([Joanna Hulboj](https://github.com/jh0x)). +* Implement a slight improvement for the rare case when ClickHouse is run inside LXC and LXCFS is used. The LXCFS has an issue: sometimes it returns an error "Transport endpoint is not connected" on reading from the file inside `/proc`. This error was correctly logged into ClickHouse's server log. We have additionally workaround this issue by reopening a file. This is a minuscule change. [#48922](https://github.com/ClickHouse/ClickHouse/pull/48922) ([Real](https://github.com/RunningXie)). +* Improve memory accounting for prefetches. Randomise prefetch settings In CI. [#48973](https://github.com/ClickHouse/ClickHouse/pull/48973) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Correctly set headers for native copy operations on GCS. [#48981](https://github.com/ClickHouse/ClickHouse/pull/48981) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for specifying setting names in the command line with dashes instead of underscores, for example, `--max-threads` instead of `--max_threads`. Additionally, support Unicode dash characters like `—` instead of `--` - this is useful when you communicate with a team in another company, and a manager from that team copy-pasted code from MS Word. [#48985](https://github.com/ClickHouse/ClickHouse/pull/48985) ([alekseygolub](https://github.com/alekseygolub)). +* Add fallback to password authentication when authentication with SSL user certificate has failed. Closes [#48974](https://github.com/ClickHouse/ClickHouse/issues/48974). [#48989](https://github.com/ClickHouse/ClickHouse/pull/48989) ([Nikolay Degterinsky](https://github.com/evillique)). +* Improve the embedded dashboard. Close [#46671](https://github.com/ClickHouse/ClickHouse/issues/46671). [#49036](https://github.com/ClickHouse/ClickHouse/pull/49036) ([Kevin Zhang](https://github.com/Kinzeng)). +* Add profile events for log messages, so you can easily see the count of log messages by severity. [#49042](https://github.com/ClickHouse/ClickHouse/pull/49042) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In previous versions, the `LineAsString` format worked inconsistently when the parallel parsing was enabled or not, in presence of DOS or MacOS Classic line breaks. This closes [#49039](https://github.com/ClickHouse/ClickHouse/issues/49039). [#49052](https://github.com/ClickHouse/ClickHouse/pull/49052) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The exception message about the unparsed query parameter will also tell about the name of the parameter. Reimplement [#48878](https://github.com/ClickHouse/ClickHouse/issues/48878). Close [#48772](https://github.com/ClickHouse/ClickHouse/issues/48772). [#49061](https://github.com/ClickHouse/ClickHouse/pull/49061) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Update time zones. The following were updated: Africa/Cairo, Africa/Casablanca, Africa/El_Aaiun, America/Bogota, America/Cambridge_Bay, America/Ciudad_Juarez, America/Godthab, America/Inuvik, America/Iqaluit, America/Nuuk, America/Ojinaga, America/Pangnirtung, America/Rankin_Inlet, America/Resolute, America/Whitehorse, America/Yellowknife, Asia/Gaza, Asia/Hebron, Asia/Kuala_Lumpur, Asia/Singapore, Canada/Yukon, Egypt, Europe/Kirov, Europe/Volgograd, Singapore. [#48572](https://github.com/ClickHouse/ClickHouse/pull/48572) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Reduce the number of dependencies in the header files to speed up the build. [#47984](https://github.com/ClickHouse/ClickHouse/pull/47984) ([Dmitry Novik](https://github.com/novikd)). +* Randomize compression of marks and indices in tests. [#48286](https://github.com/ClickHouse/ClickHouse/pull/48286) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Bump internal ZSTD from 1.5.4 to 1.5.5. [#46797](https://github.com/ClickHouse/ClickHouse/pull/46797) ([Robert Schulze](https://github.com/rschu1ze)). +* Randomize vertical merges from compact to wide parts in tests. [#48287](https://github.com/ClickHouse/ClickHouse/pull/48287) ([Raúl Marín](https://github.com/Algunenano)). +* Support for CRC32 checksum in HDFS. Fix performance issues. [#48614](https://github.com/ClickHouse/ClickHouse/pull/48614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove remainders of GCC support. [#48671](https://github.com/ClickHouse/ClickHouse/pull/48671) ([Robert Schulze](https://github.com/rschu1ze)). +* Add CI run with new analyzer infrastructure enabled. [#48719](https://github.com/ClickHouse/ClickHouse/pull/48719) ([Dmitry Novik](https://github.com/novikd)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix system.query_views_log for MVs that are pushed from background threads [#46668](https://github.com/ClickHouse/ClickHouse/pull/46668) ([Azat Khuzhin](https://github.com/azat)). +* Fix several `RENAME COLUMN` bugs [#46946](https://github.com/ClickHouse/ClickHouse/pull/46946) ([alesapin](https://github.com/alesapin)). +* Fix minor hiliting issues in clickhouse-format [#47610](https://github.com/ClickHouse/ClickHouse/pull/47610) ([Natasha Murashkina](https://github.com/murfel)). +* Fix a bug in LLVM's libc++ leading to a crash for uploading parts to S3 which size is greater then INT_MAX [#47693](https://github.com/ClickHouse/ClickHouse/pull/47693) ([Azat Khuzhin](https://github.com/azat)). +* Fix overflow in the `sparkbar` function [#48121](https://github.com/ClickHouse/ClickHouse/pull/48121) ([Vladimir C](https://github.com/vdimir)). +* Fix race in S3 [#48190](https://github.com/ClickHouse/ClickHouse/pull/48190) ([Anton Popov](https://github.com/CurtizJ)). +* Disable JIT for aggregate functions due to inconsistent behavior [#48195](https://github.com/ClickHouse/ClickHouse/pull/48195) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix alter formatting (minor) [#48289](https://github.com/ClickHouse/ClickHouse/pull/48289) ([Natasha Murashkina](https://github.com/murfel)). +* Fix cpu usage in RabbitMQ (was worsened in 23.2 after [#44404](https://github.com/ClickHouse/ClickHouse/issues/44404)) [#48311](https://github.com/ClickHouse/ClickHouse/pull/48311) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix crash in EXPLAIN PIPELINE for Merge over Distributed [#48320](https://github.com/ClickHouse/ClickHouse/pull/48320) ([Azat Khuzhin](https://github.com/azat)). +* Fix serializing LowCardinality as Arrow dictionary [#48361](https://github.com/ClickHouse/ClickHouse/pull/48361) ([Kruglov Pavel](https://github.com/Avogar)). +* Reset downloader for cache file segment in TemporaryFileStream [#48386](https://github.com/ClickHouse/ClickHouse/pull/48386) ([Vladimir C](https://github.com/vdimir)). +* Fix possible SYSTEM SYNC REPLICA stuck in case of DROP/REPLACE PARTITION [#48391](https://github.com/ClickHouse/ClickHouse/pull/48391) ([Azat Khuzhin](https://github.com/azat)). +* Fix a startup error when loading a distributed table that depends on a dictionary [#48419](https://github.com/ClickHouse/ClickHouse/pull/48419) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Don't check dependencies when renaming system tables automatically [#48431](https://github.com/ClickHouse/ClickHouse/pull/48431) ([Raúl Marín](https://github.com/Algunenano)). +* Update only affected rows in KeeperMap storage [#48435](https://github.com/ClickHouse/ClickHouse/pull/48435) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix possible segfault in the VFS cache [#48469](https://github.com/ClickHouse/ClickHouse/pull/48469) ([Kseniia Sumarokova](https://github.com/kssenii)). +* `toTimeZone` function throws an error when no constant string is provided [#48471](https://github.com/ClickHouse/ClickHouse/pull/48471) ([Jordi Villar](https://github.com/jrdi)). +* Fix logical error with IPv4 in Protobuf, add support for Date32 [#48486](https://github.com/ClickHouse/ClickHouse/pull/48486) ([Kruglov Pavel](https://github.com/Avogar)). +* "changed" flag in system.settings was calculated incorrectly for settings with multiple values [#48516](https://github.com/ClickHouse/ClickHouse/pull/48516) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix storage `Memory` with enabled compression [#48517](https://github.com/ClickHouse/ClickHouse/pull/48517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bracketed-paste mode messing up password input in the event of client reconnection [#48528](https://github.com/ClickHouse/ClickHouse/pull/48528) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix nested map for keys of IP and UUID types [#48556](https://github.com/ClickHouse/ClickHouse/pull/48556) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix an uncaught exception in case of parallel loader for hashed dictionaries [#48571](https://github.com/ClickHouse/ClickHouse/pull/48571) ([Azat Khuzhin](https://github.com/azat)). +* The `groupArray` aggregate function correctly works for empty result over nullable types [#48593](https://github.com/ClickHouse/ClickHouse/pull/48593) ([lgbo](https://github.com/lgbo-ustc)). +* Fix bug in Keeper when a node is not created with scheme `auth` in ACL sometimes. [#48595](https://github.com/ClickHouse/ClickHouse/pull/48595) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Allow IPv4 comparison operators with UInt [#48611](https://github.com/ClickHouse/ClickHouse/pull/48611) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible error from cache [#48636](https://github.com/ClickHouse/ClickHouse/pull/48636) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Async inserts with empty data will no longer throw exception. [#48663](https://github.com/ClickHouse/ClickHouse/pull/48663) ([Anton Popov](https://github.com/CurtizJ)). +* Fix table dependencies in case of failed RENAME TABLE [#48683](https://github.com/ClickHouse/ClickHouse/pull/48683) ([Azat Khuzhin](https://github.com/azat)). +* If the primary key has duplicate columns (which is only possible for projections), in previous versions it might lead to a bug [#48838](https://github.com/ClickHouse/ClickHouse/pull/48838) ([Amos Bird](https://github.com/amosbird)). +* Fix for a race condition in ZooKeeper when joining send_thread/receive_thread [#48849](https://github.com/ClickHouse/ClickHouse/pull/48849) ([Alexander Gololobov](https://github.com/davenger)). +* Fix unexpected part name error when trying to drop a ignored detached part with zero copy replication [#48862](https://github.com/ClickHouse/ClickHouse/pull/48862) ([Michael Lex](https://github.com/mlex)). +* Fix reading `Date32` Parquet/Arrow column into not a `Date32` column [#48864](https://github.com/ClickHouse/ClickHouse/pull/48864) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix `UNKNOWN_IDENTIFIER` error while selecting from table with row policy and column with dots [#48976](https://github.com/ClickHouse/ClickHouse/pull/48976) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix aggregation by empty nullable strings [#48999](https://github.com/ClickHouse/ClickHouse/pull/48999) ([LiuNeng](https://github.com/liuneng1994)). + + ### ClickHouse release 23.3 LTS, 2023-03-30 #### Upgrade Notes From a6664e6b084b89518e6dc08ff6cbd4e813f18024 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 25 Apr 2023 18:15:09 +0200 Subject: [PATCH 31/34] Add typing, capitalize only the first letter w/o lowering the rest --- tests/ci/report.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 15d8ff9010e..a40eb559792 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -262,17 +262,20 @@ class ReportColorTheme: ColorTheme = Tuple[str, str, str] -def _format_header(header, branch_name, branch_url=None): - result = " ".join([w.capitalize() for w in header.split(" ")]) +def _format_header( + header: str, branch_name: str, branch_url: Optional[str] = None +) -> str: + # Following line does not lower CI->Ci and SQLancer->Sqlancer. It only + # capitalizes the first letter and doesn't touch the rest of the word + result = " ".join([w[0].upper() + w[1:] for w in header.split(" ") if w]) result = result.replace("Clickhouse", "ClickHouse") result = result.replace("clickhouse", "ClickHouse") if "ClickHouse" not in result: - result = "ClickHouse " + result - result += " for " + result = f"ClickHouse {result}" if branch_url: - result += f'{branch_name}' + result = f'{result} for {branch_name}' else: - result += branch_name + result = f"{result} for {branch_name}" return result From 9a7f09bae4e4546ad34f25d2471082bb7099b434 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 25 Apr 2023 18:16:29 +0200 Subject: [PATCH 32/34] Avoid splitting check_name into subdirectories --- tests/ci/performance_comparison_check.py | 1 + tests/ci/upload_result_helper.py | 7 ++++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index d0c84d56496..0da41e0ae82 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -142,6 +142,7 @@ if __name__ == "__main__": .replace("(", "_") .replace(")", "_") .replace(",", "_") + .replace("/", "_") ) docker_image = get_image_with_version(reports_path, IMAGE_NAME) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index b988e240b0e..150af7aff4a 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -59,9 +59,10 @@ def upload_results( additional_files: List[str], check_name: str, ) -> str: - s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace( - " ", "_" - ).replace("(", "_").replace(")", "_").replace(",", "_") + normalized_check_name = check_name.lower() + for r in ((" ", "_"), ("(", "_"), (")", "_"), (",", "_"), ("/", "_")): + normalized_check_name = normalized_check_name.replace(*r) + s3_path_prefix = f"{pr_number}/{commit_sha}/{normalized_check_name}" additional_urls = process_logs( s3_client, additional_files, s3_path_prefix, test_results ) From f65b5264726ee68e464ddd03389581360b168cb7 Mon Sep 17 00:00:00 2001 From: Aram Peres <6775216+aramperes@users.noreply.github.com> Date: Tue, 25 Apr 2023 20:48:46 -0400 Subject: [PATCH 33/34] Fix an unclosed XML tag in documentation --- docs/en/operations/settings/constraints-on-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/constraints-on-settings.md b/docs/en/operations/settings/constraints-on-settings.md index 83ef46053a4..1895a79cd3e 100644 --- a/docs/en/operations/settings/constraints-on-settings.md +++ b/docs/en/operations/settings/constraints-on-settings.md @@ -40,7 +40,7 @@ If the user tries to violate the constraints an exception is thrown and the sett There are supported few types of constraints: `min`, `max`, `readonly` (with alias `const`) and `changeable_in_readonly`. The `min` and `max` constraints specify upper and lower boundaries for a numeric setting and can be used in combination. The `readonly` or `const` constraint specifies that the user cannot change the corresponding setting at all. The `changeable_in_readonly` constraint type allows user to change the setting within `min`/`max` range even if `readonly` setting is set to 1, otherwise settings are not allow to be changed in `readonly=1` mode. Note that `changeable_in_readonly` is supported only if `settings_constraints_replace_previous` is enabled: ``` xml - true + true ``` From 4eb83a94211dd8c8841f72f32681685fe0272b90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=88=98=E6=B5=A9=E6=9E=97?= Date: Tue, 25 Apr 2023 18:22:07 +0800 Subject: [PATCH 34/34] fix: add slash for close tag --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 7a75d7251a9..1aeda624db2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1293,7 +1293,7 @@ - +