From fb8452aa4909058962f2ddc055fe3e029fee44f3 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 13 Jan 2023 11:26:41 +0800 Subject: [PATCH 01/15] Add last_exception_time to replication_queue --- src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp | 1 + src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 1 + src/Storages/System/StorageSystemReplicationQueue.cpp | 4 +++- 4 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 36a5957bf1d..de416c3388a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -91,6 +91,7 @@ bool ReplicatedMergeMutateTaskBase::executeStep() auto & log_entry = selected_entry->log_entry; log_entry->exception = saved_exception; + log_entry->last_exception_time = time(nullptr); if (log_entry->type == ReplicatedMergeTreeLogEntryData::MUTATE_PART) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 84115cb2b8a..476d4797eef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -157,6 +157,7 @@ struct ReplicatedMergeTreeLogEntryData /// Access under queue_mutex, see ReplicatedMergeTreeQueue. size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. + time_t last_exception_time = 0; /// The time at which the last exception occurred. time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. size_t num_postponed = 0; /// The number of times the action was postponed. String postpone_reason; /// The reason why the action was postponed, if it was postponed. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index bd75d76109a..628b3048cfc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1691,6 +1691,7 @@ bool ReplicatedMergeTreeQueue::processEntry( { std::lock_guard lock(state_mutex); entry->exception = saved_exception; + entry->last_exception_time = time(nullptr); return false; } diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 8acd192eac4..fd9f874052d 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -38,6 +38,7 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() { "is_currently_executing", std::make_shared() }, { "num_tries", std::make_shared() }, { "last_exception", std::make_shared() }, + { "last_exception_time", std::make_shared() }, { "last_attempt_time", std::make_shared() }, { "num_postponed", std::make_shared() }, { "postpone_reason", std::make_shared() }, @@ -141,7 +142,8 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, Conte res_columns[col_num++]->insert(entry.detach); res_columns[col_num++]->insert(entry.currently_executing); res_columns[col_num++]->insert(entry.num_tries); - res_columns[col_num++]->insert(entry.exception ? getExceptionMessage(entry.exception, false) : ""); + res_columns[col_num++]->insert(entry.exception ? getExceptionMessage(entry.exception, true) : ""); + res_columns[col_num++]->insert(UInt64(entry.last_exception_time)); res_columns[col_num++]->insert(UInt64(entry.last_attempt_time)); res_columns[col_num++]->insert(entry.num_postponed); res_columns[col_num++]->insert(entry.postpone_reason); From be8ceeb4bb857028c8bba7076e65dc33dbdd5fce Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 13 Jan 2023 18:42:52 +0800 Subject: [PATCH 02/15] Save stack trace in part_log --- src/Common/Exception.cpp | 6 ++++++ src/Common/Exception.h | 2 ++ src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 11 files changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 35231354651..b83ddf09c7f 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -546,6 +546,12 @@ ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_ return ExecutionStatus(getCurrentExceptionCode(), msg); } +ExecutionStatus ExecutionStatus::fromCurrentExceptionWithStackTrace(const std::string & start_of_message) +{ + String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(true, true); + return ExecutionStatus(getCurrentExceptionCode(), msg); +} + ExecutionStatus ExecutionStatus::fromText(const std::string & data) { ExecutionStatus status; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 0f459a887f1..58da20359ad 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -204,6 +204,8 @@ struct ExecutionStatus : code(return_code), message(exception_message) {} static ExecutionStatus fromCurrentException(const std::string & start_of_message = ""); + + static ExecutionStatus fromCurrentExceptionWithStackTrace(const std::string & start_of_message = ""); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 97226825629..ca3e25ef077 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index cc5e87956a1..c38fefd9d32 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b1384dc799f..e1e1312cd6c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index b432841d5b0..65b3f8b15a8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index b1714076a46..28452aaf856 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 36a5957bf1d..09fda943600 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException()); + part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException()); + part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 14ae620176a..9ebed2467ad 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c8dc53f5036..ec17c765e05 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d4fd329ef90..04fbdad22d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 38799eb85238003b15f6a0e397a3e0650f0aa9de Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 16:35:28 +0800 Subject: [PATCH 03/15] Merge two methods as one --- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 9 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index ca3e25ef077..16efde3e4cc 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index c38fefd9d32..9f24839f1e1 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e1e1312cd6c..6184669b223 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 65b3f8b15a8..8b5c6bf013d 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 28452aaf856..e801c0a8879 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 09fda943600..7225a7c41d3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9ebed2467ad..c4a8f837ec9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ec17c765e05..5536e8af793 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 04fbdad22d1..c0d0780a550 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From f867c4c2061858bc2a3e58855acaa90b93e753dc Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 16:43:10 +0800 Subject: [PATCH 04/15] Add an overriden fromCurrentException to simply the code --- src/Common/Exception.cpp | 9 ++++----- src/Common/Exception.h | 5 ++--- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 11 files changed, 22 insertions(+), 24 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index b83ddf09c7f..10fc3dcbfae 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -540,15 +540,14 @@ bool ExecutionStatus::tryDeserializeText(const std::string & data) return true; } -ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message) +ExecutionStatus ExecutionStatus::fromCurrentException(bool with_stacktrace) { - String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(false, true); - return ExecutionStatus(getCurrentExceptionCode(), msg); + return ExecutionStatus(getCurrentExceptionCode(), getCurrentExceptionMessage(with_stacktrace, true)); } -ExecutionStatus ExecutionStatus::fromCurrentExceptionWithStackTrace(const std::string & start_of_message) +ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message, bool with_stacktrace) { - String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(true, true); + String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(with_stacktrace, true); return ExecutionStatus(getCurrentExceptionCode(), msg); } diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 58da20359ad..96668c68cd3 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -203,9 +203,8 @@ struct ExecutionStatus explicit ExecutionStatus(int return_code, const std::string & exception_message = "") : code(return_code), message(exception_message) {} - static ExecutionStatus fromCurrentException(const std::string & start_of_message = ""); - - static ExecutionStatus fromCurrentExceptionWithStackTrace(const std::string & start_of_message = ""); + static ExecutionStatus fromCurrentException(bool with_stacktrace = false); + static ExecutionStatus fromCurrentException(const std::string & start_of_message, bool with_stacktrace = false); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 16efde3e4cc..b3934098d0d 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 9f24839f1e1..e59f7fda44c 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6184669b223..5addb24b0ca 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 8b5c6bf013d..7bf3a95876b 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index e801c0a8879..1ab1e2a82ec 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 7225a7c41d3..bbe83814497 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException("", true)); + part_log_writer(ExecutionStatus::fromCurrentException(true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException("", true)); + part_log_writer(ExecutionStatus::fromCurrentException(true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c4a8f837ec9..03b2dadde5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5536e8af793..20b0c16edb7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c0d0780a550..82ff0a0bd83 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 03109f942d424e8d0d67bde5484c45b2ed569dc8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Feb 2023 01:02:12 +0100 Subject: [PATCH 05/15] better context for stress tests failures --- docker/test/stress/run.sh | 44 +++++++++++++------ .../util/process_functional_tests_result.py | 5 ++- tests/ci/clickhouse_helper.py | 6 ++- tests/ci/report.py | 3 +- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index aa242bfa98d..8347f67fed6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -13,15 +13,28 @@ sysctl kernel.core_pattern='core.%e.%p-%P' OK="\tOK\t\\N\t" FAIL="\tFAIL\t\\N\t" + +FAILURE_CONTEXT_LINES=50 +FAILURE_CONTEXT_MAX_LINE_WIDTH=400 + function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. - clickhouse local -S 's String' --input-format=LineAsString -q "select * from table format CustomSeparated settings format_custom_row_after_delimiter='\\\\\\\\n'" + # Also limit lines width just in case (too long lines are not really useful usually) + clickhouse local -S 's String' --input-format=LineAsString -q "select substr(s, 1, $FAILURE_CONTEXT_MAX_LINE_WIDTH) + from table format CustomSeparated settings format_custom_row_after_delimiter='\\\\\\\\n'" } - function head_escaped() { - head -50 $1 | escaped + head -n $FAILURE_CONTEXT_LINES $1 | escaped +} +function unts() +{ + grep -Po "[0-9][0-9]:[0-9][0-9] \K.*" +} +function trim_server_logs() +{ + head -n $FAILURE_CONTEXT_LINES "/test_output/$1" | grep -Eo " \[ [0-9]+ \] \{.*" | escaped } function install_packages() @@ -167,7 +180,7 @@ function start() then echo "Cannot start clickhouse-server" rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt ||: - echo -e "Cannot start clickhouse-server$FAIL$(head_escaped /test_output/application_errors.txt)" >> /test_output/test_results.tsv + echo -e "Cannot start clickhouse-server$FAIL$(trim_server_logs application_errors.txt)" >> /test_output/test_results.tsv cat /var/log/clickhouse-server/stdout.log tail -n100 /var/log/clickhouse-server/stderr.log tail -n100000 /var/log/clickhouse-server/clickhouse-server.log | rg -F -v -e ' RaftInstance:' -e ' RaftInstance' | tail -n100 @@ -389,7 +402,7 @@ start # NOTE Hung check is implemented in docker/tests/stress/stress rg -Fa "No queries hung" /test_output/test_results.tsv | grep -Fa "OK" \ - || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log)" + || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log | unts)" stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.stress.log @@ -402,7 +415,7 @@ start clickhouse-client --query "SELECT 'Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \ || (rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt \ - && echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(head_escaped /test_output/application_errors.txt)" \ + && echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(trim_server_logs application_errors.txt)" \ >> /test_output/test_results.tsv) stop @@ -435,7 +448,7 @@ rg -Fa "Code: 49. DB::Exception: " /var/log/clickhouse-server/clickhouse-server* # No such key errors rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log > /test_output/no_such_key_errors.txt \ - && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(head_escaped /test_output/no_such_key_errors.txt)" >> /test_output/test_results.tsv \ + && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \ || echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv # Remove file no_such_key_errors.txt if it's empty @@ -448,7 +461,7 @@ rg -Fa "########################################" /var/log/clickhouse-server/cli # It also checks for crash without stacktrace (printed by watchdog) rg -Fa " " /var/log/clickhouse-server/clickhouse-server*.log > /test_output/fatal_messages.txt \ - && echo -e "Fatal message in clickhouse-server.log (see fatal_messages.txt)$FAIL$(head_escaped /test_output/fatal_messages.txt)" >> /test_output/test_results.tsv \ + && echo -e "Fatal message in clickhouse-server.log (see fatal_messages.txt)$FAIL$(trim_server_logs fatal_messages.txt)" >> /test_output/test_results.tsv \ || echo -e "No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv # Remove file fatal_messages.txt if it's empty @@ -457,8 +470,13 @@ rg -Fa " " /var/log/clickhouse-server/clickhouse-server*.log > /test_out rg -Fa "########################################" /test_output/* > /dev/null \ && echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv +function get_gdb_log_context() +{ + rg -A50 -Fa " received signal " /test_output/gdb.log | head_escaped +} + rg -Fa " received signal " /test_output/gdb.log > /dev/null \ - && echo -e "Found signal in gdb.log$FAIL$(rg -A50 -Fa " received signal " /test_output/gdb.log | escaped)" >> /test_output/test_results.tsv + && echo -e "Found signal in gdb.log$FAIL$(get_gdb_log_context)" >> /test_output/test_results.tsv if [ "$DISABLE_BC_CHECK" -ne "1" ]; then echo -e "Backward compatibility check\n" @@ -579,7 +597,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then start 500 clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \ || (rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt \ - && echo -e "Backward compatibility check: Server failed to start$FAIL$(head_escaped /test_output/bc_check_application_errors.txt)" >> /test_output/test_results.tsv) + && echo -e "Backward compatibility check: Server failed to start$FAIL$(trim_server_logs bc_check_application_errors.txt)" >> /test_output/test_results.tsv) clickhouse-client --query="SELECT 'Server version: ', version()" @@ -634,7 +652,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then -e "Session expired" \ -e "TOO_MANY_PARTS" \ /var/log/clickhouse-server/clickhouse-server.backward.dirty.log | rg -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(head_escaped /test_output/bc_check_error_messages.txt)" \ + && echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(trim_server_logs bc_check_error_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No Error messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv @@ -657,7 +675,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then # Logical errors echo "Check for Logical errors in server log:" rg -Fa -A20 "Code: 49. DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \ - && echo -e "Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)$FAIL$(head_escaped /test_output/bc_check_logical_errors.txt)" \ + && echo -e "Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)$FAIL$(trim_server_logs bc_check_logical_errors.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No logical errors$OK" >> /test_output/test_results.tsv @@ -672,7 +690,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" rg -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e "Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)$FAIL$(head_escaped /test_output/bc_check_fatal_messages.txt)" \ + && echo -e "Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)$FAIL$(trim_server_logs bc_check_fatal_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 28f3e211157..257d1587750 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -85,8 +85,11 @@ def process_test_log(log_path): if DATABASE_SIGN in line: test_end = True + # Python does not support TSV, so we have to escape '\t' and '\n' manually + # and hope that complex escape sequences will not break anything test_results = [ - (test[0], test[1], test[2], "".join(test[3])[:4096]) for test in test_results + (test[0], test[1], test[2], + "".join(test[3])[:4096].replace('\t', '\\t').replace('\n', '\\n')) for test in test_results ] return ( diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 654cd6869dc..c873f129468 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -183,7 +183,11 @@ def prepare_tests_results_for_clickhouse( current_row["test_duration_ms"] = int(test_time * 1000) current_row["test_name"] = test_name current_row["test_status"] = test_status - current_row["test_context_raw"] = test_result.raw_logs or "" + if test_result.raw_logs: + # Protect from too big blobs that contain garbage + current_row["test_context_raw"] = test_result.raw_logs[:32 * 1024] + else: + current_row["test_context_raw"] = "" result.append(current_row) return result diff --git a/tests/ci/report.py b/tests/ci/report.py index d33ed4a9d91..0096294f66a 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -224,7 +224,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # The value can be emtpy, but when it's not, # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: - result.set_raw_logs(line[3]) + # Python does not support TSV, so we unescape manually + result.set_raw_logs(line[3]).replace('\\t', '\t').replace('\\n', '\n') else: result.set_log_files(line[3]) From ddbc8ceee8994938c1831a36519db48e8d34c5a5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Feb 2023 00:59:49 +0000 Subject: [PATCH 06/15] Automatic style fix --- docker/test/util/process_functional_tests_result.py | 9 +++++++-- tests/ci/clickhouse_helper.py | 2 +- tests/ci/report.py | 4 +++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 257d1587750..da58db8e45d 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -88,8 +88,13 @@ def process_test_log(log_path): # Python does not support TSV, so we have to escape '\t' and '\n' manually # and hope that complex escape sequences will not break anything test_results = [ - (test[0], test[1], test[2], - "".join(test[3])[:4096].replace('\t', '\\t').replace('\n', '\\n')) for test in test_results + ( + test[0], + test[1], + test[2], + "".join(test[3])[:4096].replace("\t", "\\t").replace("\n", "\\n"), + ) + for test in test_results ] return ( diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index c873f129468..d60a9e6afd1 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -185,7 +185,7 @@ def prepare_tests_results_for_clickhouse( current_row["test_status"] = test_status if test_result.raw_logs: # Protect from too big blobs that contain garbage - current_row["test_context_raw"] = test_result.raw_logs[:32 * 1024] + current_row["test_context_raw"] = test_result.raw_logs[: 32 * 1024] else: current_row["test_context_raw"] = "" result.append(current_row) diff --git a/tests/ci/report.py b/tests/ci/report.py index 0096294f66a..e1601cab805 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,7 +225,9 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3]).replace('\\t', '\t').replace('\\n', '\n') + result.set_raw_logs(line[3]).replace("\\t", "\t").replace( + "\\n", "\n" + ) else: result.set_log_files(line[3]) From 8513e5a721ad0178a0a798ce516fbe062ad6287d Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 08:48:13 +0000 Subject: [PATCH 07/15] Fix tupleElement with Null arguments --- src/Functions/tupleElement.cpp | 7 +++---- .../02541_tuple_element_with_null.reference | 2 ++ .../02541_tuple_element_with_null.sql | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02541_tuple_element_with_null.reference create mode 100644 tests/queries/0_stateless/02541_tuple_element_with_null.sql diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 829262de30a..879b6feed40 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -56,10 +56,9 @@ public: return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override - { - return {1}; - } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + bool useDefaultImplementationForNulls() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/tests/queries/0_stateless/02541_tuple_element_with_null.reference b/tests/queries/0_stateless/02541_tuple_element_with_null.reference new file mode 100644 index 00000000000..7afa6248576 --- /dev/null +++ b/tests/queries/0_stateless/02541_tuple_element_with_null.reference @@ -0,0 +1,2 @@ +1 1 2 2 +\N \N 3 3 diff --git a/tests/queries/0_stateless/02541_tuple_element_with_null.sql b/tests/queries/0_stateless/02541_tuple_element_with_null.sql new file mode 100644 index 00000000000..d2062b60d49 --- /dev/null +++ b/tests/queries/0_stateless/02541_tuple_element_with_null.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test_tuple_element; +CREATE TABLE test_tuple_element +( + tuple Tuple(k1 Nullable(UInt64), k2 UInt64) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192; + +INSERT INTO test_tuple_element VALUES (tuple(1,2)), (tuple(NULL, 3)); + +SELECT + tupleElement(tuple, 'k1', 0) fine_k1_with_0, + tupleElement(tuple, 'k1', NULL) k1_with_null, + tupleElement(tuple, 'k2', 0) k2_with_0, + tupleElement(tuple, 'k2', NULL) k2_with_null +FROM test_tuple_element; + +DROP TABLE test_tuple_element; From f7665e136aa19ed3f7fc65d140d38c48754d8afc Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 09:34:42 +0000 Subject: [PATCH 08/15] update test update test --- .../queries/0_stateless/01710_minmax_count_projection.reference | 1 - tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index bbcec98fb74..4dd46ccd22c 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -16,7 +16,6 @@ 1 1 1 -2021-10-25 10:00:00 3 2021-10-27 10:00:00 3 \N 2021-10-27 10:00:00 3 0 2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index f01e5915393..1967d9fe53c 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -58,7 +58,7 @@ select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(d select count() from d group by toDate(dt); -- fuzz crash -SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; +SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- { serverError 43 } -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; From f1b4ee059c2dc46ceb8508d2a2a3b8840aa79fd7 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 10:34:47 +0000 Subject: [PATCH 09/15] update test --- .../queries/0_stateless/01710_minmax_count_projection.reference | 1 + tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 4dd46ccd22c..bbcec98fb74 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -16,6 +16,7 @@ 1 1 1 +2021-10-25 10:00:00 3 2021-10-27 10:00:00 3 \N 2021-10-27 10:00:00 3 0 2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 1967d9fe53c..f7645414187 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -58,7 +58,7 @@ select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(d select count() from d group by toDate(dt); -- fuzz crash -SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- { serverError 43 } +SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, 'xxxx', NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; From 5e64cda339bfaf650d353af688ff0a91f1192799 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Feb 2023 15:28:01 +0300 Subject: [PATCH 10/15] Update tests/ci/report.py Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com> --- tests/ci/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index e1601cab805..5422c463fb8 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,8 +225,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3]).replace("\\t", "\t").replace( - "\\n", "\n" + result.set_raw_logs(line[3].replace("\\t", "\t").replace( + "\\n", "\n") ) else: result.set_log_files(line[3]) From 892a3ff316e44061e2567998d1cebb3ba14f4522 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Feb 2023 12:40:50 +0000 Subject: [PATCH 11/15] Automatic style fix --- tests/ci/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 5422c463fb8..947fb33d905 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,8 +225,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3].replace("\\t", "\t").replace( - "\\n", "\n") + result.set_raw_logs( + line[3].replace("\\t", "\t").replace("\\n", "\n") ) else: result.set_log_files(line[3]) From ad09721238e48d064d8ffda13193d41e42f0bb21 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 11:59:17 +0800 Subject: [PATCH 12/15] Fix test case --- .../queries/0_stateless/02117_show_create_table_system.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 9e065c455e9..f7d0e6f7b5c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -920,6 +920,7 @@ CREATE TABLE system.replication_queue `is_currently_executing` UInt8, `num_tries` UInt32, `last_exception` String, + `last_exception_time` DateTime, `last_attempt_time` DateTime, `num_postponed` UInt32, `postpone_reason` String, From d38adfab307eba3ce57cdb68fd94d6c49914d365 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 14:41:27 +0800 Subject: [PATCH 13/15] Merge two overridden functions as one --- src/Common/Exception.cpp | 5 ----- src/Common/Exception.h | 3 +-- src/Interpreters/DDLWorker.cpp | 2 +- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 12 files changed, 18 insertions(+), 24 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index fa118c2fcf5..7e7ccfa4877 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -559,11 +559,6 @@ bool ExecutionStatus::tryDeserializeText(const std::string & data) return true; } -ExecutionStatus ExecutionStatus::fromCurrentException(bool with_stacktrace) -{ - return ExecutionStatus(getCurrentExceptionCode(), getCurrentExceptionMessage(with_stacktrace, true)); -} - ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message, bool with_stacktrace) { String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(with_stacktrace, true); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 73524a0202c..8e50c1114f4 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -242,8 +242,7 @@ struct ExecutionStatus explicit ExecutionStatus(int return_code, const std::string & exception_message = "") : code(return_code), message(exception_message) {} - static ExecutionStatus fromCurrentException(bool with_stacktrace = false); - static ExecutionStatus fromCurrentException(const std::string & start_of_message, bool with_stacktrace = false); + static ExecutionStatus fromCurrentException(const std::string & start_of_message = "", bool with_stacktrace = false); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0f91212e6a9..c8673bed842 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -632,7 +632,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (task.is_initial_query) throw; tryLogCurrentException(log, "An error occurred before execution of DDL task: "); - task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution"); + task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution", false); } if (task.execution_status.code != 0) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index be3074c40d2..2d2013bd648 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -335,7 +335,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index e59f7fda44c..9f24839f1e1 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83203542cd3..477d75474b3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7469,7 +7469,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index e785a38d501..b83c058f7fd 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -215,7 +215,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index adc5b0c2c22..76ba921b705 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -103,7 +103,7 @@ bool MutatePlainMergeTreeTask::executeStep() PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message.text); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 376c3e46a22..a4a4de6d8d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException(true)); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException(true)); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index fb8affa5ee7..ee192966758 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(true)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1ee9f0e9ae0..9d6ddc2c79a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1849,7 +1849,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -1932,7 +1932,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0641d3ad8f2..ff3c039be1d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2337,7 +2337,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4190,7 +4190,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -4300,7 +4300,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -7041,7 +7041,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7270,7 +7270,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 7831cf85db5fccf18e5817efb5120124e60d53d2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 15:36:12 +0800 Subject: [PATCH 14/15] Revert an extra change --- src/Interpreters/DDLWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index c8673bed842..0f91212e6a9 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -632,7 +632,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (task.is_initial_query) throw; tryLogCurrentException(log, "An error occurred before execution of DDL task: "); - task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution", false); + task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution"); } if (task.execution_status.code != 0) From 8464357bca943ff45c57ee89414dc9d138f37bb2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Feb 2023 13:57:47 -0500 Subject: [PATCH 15/15] fix heading level --- .../sql-reference/functions/comparison-functions.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/comparison-functions.md b/docs/en/sql-reference/functions/comparison-functions.md index ab19399bd4e..586c0dc54e6 100644 --- a/docs/en/sql-reference/functions/comparison-functions.md +++ b/docs/en/sql-reference/functions/comparison-functions.md @@ -21,14 +21,14 @@ For example, you can’t compare a date with a string. You have to use a functio Strings are compared by bytes. A shorter string is smaller than all strings that start with it and that contain at least one more character. -## equals, a = b and a == b operator +### equals, a `=` b and a `==` b operator -## notEquals, a != b and a \<\> b operator +### notEquals, a `!=` b and a `<>` b operator -## less, \< operator +### less, `<` operator -## greater, \> operator +### greater, `>` operator -## lessOrEquals, \<= operator +### lessOrEquals, `<=` operator -## greaterOrEquals, \>= operator +### greaterOrEquals, `>=` operator