Merge branch 'master' into pr-fix-setting-history

This commit is contained in:
Igor Nikonov 2024-12-02 19:29:41 +01:00 committed by GitHub
commit 8c4b73ed64
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 28 additions and 16 deletions

View File

@ -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 = 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); 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_hard_limit = hard_limit.load(std::memory_order_relaxed);
Int64 current_profiler_limit = profiler_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) if (peak_updated && allocation_traced)
{ {
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);

View File

@ -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 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(removeNullable(result)->getDataType()))
{ {
processDateTime64Column(column_to_cast, result, null_map_holder, null_map); processDateTime64Column(column_to_cast, result, null_map_holder, null_map);
} }

View File

@ -142,7 +142,7 @@ size_t TokenInfo::getTotalSize() const
return size + parts.size() - 1; return size + parts.size() - 1;
} }
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
void CheckTokenTransform::transform(Chunk & chunk) void CheckTokenTransform::transform(Chunk & chunk)
{ {
auto token_info = chunk.getChunkInfos().get<TokenInfo>(); auto token_info = chunk.getChunkInfos().get<TokenInfo>();

View File

@ -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 /// use that class only with debug builds in CI for introspection
class CheckTokenTransform : public ISimpleTransform class CheckTokenTransform : public ISimpleTransform
{ {

View File

@ -381,7 +381,7 @@ std::optional<Chain> generateViewChain(
table_prefers_large_blocks ? settings[Setting::min_insert_block_size_bytes] : 0ULL)); 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<DeduplicationToken::CheckTokenTransform>("Before squashing", out.getInputHeader())); out.addSource(std::make_shared<DeduplicationToken::CheckTokenTransform>("Before squashing", out.getInputHeader()));
#endif #endif
@ -427,7 +427,7 @@ std::optional<Chain> generateViewChain(
if (type == QueryViewsLogElement::ViewType::MATERIALIZED) if (type == QueryViewsLogElement::ViewType::MATERIALIZED)
{ {
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
out.addSource(std::make_shared<DeduplicationToken::CheckTokenTransform>("Right after Inner query", out.getInputHeader())); out.addSource(std::make_shared<DeduplicationToken::CheckTokenTransform>("Right after Inner query", out.getInputHeader()));
#endif #endif
@ -450,7 +450,7 @@ std::optional<Chain> generateViewChain(
} }
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
out.addSource(std::make_shared<DeduplicationToken::CheckTokenTransform>("Right before Inner query", out.getInputHeader())); out.addSource(std::make_shared<DeduplicationToken::CheckTokenTransform>("Right before Inner query", out.getInputHeader()));
#endif #endif
} }

View File

@ -364,7 +364,7 @@ void RefreshTask::refreshTask()
if (coordination.root_znode.last_attempt_replica == coordination.replica_name) 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"); 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"); abortOnFailedAssertion("Unexpected refresh lock in keeper");
#else #else
coordination.running_znode_exists = false; coordination.running_znode_exists = false;

View File

@ -88,7 +88,8 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_, Merg
void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const IReservation * reservation) void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const IReservation * reservation)
{ {
path = storage.getFullPathOnDisk(reservation->getDisk()) + name + "/"; path = fs::path(storage.getFullPathOnDisk(reservation->getDisk())) / name;
path += "/";
} }
} }

View File

@ -209,7 +209,7 @@ struct DeltaLakeMetadataImpl
if (!object) if (!object)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to parse metadata file"); 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 std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
object->stringify(oss); object->stringify(oss);
LOG_TEST(log, "Metadata: {}", oss.str()); LOG_TEST(log, "Metadata: {}", oss.str());

View File

@ -1596,7 +1596,7 @@ void StorageWindowView::writeIntoWindowView(
return std::make_shared<DeduplicationToken::SetViewBlockNumberTransform>(stream_header); return std::make_shared<DeduplicationToken::SetViewBlockNumberTransform>(stream_header);
}); });
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
builder.addSimpleTransform([&](const Block & stream_header) builder.addSimpleTransform([&](const Block & stream_header)
{ {
return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Afrer tmp table before squashing", stream_header); return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Afrer tmp table before squashing", stream_header);
@ -1643,7 +1643,7 @@ void StorageWindowView::writeIntoWindowView(
lateness_upper_bound); lateness_upper_bound);
}); });
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
builder.addSimpleTransform([&](const Block & stream_header) builder.addSimpleTransform([&](const Block & stream_header)
{ {
return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Afrer WatermarkTransform", stream_header); return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Afrer WatermarkTransform", stream_header);
@ -1668,7 +1668,7 @@ void StorageWindowView::writeIntoWindowView(
builder.addSimpleTransform([&](const Block & header_) { return std::make_shared<ExpressionTransform>(header_, convert_actions); }); builder.addSimpleTransform([&](const Block & header_) { return std::make_shared<ExpressionTransform>(header_, convert_actions); });
} }
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef DEBUG_OR_SANITIZER_BUILD
builder.addSimpleTransform([&](const Block & stream_header) builder.addSimpleTransform([&](const Block & stream_header)
{ {
return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Before out", stream_header); return std::make_shared<DeduplicationToken::CheckTokenTransform>("StorageWindowView: Before out", stream_header);

View File

@ -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

View File

@ -0,0 +1,9 @@
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));
DROP TABLE datetime64_issue;