From bf37c3a2687e94f4df3c05eb7ba48ee773dfb767 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 22:17:08 +0100 Subject: [PATCH 1/7] Remove obsolete usage of ABORT_ON_LOGICAL_ERROR --- src/Processors/Transforms/DeduplicationTokenTransforms.cpp | 2 +- src/Processors/Transforms/DeduplicationTokenTransforms.h | 2 +- src/Processors/Transforms/buildPushingToViewsChain.cpp | 6 +++--- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp index 841090f029e..79b5bb3be4a 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -142,7 +142,7 @@ size_t TokenInfo::getTotalSize() const return size + parts.size() - 1; } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD void CheckTokenTransform::transform(Chunk & chunk) { auto token_info = chunk.getChunkInfos().get(); diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.h b/src/Processors/Transforms/DeduplicationTokenTransforms.h index 7e5a6101734..f22746f6705 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.h +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.h @@ -93,7 +93,7 @@ namespace DeduplicationToken }; -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD /// use that class only with debug builds in CI for introspection class CheckTokenTransform : public ISimpleTransform { diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 9d914149b4a..46440a2a953 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -378,7 +378,7 @@ std::optional generateViewChain( table_prefers_large_blocks ? settings[Setting::min_insert_block_size_bytes] : 0ULL)); } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Before squashing", out.getInputHeader())); #endif @@ -424,7 +424,7 @@ std::optional generateViewChain( if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); #endif @@ -434,7 +434,7 @@ std::optional generateViewChain( out.addSource(std::move(executing_inner_query)); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); #endif } diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 7aa05ede737..27a2c28f26c 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -364,7 +364,7 @@ void RefreshTask::refreshTask() if (coordination.root_znode.last_attempt_replica == coordination.replica_name) { LOG_ERROR(log, "Znode {} indicates that this replica is running a refresh, but it isn't. Likely a bug.", coordination.path + "/running"); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD abortOnFailedAssertion("Unexpected refresh lock in keeper"); #else coordination.running_znode_exists = false; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index ef0adc15186..a50865994d0 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -209,7 +209,7 @@ struct DeltaLakeMetadataImpl if (!object) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to parse metadata file"); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM object->stringify(oss); LOG_TEST(log, "Metadata: {}", oss.str()); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 0739c03b963..b7b58887a4f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1596,7 +1596,7 @@ void StorageWindowView::writeIntoWindowView( return std::make_shared(stream_header); }); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Afrer tmp table before squashing", stream_header); @@ -1643,7 +1643,7 @@ void StorageWindowView::writeIntoWindowView( lateness_upper_bound); }); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Afrer WatermarkTransform", stream_header); @@ -1668,7 +1668,7 @@ void StorageWindowView::writeIntoWindowView( builder.addSimpleTransform([&](const Block & header_) { return std::make_shared(header_, convert_actions); }); } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Before out", stream_header); From 92a79b750e98d177d9b9327341001a0d72dd7f81 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 27 Nov 2024 23:50:23 +0100 Subject: [PATCH 2/7] Fix --- src/Storages/MergeTree/FutureMergedMutatedPart.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp index 959a0e50bca..83919e90569 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp @@ -88,7 +88,8 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_, Merg void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const IReservation * reservation) { - path = storage.getFullPathOnDisk(reservation->getDisk()) + name + "/"; + path = fs::path(storage.getFullPathOnDisk(reservation->getDisk())) / name; + path += "/"; } } From 27ffde3f3ed757cdcb8020eec53c0b6f1a0afeba Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 18:45:53 +0100 Subject: [PATCH 3/7] Init --- src/Interpreters/Set.cpp | 2 +- .../0_stateless/03278_dateTime_in_dateTime_bug.reference | 2 ++ .../queries/0_stateless/03278_dateTime_in_dateTime_bug.sql | 7 +++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference create mode 100644 tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index c6f0455652a..3a394a5dd02 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(result->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference new file mode 100644 index 00000000000..b451df8c07a --- /dev/null +++ b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference @@ -0,0 +1,2 @@ +1 2001-01-11 01:11:21.100 2001-01-11 01:11:21.100 +1 2001-01-11 01:11:21.100 2001-01-11 01:11:21.100 diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql new file mode 100644 index 00000000000..8b8bf3b56da --- /dev/null +++ b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql @@ -0,0 +1,7 @@ +CREATE TABLE datetime64_issue (id int, dt DateTime64(3), dtn Nullable(DateTime64(3))) ENGINE = MergeTree() ORDER BY id PRIMARY KEY id; + +INSERT INTO datetime64_issue(id, dt, dtn) VALUES (1, toDateTime64('2001-01-11 01:11:21.100', 3), toDateTime64('2001-01-11 01:11:21.100', 3)); + +SELECT * FROM datetime64_issue WHERE dt in (toDateTime64('2001-01-11 01:11:21.100', 3)); + +SELECT * FROM datetime64_issue WHERE dtn in (toDateTime64('2001-01-11 01:11:21.100', 3)); From 98a358db10bc1401d8c4199138c42959061a0618 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 18:49:09 +0100 Subject: [PATCH 4/7] Fix tests --- ...g.reference => 03278_dateTime64_in_dateTime64_bug.reference} | 0 ..._dateTime_bug.sql => 03278_dateTime64_in_dateTime64_bug.sql} | 2 ++ 2 files changed, 2 insertions(+) rename tests/queries/0_stateless/{03278_dateTime_in_dateTime_bug.reference => 03278_dateTime64_in_dateTime64_bug.reference} (100%) rename tests/queries/0_stateless/{03278_dateTime_in_dateTime_bug.sql => 03278_dateTime64_in_dateTime64_bug.sql} (93%) diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.reference similarity index 100% rename from tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference rename to tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.reference diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql similarity index 93% rename from tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql rename to tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql index 8b8bf3b56da..5d6bd825faf 100644 --- a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql +++ b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql @@ -5,3 +5,5 @@ INSERT INTO datetime64_issue(id, dt, dtn) VALUES (1, toDateTime64('2001-01-11 01 SELECT * FROM datetime64_issue WHERE dt in (toDateTime64('2001-01-11 01:11:21.100', 3)); SELECT * FROM datetime64_issue WHERE dtn in (toDateTime64('2001-01-11 01:11:21.100', 3)); + +DROP TABLE datetime64_issue; From ca5faf14ca3e2e6546bb7c7a7f106ec535003a2e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 20:18:41 +0100 Subject: [PATCH 5/7] Update Set.cpp --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 3a394a5dd02..0b1b8d3c2b5 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(result->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && isDateTime64(result->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } From bc3bc8913e3e2b9d24f817c0d12fe45b78d653f4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 20:59:07 +0100 Subject: [PATCH 6/7] Add nullable check for data types --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 0b1b8d3c2b5..60328123aa0 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType()) && isDateTime64(result->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(removeNullable(result)->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } From 88a3dbe716796e5575be5b7a3c579c8e8d32f479 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Nov 2024 18:15:14 +0100 Subject: [PATCH 7/7] Fix inaccurate MemoryTracking metric in case of allocation failures MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit MemoryTracking metric takes into account memory allocation even in case of this allocation will fail with MEMORY_LIMIT_EXCEEDED, which is not good, which eventually will lead to `amount` and `MemoryTracking` mismatch, I found one server with **43x difference**.
```sql SELECT event_time_microseconds, message FROM system.text_log WHERE (event_date = (today() - 1)) AND (logger_name = 'MemoryTracker') AND (message LIKE '%total%') ORDER BY 1 DESC LIMIT 1 Query id: 64d60852-fa14-4ed1-adb1-d4bbd6159475 ┌────event_time_microseconds─┬─message───────────────────────────────────┐ 1. │ 2024-11-27 05:09:48.157608 │ Current memory usage (total): 471.00 GiB. │ └────────────────────────────┴───────────────────────────────────────────┘ ``` ```sql SELECT metric, formatReadableSize(value) FROM system.metrics WHERE (metric ILIKE '%mem%') OR (metric ILIKE '%jemalloc%') ORDER BY value ASC Query id: af7908a8-956a-4684-b7c5-b2e0c6fa06f4 ┌─metric────────────────────────┬─formatReadableSize(value)─┐ 1. │ MergesMutationsMemoryTracking │ 0.00 B │ 2. │ MemoryTracking │ 20.37 TiB │ └───────────────────────────────┴───────────────────────────┘ ```
Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index f4af019605e..829d5dfcd79 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -256,10 +256,6 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed); - auto metric_loaded = metric.load(std::memory_order_relaxed); - if (metric_loaded != CurrentMetrics::end() && size) - CurrentMetrics::add(metric_loaded, size); - Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed); Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed); @@ -371,6 +367,10 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } + auto metric_loaded = metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end() && size) + CurrentMetrics::add(metric_loaded, size); + if (peak_updated && allocation_traced) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);