add system log for introspection

This commit is contained in:
Alexander Tokmakov 2022-01-14 17:03:00 +03:00
parent 158fbaaa29
commit e1a41fc694
19 changed files with 342 additions and 44 deletions

View File

@ -159,7 +159,7 @@ tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
# Replace the engine with Ordinary to avoid extra symlinks stuff in artifacts.
# (so that clickhouse-local --path can read it w/o extra care).
sed -i -e "s/ATTACH DATABASE _ UUID '[^']*'/ATTACH DATABASE system/" -e "s/Atomic/Ordinary/" /var/lib/clickhouse/metadata/system.sql
for table in text_log query_log zookeeper_log trace_log; do
for table in text_log query_log zookeeper_log trace_log transactions_info_log; do
sed -i "s/ATTACH TABLE _ UUID '[^']*'/ATTACH TABLE $table/" /var/lib/clickhouse/metadata/system/${table}.sql
tar -chf /test_output/${table}_dump.tar /var/lib/clickhouse/metadata/system.sql /var/lib/clickhouse/metadata/system/${table}.sql /var/lib/clickhouse/data/system/${table} ||:
done

View File

@ -2383,6 +2383,17 @@ std::shared_ptr<ZooKeeperLog> Context::getZooKeeperLog() const
}
std::shared_ptr<TransactionsInfoLog> Context::getTransactionsInfoLog() const
{
auto lock = getLock();
if (!shared->system_logs)
return {};
return shared->system_logs->transactions_info_log;
}
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
{
auto lock = getLock();

View File

@ -80,6 +80,7 @@ class AsynchronousMetricLog;
class OpenTelemetrySpanLog;
class ZooKeeperLog;
class SessionLog;
class TransactionsInfoLog;
struct MergeTreeSettings;
class StorageS3Settings;
class IDatabase;
@ -790,6 +791,7 @@ public:
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog() const;
std::shared_ptr<ZooKeeperLog> getZooKeeperLog() const;
std::shared_ptr<SessionLog> getSessionLog() const;
std::shared_ptr<TransactionsInfoLog> getTransactionsInfoLog() const;
/// Returns an object used to log operations with parts if it possible.
/// Provide table name to make required checks.

View File

@ -29,6 +29,7 @@
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Interpreters/ZooKeeperLog.h>
#include <Interpreters/TransactionsInfoLog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Access/ContextAccess.h>
#include <Access/Common/AllowedClientHosts.h>
@ -446,7 +447,8 @@ BlockIO InterpreterSystemQuery::execute()
[&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); },
[&] { if (auto query_views_log = getContext()->getQueryViewsLog()) query_views_log->flush(true); },
[&] { if (auto zookeeper_log = getContext()->getZooKeeperLog()) zookeeper_log->flush(true); },
[&] { if (auto session_log = getContext()->getSessionLog()) session_log->flush(true); }
[&] { if (auto session_log = getContext()->getSessionLog()) session_log->flush(true); },
[&] { if (auto transactions_info_log = getContext()->getTransactionsInfoLog()) transactions_info_log->flush(true); }
);
break;
}

View File

@ -33,7 +33,7 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart
{
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
new_part->versions.setMinTID(tid);
new_part->versions.setMinTID(tid, TransactionInfoContext{storage->getStorageID(), new_part->name});
if (txn)
txn->addNewPart(storage, new_part);
}
@ -41,10 +41,8 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart
void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn)
{
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
String error_context = fmt::format("Table: {}, part name: {}",
part_to_remove->storage.getStorageID().getNameForLogs(),
part_to_remove->name);
part_to_remove->versions.lockMaxTID(tid, error_context);
TransactionInfoContext context{storage->getStorageID(), part_to_remove->name};
part_to_remove->versions.lockMaxTID(tid, context);
if (txn)
txn->removeOldPart(storage, part_to_remove);
}
@ -53,17 +51,16 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage
{
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
new_part->versions.setMinTID(tid);
TransactionInfoContext context{storage->getStorageID(), new_part->name};
new_part->versions.setMinTID(tid, context);
if (txn)
txn->addNewPart(storage, new_part);
String error_context = fmt::format("Table: {}, covering part name: {}",
new_part->storage.getStorageID().getNameForLogs(),
new_part->name);
error_context += ", part_name: {}";
context.covering_part = std::move(context.part_name);
for (const auto & covered : covered_parts)
{
covered->versions.lockMaxTID(tid, fmt::format(error_context, covered->name));
context.part_name = covered->name;
covered->versions.lockMaxTID(tid, context);
if (txn)
txn->removeOldPart(storage, covered);
}
@ -152,7 +149,7 @@ bool MergeTreeTransaction::rollback() noexcept
part->versions.mincsn.store(Tx::RolledBackCSN);
for (const auto & part : removing_parts)
part->versions.unlockMaxTID(tid);
part->versions.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name});
/// FIXME const_cast
for (const auto & part : creating_parts)

View File

@ -10,6 +10,7 @@
#include <Interpreters/TextLog.h>
#include <Interpreters/TraceLog.h>
#include <Interpreters/ZooKeeperLog.h>
#include <Interpreters/TransactionsInfoLog.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <base/logger_useful.h>
@ -130,6 +131,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
query_views_log = createSystemLog<QueryViewsLog>(global_context, "system", "query_views_log", config, "query_views_log");
zookeeper_log = createSystemLog<ZooKeeperLog>(global_context, "system", "zookeeper_log", config, "zookeeper_log");
session_log = createSystemLog<SessionLog>(global_context, "system", "session_log", config, "session_log");
transactions_info_log = createSystemLog<TransactionsInfoLog>(
global_context, "system", "transactions_info_log", config, "transactions_info_log");
if (query_log)
logs.emplace_back(query_log.get());
@ -155,6 +158,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
logs.emplace_back(zookeeper_log.get());
if (session_log)
logs.emplace_back(session_log.get());
if (transactions_info_log)
logs.emplace_back(transactions_info_log.get());
try
{

View File

@ -78,6 +78,7 @@ class OpenTelemetrySpanLog;
class QueryViewsLog;
class ZooKeeperLog;
class SessionLog;
class TransactionsInfoLog;
class ISystemLog
@ -125,6 +126,8 @@ struct SystemLogs
std::shared_ptr<ZooKeeperLog> zookeeper_log;
/// Login, LogOut and Login failure events
std::shared_ptr<SessionLog> session_log;
/// Events related to transactions
std::shared_ptr<TransactionsInfoLog> transactions_info_log;
std::vector<ISystemLog *> logs;
};

View File

@ -1,6 +1,7 @@
#include <Interpreters/TransactionLog.h>
#include <Interpreters/TransactionVersionMetadata.h>
#include <Interpreters/Context.h>
#include <Interpreters/TransactionsInfoLog.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
@ -14,13 +15,34 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int LOGICAL_ERROR;
}
static void tryWriteEventToSystemLog(Poco::Logger * log, ContextPtr context,
TransactionsInfoLogElement::Type type, const TransactionID & tid, CSN csn = Tx::UnknownCSN)
try
{
auto system_log = context->getTransactionsInfoLog();
if (!system_log)
return;
TransactionsInfoLogElement elem;
elem.type = type;
elem.tid = tid;
elem.csn = csn;
elem.fillCommonFields(nullptr);
system_log->add(elem);
}
catch (...)
{
tryLogCurrentException(log);
}
TransactionLog & TransactionLog::instance()
{
static TransactionLog inst;
return inst;
/// Use unique_ptr to avoid races on initialization retries if exceptions was thrown from ctor
static std::unique_ptr<TransactionLog> inst = std::make_unique<TransactionLog>();
return *inst;
}
TransactionLog::TransactionLog()
@ -214,7 +236,10 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction()
throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid);
txn->snapshot_in_use_it = snapshots_in_use.insert(snapshots_in_use.end(), snapshot);
}
LOG_TRACE(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash());
LOG_TEST(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash());
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::BEGIN, txn->tid);
return txn;
}
@ -226,8 +251,9 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn)
/// TODO Transactions: reset local_tid_counter
if (txn->isReadOnly())
{
LOG_TRACE(log, "Closing readonly transaction {}", txn->tid);
LOG_TEST(log, "Closing readonly transaction {}", txn->tid);
new_csn = txn->snapshot;
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::COMMIT, txn->tid, new_csn);
}
else
{
@ -236,7 +262,9 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn)
/// TODO support batching
String path_created = zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point
new_csn = parseCSN(path_created.substr(zookeeper_path.size() + 1));
LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn);
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::COMMIT, txn->tid, new_csn);
/// Wait for committed changes to become actually visible, so the next transaction will see changes
/// TODO it's optional, add a setting for this
@ -257,13 +285,16 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn)
throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid);
snapshots_in_use.erase(txn->snapshot_in_use_it);
}
return new_csn;
}
void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept
{
LOG_TRACE(log, "Rolling back transaction {}", txn->tid);
if (txn->rollback())
if (!txn->rollback())
return;
{
std::lock_guard lock{running_list_mutex};
bool removed = running_list.erase(txn->tid.getHash());
@ -271,6 +302,8 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no
abort();
snapshots_in_use.erase(txn->snapshot_in_use_it);
}
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::ROLLBACK, txn->tid);
}
MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & tid)

View File

@ -10,6 +10,8 @@
namespace DB
{
class TransactionsInfoLog;
using TransactionsInfoLogPtr = std::shared_ptr<TransactionsInfoLog>;
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
class TransactionLog final : private boost::noncopyable

View File

@ -1,4 +1,5 @@
#include <Interpreters/TransactionVersionMetadata.h>
#include <Interpreters/TransactionsInfoLog.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
@ -6,8 +7,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
//#include <base/logger_useful.h>
#include <base/logger_useful.h>
namespace DB
{
@ -19,6 +19,32 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_TEXT;
}
static void tryWriteEventToSystemLog(Poco::Logger * log,
TransactionsInfoLogElement::Type type, const TransactionID & tid,
const TransactionInfoContext & context)
try
{
auto system_log = Context::getGlobalContextInstance()->getTransactionsInfoLog();
if (!system_log)
return;
TransactionsInfoLogElement elem;
elem.type = type;
elem.tid = tid;
elem.fillCommonFields(&context);
system_log->add(elem);
}
catch (...)
{
tryLogCurrentException(log);
}
VersionMetadata::VersionMetadata()
{
/// It would be better to make it static, but static loggers do not work for some reason (initialization order?)
log = &Poco::Logger::get("VersionMetadata");
}
/// It can be used for introspection purposes only
TransactionID VersionMetadata::getMaxTID() const
{
@ -40,21 +66,26 @@ TransactionID VersionMetadata::getMaxTID() const
return Tx::EmptyTID;
}
void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context)
void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context)
{
//LOG_TRACE(&Poco::Logger::get("WTF"), "Trying to lock maxtid by {}: {}\n{}", tid, error_context, StackTrace().toString());
LOG_TEST(log, "Trying to lock maxtid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name);
TIDHash locked_by = 0;
if (tryLockMaxTID(tid, &locked_by))
if (tryLockMaxTID(tid, context, &locked_by))
return;
String part_desc;
if (context.covering_part.empty())
part_desc = context.part_name;
else
part_desc = fmt::format("{} (covered by {})", context.part_name, context.covering_part);
throw Exception(ErrorCodes::SERIALIZATION_ERROR,
"Serialization error: "
"Transaction {} tried to remove data part, "
"but it's locked ({}) by another transaction {} which is currently removing this part. {}",
tid, locked_by, getMaxTID(), error_context);
"Transaction {} tried to remove data part {} from {}, "
"but it's locked by another transaction (TID: {}, TIDH: {}) which is currently removing this part.",
tid, part_desc, context.table.getNameForLogs(), getMaxTID(), locked_by);
}
bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id)
bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id)
{
assert(!tid.isEmpty());
TIDHash max_lock_value = tid.getHash();
@ -66,6 +97,7 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_
{
/// Don't need to lock part for queries without transaction
//FIXME Transactions: why is it possible?
LOG_TEST(log, "Assuming maxtid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name);
return true;
}
@ -75,12 +107,13 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_
}
maxtid = tid;
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::LOCK_PART, tid, context);
return true;
}
void VersionMetadata::unlockMaxTID(const TransactionID & tid)
void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context)
{
//LOG_TRACE(&Poco::Logger::get("WTF"), "Unlocking maxtid by {}", tid);
LOG_TEST(log, "Unlocking maxtid by {}", tid);
assert(!tid.isEmpty());
TIDHash max_lock_value = tid.getHash();
TIDHash locked_by = maxtid_lock.load();
@ -99,6 +132,8 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid)
bool unlocked = maxtid_lock.compare_exchange_strong(locked_by, 0);
if (!unlocked)
throw_cannot_unlock();
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::UNLOCK_PART, tid, context);
}
bool VersionMetadata::isMaxTIDLocked() const
@ -106,12 +141,14 @@ bool VersionMetadata::isMaxTIDLocked() const
return maxtid_lock.load() != 0;
}
void VersionMetadata::setMinTID(const TransactionID & tid)
void VersionMetadata::setMinTID(const TransactionID & tid, const TransactionInfoContext & context)
{
/// TODO Transactions: initialize it in constructor on part creation and remove this method
/// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times
assert(mintid.isEmpty() || mintid == tid);
const_cast<TransactionID &>(mintid) = tid;
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, context);
}
bool VersionMetadata::isVisible(const MergeTreeTransaction & txn)
@ -121,13 +158,12 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn)
bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid)
{
//Poco::Logger * log = &Poco::Logger::get("WTF");
assert(!mintid.isEmpty());
CSN min = mincsn.load(std::memory_order_relaxed);
TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed);
CSN max = maxcsn.load(std::memory_order_relaxed);
//LOG_TRACE(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid);
//LOG_TEST(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid);
[[maybe_unused]] bool had_mincsn = min;
[[maybe_unused]] bool had_maxtid = max_lock;
@ -184,7 +220,6 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current
/// But for long-running writing transactions we will always do
/// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back.
min = TransactionLog::instance().getCSN(mintid);
//LOG_TRACE(log, "Got min {}", min);
if (!min)
return false; /// Part creation is not committed yet
@ -195,7 +230,6 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current
if (max_lock)
{
max = TransactionLog::instance().getCSN(max_lock);
//LOG_TRACE(log, "Got ax {}", max);
if (max)
maxcsn.store(max, std::memory_order_relaxed);
}
@ -206,11 +240,13 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current
bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version)
{
CSN min = mincsn.load(std::memory_order_relaxed);
/// We can safely remove part if its creation was rolled back
if (min == Tx::RolledBackCSN)
return true;
if (!min)
{
/// Cannot remove part if its creation not committed yet
min = TransactionLog::instance().getCSN(mintid);
if (min)
mincsn.store(min, std::memory_order_relaxed);
@ -218,16 +254,19 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version)
return false;
}
/// Part is probably visible for some transactions (part is too new or the oldest snapshot is too old)
if (oldest_snapshot_version < min)
return false;
TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed);
/// Part is active
if (!max_lock)
return false;
CSN max = maxcsn.load(std::memory_order_relaxed);
if (!max)
{
/// Part removal is not committed yet
max = TransactionLog::instance().getCSN(max_lock);
if (max)
maxcsn.store(max, std::memory_order_relaxed);
@ -235,6 +274,7 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version)
return false;
}
/// We can safely remove part if all running transactions were started after part removal was committed
return max <= oldest_snapshot_version;
}

View File

@ -1,9 +1,24 @@
#pragma once
#include <Common/TransactionID.h>
#include <Interpreters/StorageID.h>
namespace Poco
{
class Logger;
}
namespace DB
{
struct TransactionInfoContext
{
StorageID table = StorageID::createEmpty();
String part_name;
String covering_part;
TransactionInfoContext(StorageID id, String part) : table(std::move(id)), part_name(std::move(part)) {}
};
struct VersionMetadata
{
TransactionID mintid = Tx::EmptyTID;
@ -20,14 +35,14 @@ struct VersionMetadata
TransactionID getMinTID() const { return mintid; }
TransactionID getMaxTID() const;
bool tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id = nullptr);
void lockMaxTID(const TransactionID & tid, const String & error_context = {});
void unlockMaxTID(const TransactionID & tid);
bool tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr);
void lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context);
void unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context);
bool isMaxTIDLocked() const;
/// It can be called only from MergeTreeTransaction or on server startup
void setMinTID(const TransactionID & tid);
void setMinTID(const TransactionID & tid, const TransactionInfoContext & context);
bool canBeRemoved(Snapshot oldest_snapshot_version);
@ -35,6 +50,9 @@ struct VersionMetadata
void read(ReadBuffer & buf);
String toString(bool one_line = true) const;
Poco::Logger * log;
VersionMetadata();
};
DataTypePtr getTransactionIDDataType();

View File

@ -0,0 +1,88 @@
#include <Interpreters/TransactionsInfoLog.h>
#include <Interpreters/TransactionVersionMetadata.h>
#include <Common/TransactionID.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
#include <base/getThreadId.h>
namespace DB
{
NamesAndTypesList TransactionsInfoLogElement::getNamesAndTypes()
{
auto type_enum = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values
{
{"Begin", static_cast<Int8>(BEGIN)},
{"Commit", static_cast<Int8>(COMMIT)},
{"Rollback", static_cast<Int8>(ROLLBACK)},
{"AddPart", static_cast<Int8>(ADD_PART)},
{"LockPart", static_cast<Int8>(LOCK_PART)},
{"UnlockPart", static_cast<Int8>(UNLOCK_PART)},
});
return
{
{"type", std::move(type_enum)},
{"event_date", std::make_shared<DataTypeDate>()},
{"event_time", std::make_shared<DataTypeDateTime64>(6)},
{"thread_id", std::make_shared<DataTypeUInt64>()},
{"query_id", std::make_shared<DataTypeString>()},
{"tid", getTransactionIDDataType()},
{"tid_hash", std::make_shared<DataTypeUInt64>()},
{"csn", std::make_shared<DataTypeUInt64>()},
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"uuid", std::make_shared<DataTypeUUID>()},
{"part", std::make_shared<DataTypeString>()},
};
}
void TransactionsInfoLogElement::fillCommonFields(const TransactionInfoContext * context)
{
event_time = std::chrono::duration_cast<std::chrono::microseconds>(std::chrono::system_clock::now().time_since_epoch()).count();
thread_id = getThreadId();
query_id = CurrentThread::getQueryId().toString();
if (!context)
return;
table = context->table;
part_name = context->part_name;
}
void TransactionsInfoLogElement::appendToBlock(MutableColumns & columns) const
{
assert(type != UNKNOWN);
assert(!tid.isEmpty());
size_t i = 0;
columns[i++]->insert(type);
auto event_time_seconds = event_time / 1000000;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time_seconds).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(thread_id);
columns[i++]->insert(query_id);
columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id});
columns[i++]->insert(tid.getHash());
columns[i++]->insert(csn);
columns[i++]->insert(table.database_name);
columns[i++]->insert(table.table_name);
columns[i++]->insert(table.uuid);
columns[i++]->insert(part_name);
}
}

View File

@ -0,0 +1,54 @@
#pragma once
#include <Core/NamesAndAliases.h>
#include <Interpreters/SystemLog.h>
#include <Common/TransactionID.h>
namespace DB
{
struct TransactionInfoContext;
struct TransactionsInfoLogElement
{
enum Type
{
UNKNOWN = 0,
BEGIN = 1,
COMMIT = 2,
ROLLBACK = 3,
ADD_PART = 10,
LOCK_PART = 11,
UNLOCK_PART = 12,
};
Type type = UNKNOWN;
Decimal64 event_time = 0;
UInt64 thread_id;
String query_id;
TransactionID tid = Tx::EmptyTID;
/// For COMMIT events
CSN csn = Tx::UnknownCSN;
/// For *_PART events
StorageID table = StorageID::createEmpty();
String part_name;
static std::string name() { return "TransactionsInfoLog"; }
static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const;
void fillCommonFields(const TransactionInfoContext * context = nullptr);
};
class TransactionsInfoLog : public SystemLog<TransactionsInfoLogElement>
{
using SystemLog<TransactionsInfoLogElement>::SystemLog;
};
}

View File

@ -1156,13 +1156,15 @@ void IMergeTreeDataPart::loadVersionMetadata() const
/// so part will be ether broken or known to be created by transaction.
/// 4. Fsyncs in storeVersionMetadata() work incorrectly.
TransactionInfoContext txn_context{storage.getStorageID(), name};
if (!disk->exists(tmp_version_file_name))
{
/// Case 1 (or 3).
/// We do not have version metadata and transactions history for old parts,
/// so let's consider that such parts were created by some ancient transaction
/// and were committed with some prehistoric CSN.
versions.setMinTID(Tx::PrehistoricTID);
versions.setMinTID(Tx::PrehistoricTID, txn_context);
versions.mincsn = Tx::PrehistoricCSN;
return;
}
@ -1170,7 +1172,7 @@ void IMergeTreeDataPart::loadVersionMetadata() const
/// Case 2.
/// Content of *.tmp file may be broken, just use fake TID.
/// Transaction was not committed if *.tmp file was not renamed, so we should complete rollback by removing part.
versions.setMinTID(Tx::DummyTID);
versions.setMinTID(Tx::DummyTID, txn_context);
versions.mincsn = Tx::RolledBackCSN;
remove_tmp_file();
}

View File

@ -1329,7 +1329,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
/// Transaction that tried to remove this part was not committed. Clear maxtid.
LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: clearing maxtid={}",
part->name, versions.maxtid);
versions.unlockMaxTID(versions.maxtid);
versions.unlockMaxTID(versions.maxtid, TransactionInfoContext{getStorageID(), part->name});
}
versions_updated = true;
}
@ -4445,7 +4445,7 @@ void MergeTreeData::Transaction::rollback()
DataPartPtr covering_part;
DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, lock);
for (auto & covered : covered_parts)
covered->versions.unlockMaxTID(Tx::PrehistoricTID);
covered->versions.unlockMaxTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name});
}
}

View File

@ -1,3 +1,10 @@
<yandex>
<_enable_experimental_mvcc_prototype_test_helper_dev>42</_enable_experimental_mvcc_prototype_test_helper_dev>
<transactions_info_log>
<database>system</database>
<table>transactions_info_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</transactions_info_log>
</yandex>

View File

@ -14,4 +14,5 @@
<opentelemetry_span_log remove="remove" />
<!-- just in case it will be enabled by default -->
<zookeeper_log remove="remove" />
<transactions_info_log remove="remove" />
</clickhouse>

View File

@ -16,3 +16,25 @@
7 all_3_3_0 (0,0,'00000000-0000-0000-0000-000000000000') 0
7 all_4_4_0 (0,0,'00000000-0000-0000-0000-000000000000') 0
8 1
1 1 AddPart 1 1 1 1 all_1_1_0
2 1 Begin 1 1 1 1
2 1 AddPart 1 1 1 1 all_2_2_0
2 1 Rollback 1 1 1 1
3 1 Begin 1 1 1 1
3 1 AddPart 1 1 1 1 all_3_3_0
3 1 Commit 1 1 1 0
1 1 AddPart 1 1 1 1 all_1_1_0
4 1 Begin 1 1 1 1
4 1 AddPart 1 1 1 1 all_4_4_0
4 1 Commit 1 1 1 0
5 1 Begin 1 1 1 1
5 1 AddPart 1 1 1 1 all_5_5_0
5 1 LockPart 1 1 1 1 all_1_1_0
5 1 LockPart 1 1 1 1 all_3_3_0
5 1 LockPart 1 1 1 1 all_4_4_0
5 1 LockPart 1 1 1 1 all_5_5_0
5 1 UnlockPart 1 1 1 1 all_1_1_0
5 1 UnlockPart 1 1 1 1 all_3_3_0
5 1 UnlockPart 1 1 1 1 all_4_4_0
5 1 UnlockPart 1 1 1 1 all_5_5_0
5 1 Rollback 1 1 1 1

View File

@ -31,5 +31,16 @@ select 7, name, maxtid, maxcsn from system.parts where database=currentDatabase(
select 8, transactionID().3 == serverUUID();
commit;
begin transaction;
insert into txn_counters(n) values (5);
alter table txn_counters drop partition id 'all';
rollback;
system flush logs;
select indexOf((select arraySort(groupUniqArray(tid)) from system.transactions_info_log where database=currentDatabase() and table='txn_counters'), tid),
(toDecimal64(now64(6), 6) - toDecimal64(event_time, 6)) < 100, type, thread_id!=0, length(query_id)=length(queryID()), tid_hash!=0, csn=0, part
from system.transactions_info_log
where tid in (select tid from system.transactions_info_log where database=currentDatabase() and table='txn_counters' and not (tid.1=1 and tid.2=1))
or (database=currentDatabase() and table='txn_counters') order by event_time;
drop table txn_counters;