mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
add system log for introspection
This commit is contained in:
parent
158fbaaa29
commit
e1a41fc694
@ -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
|
||||
|
@ -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();
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
88
src/Interpreters/TransactionsInfoLog.cpp
Normal file
88
src/Interpreters/TransactionsInfoLog.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
54
src/Interpreters/TransactionsInfoLog.h
Normal file
54
src/Interpreters/TransactionsInfoLog.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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();
|
||||
}
|
||||
|
@ -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});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user