ClickHouse/src/Interpreters/TransactionVersionMetadata.cpp

230 lines
7.8 KiB
C++
Raw Normal View History

2021-11-08 18:56:09 +00:00
#include <Interpreters/TransactionVersionMetadata.h>
2021-03-31 17:55:04 +00:00
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
2021-04-08 17:20:45 +00:00
#include <Interpreters/TransactionLog.h>
2021-03-31 17:55:04 +00:00
2021-11-08 18:56:09 +00:00
//#include <base/logger_useful.h>
2021-03-31 17:55:04 +00:00
namespace DB
{
2021-04-08 17:20:45 +00:00
namespace ErrorCodes
{
2021-11-08 18:56:09 +00:00
extern const int SERIALIZATION_ERROR;
extern const int LOGICAL_ERROR;
2021-04-08 17:20:45 +00:00
}
2021-06-04 09:26:47 +00:00
/// It can be used for introspection purposes only
2021-04-08 17:20:45 +00:00
TransactionID VersionMetadata::getMaxTID() const
{
TIDHash max_lock = maxtid_lock.load();
if (max_lock)
{
if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock))
return txn->tid;
2021-06-04 09:26:47 +00:00
if (max_lock == Tx::PrehistoricTID.getHash())
return Tx::PrehistoricTID;
2021-04-08 17:20:45 +00:00
}
if (maxcsn.load(std::memory_order_relaxed))
{
/// maxtid cannot be changed since we have maxcsn, so it's readonly
return maxtid;
}
return Tx::EmptyTID;
}
void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context)
{
2021-06-04 09:26:47 +00:00
assert(tid);
2021-04-08 17:20:45 +00:00
TIDHash max_lock_value = tid.getHash();
TIDHash expected_max_lock_value = 0;
bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value);
if (!locked)
{
2021-11-17 18:14:14 +00:00
if (tid == Tx::PrehistoricTID && expected_max_lock_value == Tx::PrehistoricTID.getHash())
{
/// Don't need to lock part for queries without transaction
//FIXME Transactions: why is it possible?
return;
}
2021-04-08 17:20:45 +00:00
throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: "
2021-11-08 18:56:09 +00:00
"Transaction {} tried to remove data part, "
"but it's locked ({}) by another transaction {} which is currently removing this part. {}",
2021-04-08 17:20:45 +00:00
tid, expected_max_lock_value, getMaxTID(), error_context);
}
maxtid = tid;
}
void VersionMetadata::unlockMaxTID(const TransactionID & tid)
{
2021-06-04 09:26:47 +00:00
assert(tid);
2021-04-08 17:20:45 +00:00
TIDHash max_lock_value = tid.getHash();
TIDHash locked_by = maxtid_lock.load();
auto throw_cannot_unlock = [&]()
{
auto locked_by_txn = TransactionLog::instance().tryGetRunningTransaction(locked_by);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unlock maxtid, it's a bug. Current: {} {}, actual: {} {}",
max_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID);
};
if (locked_by != max_lock_value)
throw_cannot_unlock();
maxtid = Tx::EmptyTID;
bool unlocked = maxtid_lock.compare_exchange_strong(locked_by, 0);
if (!unlocked)
throw_cannot_unlock();
}
2021-06-04 09:26:47 +00:00
bool VersionMetadata::isMaxTIDLocked() const
{
return maxtid_lock.load() != 0;
}
2021-04-08 17:20:45 +00:00
void VersionMetadata::setMinTID(const TransactionID & tid)
{
/// TODO Transactions: initialize it in constructor on part creation and remove this method
2021-06-02 20:03:44 +00:00
/// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times
assert(!mintid || mintid == tid);
2021-04-08 17:20:45 +00:00
const_cast<TransactionID &>(mintid) = tid;
}
bool VersionMetadata::isVisible(const MergeTreeTransaction & txn)
{
2021-06-04 09:26:47 +00:00
return isVisible(txn.getSnapshot(), txn.tid);
}
bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid)
{
2021-11-08 18:56:09 +00:00
//Poco::Logger * log = &Poco::Logger::get("WTF");
2021-04-08 17:20:45 +00:00
assert(mintid);
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);
2021-11-08 18:56:09 +00:00
//LOG_TRACE(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid);
2021-04-08 17:20:45 +00:00
[[maybe_unused]] bool had_mincsn = min;
[[maybe_unused]] bool had_maxtid = max_lock;
[[maybe_unused]] bool had_maxcsn = max;
assert(!had_maxcsn || had_maxtid);
assert(!had_maxcsn || had_mincsn);
/// Fast path:
/// Part is definitely not visible if:
/// - creation was committed after we took the snapshot
/// - removal was committed before we took the snapshot
/// - current transaction is removing it
if (min && snapshot_version < min)
return false;
if (max && max <= snapshot_version)
return false;
2021-06-04 09:26:47 +00:00
if (current_tid && max_lock && max_lock == current_tid.getHash())
2021-04-08 17:20:45 +00:00
return false;
/// Otherwise, part is definitely visible if:
2021-05-18 17:07:29 +00:00
/// - creation was committed before we took the snapshot and nobody tried to remove the part
/// - creation was committed before and removal was committed after
2021-04-08 17:20:45 +00:00
/// - current transaction is creating it
2021-05-18 17:07:29 +00:00
if (min && min <= snapshot_version && !max_lock)
return true;
if (min && min <= snapshot_version && max && snapshot_version < max)
2021-04-08 17:20:45 +00:00
return true;
2021-06-04 09:26:47 +00:00
if (current_tid && mintid == current_tid)
2021-04-08 17:20:45 +00:00
return true;
/// End of fast path.
/// Data part has mintid/maxtid, but does not have mincsn/maxcsn.
/// It means that some transaction is creating/removing the part right now or has done it recently
2021-11-08 18:56:09 +00:00
/// and we don't know if it was already committed or not.
2021-04-08 17:20:45 +00:00
assert(!had_mincsn || (had_maxtid && !had_maxcsn));
2021-06-04 09:26:47 +00:00
assert(!current_tid || (mintid != current_tid && max_lock != current_tid.getHash()));
2021-04-08 17:20:45 +00:00
/// Before doing CSN lookup, let's check some extra conditions.
2021-11-08 18:56:09 +00:00
/// If snapshot_version <= some_tid.start_csn, then changes of the transaction with some_tid
/// are definitely not visible for us (because the transaction can be committed with greater CSN only),
/// so we don't need to check if it was committed.
2021-04-08 17:20:45 +00:00
if (snapshot_version <= mintid.start_csn)
return false;
/// Check if mintid/maxtid transactions are committed and write CSNs
/// TODO Transactions: we probably need some optimizations here
/// to avoid some CSN lookups or make the lookups cheaper.
/// NOTE: Old enough committed parts always have written CSNs,
/// so we can determine their visibility through fast path.
/// 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);
2021-11-08 18:56:09 +00:00
//LOG_TRACE(log, "Got min {}", min);
2021-04-08 17:20:45 +00:00
if (!min)
return false; /// Part creation is not committed yet
2021-11-08 18:56:09 +00:00
/// We don't need to check if CSNs are already written or not,
/// because once written CSN cannot be changed, so it's safe to overwrite it (with the same value).
2021-04-08 17:20:45 +00:00
mincsn.store(min, std::memory_order_relaxed);
if (max_lock)
{
max = TransactionLog::instance().getCSN(max_lock);
2021-11-08 18:56:09 +00:00
//LOG_TRACE(log, "Got ax {}", max);
2021-04-08 17:20:45 +00:00
if (max)
maxcsn.store(max, std::memory_order_relaxed);
}
return min <= snapshot_version && (!max || snapshot_version < max);
}
2021-06-11 12:14:38 +00:00
bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version)
{
CSN min = mincsn.load(std::memory_order_relaxed);
if (min == Tx::RolledBackCSN)
return true;
if (!min)
{
min = TransactionLog::instance().getCSN(mintid);
if (min)
mincsn.store(min, std::memory_order_relaxed);
else
return false;
}
if (oldest_snapshot_version < min)
return false;
TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed);
if (!max_lock)
return false;
CSN max = maxcsn.load(std::memory_order_relaxed);
if (!max)
{
max = TransactionLog::instance().getCSN(max_lock);
if (max)
maxcsn.store(max, std::memory_order_relaxed);
else
return false;
}
return max <= oldest_snapshot_version;
}
2021-11-08 18:56:09 +00:00
DataTypePtr getTransactionIDDataType()
{
DataTypes types;
types.push_back(std::make_shared<DataTypeUInt64>());
types.push_back(std::make_shared<DataTypeUInt64>());
types.push_back(std::make_shared<DataTypeUUID>());
return std::make_shared<DataTypeTuple>(std::move(types));
}
2021-03-31 17:55:04 +00:00
}