mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge pull request #24258 from ClickHouse/mvcc_prototype
MVCC prototype for MergeTree
This commit is contained in:
commit
faae540a85
@ -828,7 +828,6 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
|
||||
|
||||
/// Setup signal handlers.
|
||||
/// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime.
|
||||
|
||||
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, &handled_signals);
|
||||
addSignalHandler({SIGHUP}, closeLogsSignalHandler, &handled_signals);
|
||||
addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals);
|
||||
|
@ -139,7 +139,7 @@ pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhous
|
||||
# directly
|
||||
# - even though ci auto-compress some files (but not *.tsv) it does this only
|
||||
# for files >64MB, we want this files to be compressed explicitly
|
||||
for table in query_log zookeeper_log trace_log
|
||||
for table in query_log zookeeper_log trace_log transactions_info_log
|
||||
do
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz &
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
|
@ -1639,6 +1639,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
server.start();
|
||||
LOG_INFO(log, "Listening for {}", server.getDescription());
|
||||
}
|
||||
|
||||
global_context->setServerCompletelyStarted();
|
||||
LOG_INFO(log, "Ready for connections.");
|
||||
}
|
||||
|
||||
|
@ -102,6 +102,7 @@ enum class AccessType
|
||||
\
|
||||
M(KILL_QUERY, "", GLOBAL, ALL) /* allows to kill a query started by another user
|
||||
(anyone can kill his own queries) */\
|
||||
M(KILL_TRANSACTION, "", GLOBAL, ALL) \
|
||||
\
|
||||
M(MOVE_PARTITION_BETWEEN_SHARDS, "", GLOBAL, ALL) /* required to be able to move a part/partition to a table
|
||||
identified by its ZooKeeper path */\
|
||||
|
@ -1129,7 +1129,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
|
||||
sendDataFromPipe(
|
||||
storage->read(
|
||||
sample.getNames(),
|
||||
storage->getStorageSnapshot(metadata),
|
||||
storage->getStorageSnapshot(metadata, global_context),
|
||||
query_info,
|
||||
global_context,
|
||||
{},
|
||||
|
@ -617,6 +617,8 @@
|
||||
M(646, CANNOT_BACKUP_DATABASE) \
|
||||
M(647, CANNOT_BACKUP_TABLE) \
|
||||
M(648, WRONG_DDL_RENAMING_SETTINGS) \
|
||||
M(649, INVALID_TRANSACTION) \
|
||||
M(650, SERIALIZATION_ERROR) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/SystemLogBase.h>
|
||||
|
@ -23,6 +23,7 @@
|
||||
M(QueryViewsLogElement) \
|
||||
M(SessionLogElement) \
|
||||
M(TraceLogElement) \
|
||||
M(TransactionsInfoLogElement) \
|
||||
M(ZooKeeperLogElement) \
|
||||
M(ProcessorProfileLogElement) \
|
||||
M(TextLogElement)
|
||||
|
43
src/Common/TransactionID.cpp
Normal file
43
src/Common/TransactionID.cpp
Normal file
@ -0,0 +1,43 @@
|
||||
#include <Common/TransactionID.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
TIDHash TransactionID::getHash() const
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(start_csn);
|
||||
hash.update(local_tid);
|
||||
hash.update(host_id);
|
||||
return hash.get64();
|
||||
}
|
||||
|
||||
|
||||
void TransactionID::write(const TransactionID & tid, WriteBuffer & buf)
|
||||
{
|
||||
writeChar('(', buf);
|
||||
writeText(tid.start_csn, buf);
|
||||
writeCString(", ", buf);
|
||||
writeText(tid.local_tid, buf);
|
||||
writeCString(", ", buf);
|
||||
writeText(tid.host_id, buf);
|
||||
writeChar(')', buf);
|
||||
}
|
||||
|
||||
TransactionID TransactionID::read(ReadBuffer & buf)
|
||||
{
|
||||
TransactionID tid = Tx::EmptyTID;
|
||||
assertChar('(', buf);
|
||||
readText(tid.start_csn, buf);
|
||||
assertString(", ", buf);
|
||||
readText(tid.local_tid, buf);
|
||||
assertString(", ", buf);
|
||||
readText(tid.host_id, buf);
|
||||
assertChar(')', buf);
|
||||
return tid;
|
||||
}
|
||||
|
||||
}
|
115
src/Common/TransactionID.h
Normal file
115
src/Common/TransactionID.h
Normal file
@ -0,0 +1,115 @@
|
||||
#pragma once
|
||||
#include <Core/Types.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <fmt/format.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
class MergeTreeTransaction;
|
||||
|
||||
/// This macro is useful for places where a pointer to current transaction should be passed,
|
||||
/// but transactions are not supported yet (e.g. when calling MergeTreeData's methods from StorageReplicatedMergeTree)
|
||||
/// or transaction object is not needed and not passed intentionally.
|
||||
#ifndef NO_TRANSACTION_PTR
|
||||
#define NO_TRANSACTION_PTR std::shared_ptr<MergeTreeTransaction>(nullptr)
|
||||
#define NO_TRANSACTION_RAW static_cast<MergeTreeTransaction *>(nullptr)
|
||||
#endif
|
||||
|
||||
/// Commit Sequence Number
|
||||
using CSN = UInt64;
|
||||
/// Local part of TransactionID
|
||||
using LocalTID = UInt64;
|
||||
/// Hash of TransactionID that fits into 64-bit atomic
|
||||
using TIDHash = UInt64;
|
||||
|
||||
namespace Tx
|
||||
{
|
||||
/// For transactions that are probably not committed (yet)
|
||||
const CSN UnknownCSN = 0;
|
||||
/// For changes were made without creating a transaction
|
||||
const CSN PrehistoricCSN = 1;
|
||||
/// Special reserved values
|
||||
const CSN CommittingCSN = 2;
|
||||
const CSN EverythingVisibleCSN = 3;
|
||||
const CSN MaxReservedCSN = 32;
|
||||
|
||||
/// So far, that changes will never become visible
|
||||
const CSN RolledBackCSN = std::numeric_limits<CSN>::max();
|
||||
|
||||
const LocalTID PrehistoricLocalTID = 1;
|
||||
const LocalTID DummyLocalTID = 2;
|
||||
const LocalTID MaxReservedLocalTID = 32;
|
||||
}
|
||||
|
||||
struct TransactionID
|
||||
{
|
||||
/// Global sequential number, the newest commit timestamp the we saw when this transaction began
|
||||
CSN start_csn = 0;
|
||||
/// Local sequential that is unique for each transaction started by this host within specific start_csn
|
||||
LocalTID local_tid = 0;
|
||||
/// UUID of host that has started this transaction
|
||||
UUID host_id = UUIDHelpers::Nil;
|
||||
|
||||
/// NOTE Maybe we could just generate UUIDv4 for each transaction, but it would be harder to debug.
|
||||
/// Partial order is defined for this TransactionID structure:
|
||||
/// (tid1.start_csn <= tid2.start_csn) <==> (tid1 <= tid2)
|
||||
/// (tid1.start_csn == tid2.start_csn && tid1.host_id == tid2.host_id && tid1.local_tid < tid2.local_tid) ==> (tid1 < tid2)
|
||||
/// If two transaction have the same start_csn, but were started by different hosts, then order is undefined.
|
||||
|
||||
bool operator == (const TransactionID & rhs) const
|
||||
{
|
||||
return start_csn == rhs.start_csn && local_tid == rhs.local_tid && host_id == rhs.host_id;
|
||||
}
|
||||
|
||||
bool operator != (const TransactionID & rhs) const
|
||||
{
|
||||
return !(*this == rhs);
|
||||
}
|
||||
|
||||
TIDHash getHash() const;
|
||||
|
||||
bool isEmpty() const
|
||||
{
|
||||
assert((local_tid == 0) == (start_csn == 0 && host_id == UUIDHelpers::Nil));
|
||||
return local_tid == 0;
|
||||
}
|
||||
|
||||
bool isPrehistoric() const
|
||||
{
|
||||
assert((local_tid == Tx::PrehistoricLocalTID) == (start_csn == Tx::PrehistoricCSN));
|
||||
return local_tid == Tx::PrehistoricLocalTID;
|
||||
}
|
||||
|
||||
|
||||
static void write(const TransactionID & tid, WriteBuffer & buf);
|
||||
static TransactionID read(ReadBuffer & buf);
|
||||
};
|
||||
|
||||
namespace Tx
|
||||
{
|
||||
const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil};
|
||||
const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil};
|
||||
const TransactionID DummyTID = {PrehistoricCSN, DummyLocalTID, UUIDHelpers::Nil};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
template<>
|
||||
struct fmt::formatter<DB::TransactionID>
|
||||
{
|
||||
template<typename ParseContext>
|
||||
constexpr auto parse(ParseContext & context)
|
||||
{
|
||||
return context.begin();
|
||||
}
|
||||
|
||||
template<typename FormatContext>
|
||||
auto format(const DB::TransactionID & tid, FormatContext & context)
|
||||
{
|
||||
return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id);
|
||||
}
|
||||
};
|
@ -1270,4 +1270,14 @@ String extractZooKeeperPath(const String & path, bool check_starts_with_slash, P
|
||||
return normalizeZooKeeperPath(path, check_starts_with_slash, log);
|
||||
}
|
||||
|
||||
String getSequentialNodeName(const String & prefix, UInt64 number)
|
||||
{
|
||||
/// NOTE Sequential counter in ZooKeeper is Int32.
|
||||
assert(number < std::numeric_limits<Int32>::max());
|
||||
constexpr size_t seq_node_digits = 10;
|
||||
String num_str = std::to_string(number);
|
||||
String name = prefix + String(seq_node_digits - num_str.size(), '0') + num_str;
|
||||
return name;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -417,4 +417,6 @@ String extractZooKeeperName(const String & path);
|
||||
|
||||
String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr);
|
||||
|
||||
String getSequentialNodeName(const String & prefix, UInt64 number);
|
||||
|
||||
}
|
||||
|
@ -581,6 +581,7 @@ class IColumn;
|
||||
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
|
||||
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
|
||||
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
|
||||
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
|
||||
// End of COMMON_SETTINGS
|
||||
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS.
|
||||
|
||||
|
@ -461,6 +461,10 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
|
||||
|
||||
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context)
|
||||
{
|
||||
|
||||
if (query_context->getCurrentTransaction() && query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported");
|
||||
|
||||
if (is_readonly)
|
||||
throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper");
|
||||
|
||||
|
@ -41,9 +41,9 @@ public:
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
return ColumnT().createColumnConst(input_rows_count, constant_value);
|
||||
return result_type->createColumnConst(input_rows_count, constant_value);
|
||||
}
|
||||
|
||||
private:
|
||||
|
71
src/Functions/FunctionsTransactionCounters.cpp
Normal file
71
src/Functions/FunctionsTransactionCounters.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include <Functions/FunctionConstantBase.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class FunctionTransactionID : public FunctionConstantBase<FunctionTransactionID, Tuple, DataTypeNothing>
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "transactionID";
|
||||
static Tuple getValue(const MergeTreeTransactionPtr & txn)
|
||||
{
|
||||
Tuple res;
|
||||
if (txn)
|
||||
res = {txn->tid.start_csn, txn->tid.local_tid, txn->tid.host_id};
|
||||
else
|
||||
res = {UInt64(0), UInt64(0), UUIDHelpers::Nil};
|
||||
return res;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return getTransactionIDDataType(); }
|
||||
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTransactionID>(context); }
|
||||
explicit FunctionTransactionID(ContextPtr context) : FunctionConstantBase(getValue(context->getCurrentTransaction()), context->isDistributed()) {}
|
||||
};
|
||||
|
||||
class FunctionTransactionLatestSnapshot : public FunctionConstantBase<FunctionTransactionLatestSnapshot, UInt64, DataTypeUInt64>
|
||||
{
|
||||
static UInt64 getLatestSnapshot(ContextPtr context)
|
||||
{
|
||||
context->checkTransactionsAreAllowed(/* explicit_tcl_query */ true);
|
||||
return TransactionLog::instance().getLatestSnapshot();
|
||||
}
|
||||
public:
|
||||
static constexpr auto name = "transactionLatestSnapshot";
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTransactionLatestSnapshot>(context); }
|
||||
explicit FunctionTransactionLatestSnapshot(ContextPtr context) : FunctionConstantBase(getLatestSnapshot(context), context->isDistributed()) {}
|
||||
};
|
||||
|
||||
class FunctionTransactionOldestSnapshot : public FunctionConstantBase<FunctionTransactionOldestSnapshot, UInt64, DataTypeUInt64>
|
||||
{
|
||||
static UInt64 getOldestSnapshot(ContextPtr context)
|
||||
{
|
||||
context->checkTransactionsAreAllowed(/* explicit_tcl_query */ true);
|
||||
return TransactionLog::instance().getOldestSnapshot();
|
||||
}
|
||||
public:
|
||||
static constexpr auto name = "transactionOldestSnapshot";
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTransactionOldestSnapshot>(context); }
|
||||
explicit FunctionTransactionOldestSnapshot(ContextPtr context) : FunctionConstantBase(getOldestSnapshot(context), context->isDistributed()) {}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionsTransactionCounters(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionTransactionID>();
|
||||
factory.registerFunction<FunctionTransactionLatestSnapshot>();
|
||||
factory.registerFunction<FunctionTransactionOldestSnapshot>();
|
||||
}
|
||||
|
||||
}
|
@ -75,6 +75,7 @@ void registerFunctionFile(FunctionFactory &);
|
||||
void registerFunctionConnectionId(FunctionFactory &);
|
||||
void registerFunctionPartitionId(FunctionFactory &);
|
||||
void registerFunctionIsIPAddressContainedIn(FunctionFactory &);
|
||||
void registerFunctionsTransactionCounters(FunctionFactory & factory);
|
||||
void registerFunctionQueryID(FunctionFactory &);
|
||||
void registerFunctionInitialQueryID(FunctionFactory &);
|
||||
void registerFunctionServerUUID(FunctionFactory &);
|
||||
@ -163,6 +164,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionConnectionId(factory);
|
||||
registerFunctionPartitionId(factory);
|
||||
registerFunctionIsIPAddressContainedIn(factory);
|
||||
registerFunctionsTransactionCounters(factory);
|
||||
registerFunctionQueryID(factory);
|
||||
registerFunctionInitialQueryID(factory);
|
||||
registerFunctionServerUUID(factory);
|
||||
|
@ -26,6 +26,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_DATE;
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
}
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
@ -137,6 +138,12 @@ void assertEOF(ReadBuffer & buf)
|
||||
throwAtAssertionFailed("eof", buf);
|
||||
}
|
||||
|
||||
void assertNotEOF(ReadBuffer & buf)
|
||||
{
|
||||
if (buf.eof())
|
||||
throw Exception("Attempt to read after EOF", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
|
||||
}
|
||||
|
||||
|
||||
void assertStringCaseInsensitive(const char * s, ReadBuffer & buf)
|
||||
{
|
||||
|
@ -163,6 +163,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
|
||||
|
||||
void assertString(const char * s, ReadBuffer & buf);
|
||||
void assertEOF(ReadBuffer & buf);
|
||||
void assertNotEOF(ReadBuffer & buf);
|
||||
|
||||
[[noreturn]] void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
|
||||
|
||||
|
@ -133,7 +133,6 @@ off_t WriteBufferFromFileDescriptor::seek(off_t offset, int whence) // NOLINT
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void WriteBufferFromFileDescriptor::truncate(off_t length) // NOLINT
|
||||
{
|
||||
int res = ftruncate(fd, length);
|
||||
|
@ -1163,3 +1163,19 @@ struct PcgSerializer
|
||||
void writePointerHex(const void * ptr, WriteBuffer & buf);
|
||||
|
||||
}
|
||||
|
||||
template<>
|
||||
struct fmt::formatter<DB::UUID>
|
||||
{
|
||||
template<typename ParseContext>
|
||||
constexpr auto parse(ParseContext & context)
|
||||
{
|
||||
return context.begin();
|
||||
}
|
||||
|
||||
template<typename FormatContext>
|
||||
auto format(const DB::UUID & uuid, FormatContext & context)
|
||||
{
|
||||
return fmt::format_to(context.out(), "{}", toString(uuid));
|
||||
}
|
||||
};
|
||||
|
@ -90,6 +90,7 @@
|
||||
#include <Interpreters/SynonymsExtensions.h>
|
||||
#include <Interpreters/Lemmatizers.h>
|
||||
#include <Interpreters/ClusterDiscovery.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <filesystem>
|
||||
|
||||
#if USE_ROCKSDB
|
||||
@ -133,6 +134,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INVALID_SETTING_VALUE;
|
||||
extern const int UNKNOWN_READ_METHOD;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
@ -281,6 +283,8 @@ struct ContextSharedPart
|
||||
|
||||
Context::ConfigReloadCallback config_reload_callback;
|
||||
|
||||
bool is_server_completely_started = false;
|
||||
|
||||
#if USE_ROCKSDB
|
||||
/// Global merge tree metadata cache, stored in rocksdb.
|
||||
MergeTreeMetadataCachePtr merge_tree_metadata_cache;
|
||||
@ -365,6 +369,8 @@ struct ContextSharedPart
|
||||
if (common_executor)
|
||||
common_executor->wait();
|
||||
|
||||
TransactionLog::shutdownIfAny();
|
||||
|
||||
std::unique_ptr<SystemLogs> delete_system_logs;
|
||||
std::unique_ptr<EmbeddedDictionaries> delete_embedded_dictionaries;
|
||||
std::unique_ptr<ExternalDictionariesLoader> delete_external_dictionaries_loader;
|
||||
@ -492,6 +498,8 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared)
|
||||
|
||||
void Context::initGlobal()
|
||||
{
|
||||
assert(!global_context_instance);
|
||||
global_context_instance = shared_from_this();
|
||||
DatabaseCatalog::init(shared_from_this());
|
||||
}
|
||||
|
||||
@ -2475,6 +2483,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;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ProcessorsProfileLog> Context::getProcessorsProfileLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -3078,6 +3097,56 @@ void Context::resetZooKeeperMetadataTransaction()
|
||||
metadata_transaction = nullptr;
|
||||
}
|
||||
|
||||
|
||||
void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) const
|
||||
{
|
||||
if (getConfigRef().getInt("allow_experimental_transactions", 0))
|
||||
return;
|
||||
|
||||
if (explicit_tcl_query)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported");
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Experimental support for transactions is disabled, "
|
||||
"however, some query or background task tried to access TransactionLog. "
|
||||
"If you have not enabled this feature explicitly, then it's a bug.");
|
||||
}
|
||||
|
||||
void Context::initCurrentTransaction(MergeTreeTransactionPtr txn)
|
||||
{
|
||||
merge_tree_transaction_holder = MergeTreeTransactionHolder(txn, false, this);
|
||||
setCurrentTransaction(std::move(txn));
|
||||
}
|
||||
|
||||
void Context::setCurrentTransaction(MergeTreeTransactionPtr txn)
|
||||
{
|
||||
assert(!merge_tree_transaction || !txn);
|
||||
assert(this == session_context.lock().get() || this == query_context.lock().get());
|
||||
merge_tree_transaction = std::move(txn);
|
||||
if (!merge_tree_transaction)
|
||||
merge_tree_transaction_holder = {};
|
||||
}
|
||||
|
||||
MergeTreeTransactionPtr Context::getCurrentTransaction() const
|
||||
{
|
||||
return merge_tree_transaction;
|
||||
}
|
||||
|
||||
bool Context::isServerCompletelyStarted() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
assert(getApplicationType() == ApplicationType::SERVER);
|
||||
return shared->is_server_completely_started;
|
||||
}
|
||||
|
||||
void Context::setServerCompletelyStarted()
|
||||
{
|
||||
auto lock = getLock();
|
||||
assert(global_context.lock().get() == this);
|
||||
assert(!shared->is_server_completely_started);
|
||||
assert(getApplicationType() == ApplicationType::SERVER);
|
||||
shared->is_server_completely_started = true;
|
||||
}
|
||||
|
||||
PartUUIDsPtr Context::getPartUUIDs() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
@ -25,6 +26,8 @@
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
|
||||
#include <thread>
|
||||
#include <exception>
|
||||
|
||||
|
||||
@ -80,6 +83,7 @@ class AsynchronousMetricLog;
|
||||
class OpenTelemetrySpanLog;
|
||||
class ZooKeeperLog;
|
||||
class SessionLog;
|
||||
class TransactionsInfoLog;
|
||||
class ProcessorsProfileLog;
|
||||
struct MergeTreeSettings;
|
||||
class StorageS3Settings;
|
||||
@ -313,6 +317,7 @@ private:
|
||||
/// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL).
|
||||
bool is_internal_query = false;
|
||||
|
||||
inline static ContextPtr global_context_instance;
|
||||
|
||||
public:
|
||||
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
|
||||
@ -340,6 +345,11 @@ private:
|
||||
/// thousands of signatures.
|
||||
/// And I hope it will be replaced with more common Transaction sometime.
|
||||
|
||||
MergeTreeTransactionPtr merge_tree_transaction; /// Current transaction context. Can be inside session or query context.
|
||||
/// It's shared with all children contexts.
|
||||
MergeTreeTransactionHolder merge_tree_transaction_holder; /// It will rollback or commit transaction on Context destruction.
|
||||
|
||||
/// Use copy constructor or createGlobal() instead
|
||||
Context();
|
||||
Context(const Context &);
|
||||
Context & operator=(const Context &);
|
||||
@ -634,6 +644,8 @@ public:
|
||||
|
||||
ContextMutablePtr getGlobalContext() const;
|
||||
|
||||
static ContextPtr getGlobalContextInstance() { return global_context_instance; }
|
||||
|
||||
bool hasGlobalContext() const { return !global_context.expired(); }
|
||||
bool isGlobalContext() const
|
||||
{
|
||||
@ -803,6 +815,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;
|
||||
std::shared_ptr<ProcessorsProfileLog> getProcessorsProfileLog() const;
|
||||
|
||||
/// Returns an object used to log operations with parts if it possible.
|
||||
@ -891,6 +904,14 @@ public:
|
||||
/// Removes context of current distributed DDL.
|
||||
void resetZooKeeperMetadataTransaction();
|
||||
|
||||
void checkTransactionsAreAllowed(bool explicit_tcl_query = false) const;
|
||||
void initCurrentTransaction(MergeTreeTransactionPtr txn);
|
||||
void setCurrentTransaction(MergeTreeTransactionPtr txn);
|
||||
MergeTreeTransactionPtr getCurrentTransaction() const;
|
||||
|
||||
bool isServerCompletelyStarted() const;
|
||||
void setServerCompletelyStarted();
|
||||
|
||||
PartUUIDsPtr getPartUUIDs() const;
|
||||
PartUUIDsPtr getIgnoredPartUUIDs() const;
|
||||
|
||||
|
@ -391,12 +391,7 @@ ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_conte
|
||||
|
||||
String DDLTaskBase::getLogEntryName(UInt32 log_entry_number)
|
||||
{
|
||||
/// Sequential counter in ZooKeeper is Int32.
|
||||
assert(log_entry_number < std::numeric_limits<Int32>::max());
|
||||
constexpr size_t seq_node_digits = 10;
|
||||
String number = toString(log_entry_number);
|
||||
String name = "query-" + String(seq_node_digits - number.size(), '0') + number;
|
||||
return name;
|
||||
return zkutil::getSequentialNodeName("query-", log_entry_number);
|
||||
}
|
||||
|
||||
UInt32 DDLTaskBase::getLogEntryNumber(const String & log_entry_name)
|
||||
|
@ -1,9 +1,16 @@
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
void IInterpreter::extendQueryLogElem(
|
||||
QueryLogElement & elem, const ASTPtr & ast, ContextPtr context, const String & query_database, const String & query_table) const
|
||||
{
|
||||
@ -21,4 +28,18 @@ void IInterpreter::extendQueryLogElem(
|
||||
|
||||
extendQueryLogElemImpl(elem, ast, context);
|
||||
}
|
||||
|
||||
void IInterpreter::checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context)
|
||||
{
|
||||
if (!context->getCurrentTransaction())
|
||||
return;
|
||||
|
||||
if (storage->supportsTransactions())
|
||||
return;
|
||||
|
||||
if (context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Storage {} (table {}) does not support transactions",
|
||||
storage->getName(), storage->getStorageID().getNameForLogs());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <QueryPipeline/BlockIO.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages//IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -33,6 +34,13 @@ public:
|
||||
|
||||
virtual void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const {}
|
||||
|
||||
/// Returns true if transactions maybe supported for this type of query.
|
||||
/// If Interpreter returns true, than it is responsible to check that specific query with specific Storage is supported.
|
||||
virtual bool supportsTransactions() const { return false; }
|
||||
|
||||
/// Helper function for some Interpreters.
|
||||
static void checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context);
|
||||
|
||||
virtual ~IInterpreter() = default;
|
||||
};
|
||||
|
||||
|
@ -86,6 +86,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
}
|
||||
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
checkStorageSupportsTransactionsIfNeeded(table, getContext());
|
||||
if (table->isStaticStorage())
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
|
||||
auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
|
||||
|
@ -26,6 +26,8 @@ public:
|
||||
|
||||
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override;
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
AccessRightsElements getRequiredAccess() const;
|
||||
|
||||
|
@ -89,7 +89,7 @@ BlockIO InterpreterDescribeQuery::execute()
|
||||
auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout);
|
||||
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
storage_snapshot = table->getStorageSnapshot(metadata_snapshot);
|
||||
storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext());
|
||||
columns = metadata_snapshot->getColumns();
|
||||
}
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/TableOverrideUtils.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
@ -400,6 +401,23 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
|
||||
override_info.appendTo(buf);
|
||||
break;
|
||||
}
|
||||
case ASTExplainQuery::CurrentTransaction:
|
||||
{
|
||||
if (ast.getSettings())
|
||||
throw Exception("Settings are not supported for EXPLAIN CURRENT TRANSACTION query.", ErrorCodes::UNKNOWN_SETTING);
|
||||
|
||||
if (auto txn = getContext()->getCurrentTransaction())
|
||||
{
|
||||
String dump = txn->dumpDescription();
|
||||
buf.write(dump.data(), dump.size());
|
||||
}
|
||||
else
|
||||
{
|
||||
writeCString("<no current transaction>", buf);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (insert_buf)
|
||||
{
|
||||
|
@ -17,6 +17,8 @@ public:
|
||||
|
||||
static Block getSampleBlock(ASTExplainQuery::ExplainKind kind);
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
ASTPtr query;
|
||||
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Parsers/ASTUseQuery.h>
|
||||
#include <Parsers/ASTWatchQuery.h>
|
||||
#include <Parsers/MySQL/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTTransactionControl.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
|
||||
#include <Parsers/Access/ASTCreateQuotaQuery.h>
|
||||
@ -62,6 +63,7 @@
|
||||
#include <Interpreters/InterpreterSystemQuery.h>
|
||||
#include <Interpreters/InterpreterUseQuery.h>
|
||||
#include <Interpreters/InterpreterWatchQuery.h>
|
||||
#include <Interpreters/InterpreterTransactionControlQuery.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
|
||||
#include <Interpreters/Access/InterpreterCreateQuotaQuery.h>
|
||||
@ -278,6 +280,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
|
||||
{
|
||||
return std::make_unique<InterpreterExternalDDLQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTTransactionControl>())
|
||||
{
|
||||
return std::make_unique<InterpreterTransactionControlQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTCreateFunctionQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterCreateFunctionQuery>(query, context, true /*persist_function*/);
|
||||
|
@ -287,6 +287,8 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
QueryPipelineBuilder pipeline;
|
||||
|
||||
StoragePtr table = getTable(query);
|
||||
checkStorageSupportsTransactionsIfNeeded(table, getContext());
|
||||
|
||||
StoragePtr inner_table;
|
||||
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table.get()))
|
||||
inner_table = mv->getTargetTable();
|
||||
|
@ -46,6 +46,8 @@ public:
|
||||
StoragePtr getTable(ASTInsertQuery & query);
|
||||
Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/CancellationCode.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
@ -358,6 +359,49 @@ BlockIO InterpreterKillQueryQuery::execute()
|
||||
|
||||
break;
|
||||
}
|
||||
case ASTKillQueryQuery::Type::Transaction:
|
||||
{
|
||||
getContext()->checkAccess(AccessType::KILL_TRANSACTION);
|
||||
|
||||
Block transactions_block = getSelectResult("tid, tid_hash, elapsed, is_readonly, state", "system.transactions");
|
||||
|
||||
if (!transactions_block)
|
||||
return res_io;
|
||||
|
||||
const ColumnUInt64 & tid_hash_col = typeid_cast<const ColumnUInt64 &>(*transactions_block.getByName("tid_hash").column);
|
||||
|
||||
auto header = transactions_block.cloneEmpty();
|
||||
header.insert(0, {ColumnString::create(), std::make_shared<DataTypeString>(), "kill_status"});
|
||||
MutableColumns res_columns = header.cloneEmptyColumns();
|
||||
|
||||
for (size_t i = 0; i < transactions_block.rows(); ++i)
|
||||
{
|
||||
UInt64 tid_hash = tid_hash_col.getUInt(i);
|
||||
|
||||
CancellationCode code = CancellationCode::Unknown;
|
||||
if (!query.test)
|
||||
{
|
||||
auto txn = TransactionLog::instance().tryGetRunningTransaction(tid_hash);
|
||||
if (txn)
|
||||
{
|
||||
txn->onException();
|
||||
if (txn->getState() == MergeTreeTransaction::ROLLED_BACK)
|
||||
code = CancellationCode::CancelSent;
|
||||
else
|
||||
code = CancellationCode::CancelCannotBeSent;
|
||||
}
|
||||
else
|
||||
{
|
||||
code = CancellationCode::NotFound;
|
||||
}
|
||||
}
|
||||
|
||||
insertResultRow(i, code, transactions_block, header, res_columns);
|
||||
}
|
||||
|
||||
res_io.pipeline = QueryPipeline(Pipe(std::make_shared<SourceFromSingleChunk>(header.cloneWithColumns(std::move(res_columns)))));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return res_io;
|
||||
|
@ -31,8 +31,9 @@ BlockIO InterpreterOptimizeQuery::execute()
|
||||
|
||||
auto table_id = getContext()->resolveStorageID(ast, Context::ResolveOrdinary);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
checkStorageSupportsTransactionsIfNeeded(table, getContext());
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot);
|
||||
auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext());
|
||||
|
||||
// Empty list of names means we deduplicate by all columns, but user can explicitly state which columns to use.
|
||||
Names column_names;
|
||||
|
@ -18,6 +18,8 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
AccessRightsElements getRequiredAccess() const;
|
||||
|
||||
|
@ -329,12 +329,28 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (!metadata_snapshot)
|
||||
metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
|
||||
storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr);
|
||||
storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context);
|
||||
}
|
||||
|
||||
if (has_input || !joined_tables.resolveTables())
|
||||
joined_tables.makeFakeTable(storage, metadata_snapshot, source_header);
|
||||
|
||||
|
||||
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
{
|
||||
if (storage)
|
||||
checkStorageSupportsTransactionsIfNeeded(storage, context);
|
||||
for (const auto & table : joined_tables.tablesWithColumns())
|
||||
{
|
||||
if (table.table.table.empty())
|
||||
continue;
|
||||
auto maybe_storage = DatabaseCatalog::instance().tryGetTable({table.table.database, table.table.table}, context);
|
||||
if (!maybe_storage)
|
||||
continue;
|
||||
checkStorageSupportsTransactionsIfNeeded(storage, context);
|
||||
}
|
||||
}
|
||||
|
||||
/// Rewrite JOINs
|
||||
if (!has_input && joined_tables.tablesCount() > 1)
|
||||
{
|
||||
@ -1791,7 +1807,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
const auto & func = desc.function;
|
||||
std::optional<UInt64> num_rows{};
|
||||
|
||||
if (!query.prewhere() && !query.where())
|
||||
if (!query.prewhere() && !query.where() && !context->getCurrentTransaction())
|
||||
{
|
||||
num_rows = storage->totalRows(settings);
|
||||
}
|
||||
|
@ -104,6 +104,8 @@ public:
|
||||
|
||||
Names getRequiredColumns() { return required_columns; }
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
InterpreterSelectQuery(
|
||||
const ASTPtr & query_ptr_,
|
||||
|
@ -39,6 +39,8 @@ public:
|
||||
|
||||
virtual void ignoreWithTotals() override;
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
std::vector<std::unique_ptr<IInterpreterUnionOrSelectQuery>> nested_interpreters;
|
||||
|
||||
|
@ -25,6 +25,8 @@ public:
|
||||
*/
|
||||
void executeForCurrentContext();
|
||||
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
@ -462,6 +463,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
[&] { 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 transactions_info_log = getContext()->getTransactionsInfoLog()) transactions_info_log->flush(true); },
|
||||
[&] { if (auto processors_profile_log = getContext()->getProcessorsProfileLog()) processors_profile_log->flush(true); }
|
||||
);
|
||||
break;
|
||||
|
90
src/Interpreters/InterpreterTransactionControlQuery.cpp
Normal file
90
src/Interpreters/InterpreterTransactionControlQuery.cpp
Normal file
@ -0,0 +1,90 @@
|
||||
#include <Interpreters/InterpreterTransactionControlQuery.h>
|
||||
#include <Parsers/ASTTransactionControl.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INVALID_TRANSACTION;
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::execute()
|
||||
{
|
||||
if (!query_context->hasSessionContext())
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction Control Language queries are allowed only inside session");
|
||||
|
||||
ContextMutablePtr session_context = query_context->getSessionContext();
|
||||
const auto & tcl = query_ptr->as<const ASTTransactionControl &>();
|
||||
|
||||
switch (tcl.action)
|
||||
{
|
||||
case ASTTransactionControl::BEGIN:
|
||||
return executeBegin(session_context);
|
||||
case ASTTransactionControl::COMMIT:
|
||||
return executeCommit(session_context);
|
||||
case ASTTransactionControl::ROLLBACK:
|
||||
return executeRollback(session_context);
|
||||
case ASTTransactionControl::SET_SNAPSHOT:
|
||||
return executeSetSnapshot(session_context, tcl.snapshot);
|
||||
}
|
||||
assert(false);
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context)
|
||||
{
|
||||
if (session_context->getCurrentTransaction())
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Nested transactions are not supported");
|
||||
|
||||
session_context->checkTransactionsAreAllowed(/* explicit_tcl_query = */ true);
|
||||
auto txn = TransactionLog::instance().beginTransaction();
|
||||
session_context->initCurrentTransaction(txn);
|
||||
query_context->setCurrentTransaction(txn);
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::executeCommit(ContextMutablePtr session_context)
|
||||
{
|
||||
auto txn = session_context->getCurrentTransaction();
|
||||
if (!txn)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "There is no current transaction");
|
||||
if (txn->getState() != MergeTreeTransaction::RUNNING)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction is not in RUNNING state");
|
||||
|
||||
TransactionLog::instance().commitTransaction(txn);
|
||||
session_context->setCurrentTransaction(NO_TRANSACTION_PTR);
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::executeRollback(ContextMutablePtr session_context)
|
||||
{
|
||||
auto txn = session_context->getCurrentTransaction();
|
||||
if (!txn)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "There is no current transaction");
|
||||
if (txn->getState() == MergeTreeTransaction::COMMITTED)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Transaction is in COMMITTED state");
|
||||
|
||||
if (txn->getState() == MergeTreeTransaction::RUNNING)
|
||||
TransactionLog::instance().rollbackTransaction(txn);
|
||||
session_context->setCurrentTransaction(NO_TRANSACTION_PTR);
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::executeSetSnapshot(ContextMutablePtr session_context, UInt64 snapshot)
|
||||
{
|
||||
auto txn = session_context->getCurrentTransaction();
|
||||
if (!txn)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "There is no current transaction");
|
||||
|
||||
if (snapshot <= Tx::MaxReservedCSN && snapshot != Tx::PrehistoricCSN && snapshot != Tx::EverythingVisibleCSN)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Cannot set snapshot to reserved CSN");
|
||||
|
||||
txn->setSnapshot(snapshot);
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
34
src/Interpreters/InterpreterTransactionControlQuery.h
Normal file
34
src/Interpreters/InterpreterTransactionControlQuery.h
Normal file
@ -0,0 +1,34 @@
|
||||
#pragma once
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterTransactionControlQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterTransactionControlQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: query_context(context_)
|
||||
, query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
bool ignoreQuota() const override { return true; }
|
||||
bool ignoreLimits() const override { return true; }
|
||||
bool supportsTransactions() const override { return true; }
|
||||
|
||||
private:
|
||||
BlockIO executeBegin(ContextMutablePtr session_context);
|
||||
static BlockIO executeCommit(ContextMutablePtr session_context);
|
||||
static BlockIO executeRollback(ContextMutablePtr session_context);
|
||||
static BlockIO executeSetSnapshot(ContextMutablePtr session_context, UInt64 snapshot);
|
||||
|
||||
private:
|
||||
ContextMutablePtr query_context;
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
||||
}
|
351
src/Interpreters/MergeTreeTransaction.cpp
Normal file
351
src/Interpreters/MergeTreeTransaction.cpp
Normal file
@ -0,0 +1,351 @@
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_TRANSACTION;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static TableLockHolder getLockForOrdinary(const StoragePtr & storage)
|
||||
{
|
||||
if (storage->getStorageID().uuid != UUIDHelpers::Nil)
|
||||
return {};
|
||||
|
||||
/// Maybe we should just throw an exception and do not support Ordinary database?
|
||||
auto default_timeout = std::chrono::milliseconds(10 * 1000);
|
||||
return storage->lockForShare(RWLockImpl::NO_QUERY, default_timeout);
|
||||
}
|
||||
|
||||
MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id)
|
||||
: tid({snapshot_, local_tid_, host_id})
|
||||
, snapshot(snapshot_)
|
||||
, csn(Tx::UnknownCSN)
|
||||
{
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::setSnapshot(CSN new_snapshot)
|
||||
{
|
||||
snapshot = new_snapshot;
|
||||
}
|
||||
|
||||
MergeTreeTransaction::State MergeTreeTransaction::getState() const
|
||||
{
|
||||
CSN c = csn.load();
|
||||
if (c == Tx::UnknownCSN || c == Tx::CommittingCSN)
|
||||
return RUNNING;
|
||||
if (c == Tx::RolledBackCSN)
|
||||
return ROLLED_BACK;
|
||||
return COMMITTED;
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::checkIsNotCancelled() const
|
||||
{
|
||||
CSN c = csn.load();
|
||||
if (c == Tx::RolledBackCSN)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled");
|
||||
else if (c != Tx::UnknownCSN)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", c);
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part, MergeTreeTransaction * txn)
|
||||
{
|
||||
/// Creation TID was written to data part earlier on part creation.
|
||||
/// We only need to ensure that it's written and add part to in-memory set of new parts.
|
||||
new_part->assertHasVersionMetadata(txn);
|
||||
if (txn)
|
||||
{
|
||||
txn->addNewPart(storage, new_part);
|
||||
/// Now we know actual part name and can write it to system log table.
|
||||
tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, txn->tid, TransactionInfoContext{storage->getStorageID(), new_part->name});
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn)
|
||||
{
|
||||
TransactionInfoContext transaction_context{storage->getStorageID(), part_to_remove->name};
|
||||
if (txn)
|
||||
{
|
||||
/// Lock part for removal and write current TID into version metadata file.
|
||||
/// If server crash just after committing transactions
|
||||
/// we will find this TID in version metadata and will finally remove part.
|
||||
txn->removeOldPart(storage, part_to_remove, transaction_context);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Lock part for removal with special TID, so transactions will not try to remove it concurrently.
|
||||
/// We lock it only in memory if part was not involved in any transactions.
|
||||
part_to_remove->version.lockRemovalTID(Tx::PrehistoricTID, transaction_context);
|
||||
if (part_to_remove->wasInvolvedInTransaction())
|
||||
part_to_remove->appendRemovalTIDToVersionMetadata();
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn)
|
||||
{
|
||||
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
|
||||
TransactionInfoContext transaction_context{storage->getStorageID(), new_part->name};
|
||||
tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, transaction_context);
|
||||
transaction_context.covering_part = std::move(transaction_context.part_name);
|
||||
new_part->assertHasVersionMetadata(txn);
|
||||
|
||||
if (txn)
|
||||
{
|
||||
txn->addNewPart(storage, new_part);
|
||||
for (const auto & covered : covered_parts)
|
||||
{
|
||||
transaction_context.part_name = covered->name;
|
||||
txn->removeOldPart(storage, covered, transaction_context);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & covered : covered_parts)
|
||||
{
|
||||
transaction_context.part_name = covered->name;
|
||||
covered->version.lockRemovalTID(tid, transaction_context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part)
|
||||
{
|
||||
auto maybe_lock = getLockForOrdinary(storage);
|
||||
std::lock_guard lock{mutex};
|
||||
checkIsNotCancelled();
|
||||
storages.insert(storage);
|
||||
if (maybe_lock)
|
||||
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
|
||||
creating_parts.push_back(new_part);
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context)
|
||||
{
|
||||
auto maybe_lock = getLockForOrdinary(storage);
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
checkIsNotCancelled();
|
||||
|
||||
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
|
||||
part_to_remove->version.lockRemovalTID(tid, context);
|
||||
storages.insert(storage);
|
||||
if (maybe_lock)
|
||||
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
|
||||
removing_parts.push_back(part_to_remove);
|
||||
}
|
||||
|
||||
part_to_remove->appendRemovalTIDToVersionMetadata();
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id)
|
||||
{
|
||||
auto maybe_lock = getLockForOrdinary(table);
|
||||
std::lock_guard lock{mutex};
|
||||
checkIsNotCancelled();
|
||||
storages.insert(table);
|
||||
if (maybe_lock)
|
||||
table_read_locks_for_ordinary_db.emplace_back(std::move(maybe_lock));
|
||||
mutations.emplace_back(table, mutation_id);
|
||||
}
|
||||
|
||||
bool MergeTreeTransaction::isReadOnly() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
assert((creating_parts.empty() && removing_parts.empty() && mutations.empty()) == storages.empty());
|
||||
return storages.empty();
|
||||
}
|
||||
|
||||
scope_guard MergeTreeTransaction::beforeCommit()
|
||||
{
|
||||
RunningMutationsList mutations_to_wait;
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
mutations_to_wait = mutations;
|
||||
}
|
||||
|
||||
/// We should wait for mutations to finish before committing transaction, because some mutation may fail and cause rollback.
|
||||
for (const auto & table_and_mutation : mutations_to_wait)
|
||||
table_and_mutation.first->waitForMutation(table_and_mutation.second);
|
||||
|
||||
assert([&]()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return mutations == mutations_to_wait;
|
||||
}());
|
||||
|
||||
CSN expected = Tx::UnknownCSN;
|
||||
bool can_commit = csn.compare_exchange_strong(expected, Tx::CommittingCSN);
|
||||
if (!can_commit)
|
||||
{
|
||||
/// Transaction was concurrently cancelled by KILL TRANSACTION or KILL MUTATION
|
||||
if (expected == Tx::RolledBackCSN)
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", expected);
|
||||
}
|
||||
|
||||
/// We should set CSN back to Unknown if we will fail to commit transaction for some reason (connection loss, etc)
|
||||
return [this]()
|
||||
{
|
||||
CSN expected_value = Tx::CommittingCSN;
|
||||
csn.compare_exchange_strong(expected_value, Tx::UnknownCSN);
|
||||
};
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept
|
||||
{
|
||||
/// Write allocated CSN into version metadata, so we will know CSN without reading it from transaction log
|
||||
/// and we will be able to remove old entries from transaction log in ZK.
|
||||
/// It's not a problem if server crash before CSN is written, because we already have TID in data part and entry in the log.
|
||||
[[maybe_unused]] CSN prev_value = csn.exchange(assigned_csn);
|
||||
assert(prev_value == Tx::CommittingCSN);
|
||||
for (const auto & part : creating_parts)
|
||||
{
|
||||
part->version.creation_csn.store(csn);
|
||||
part->appendCSNToVersionMetadata(VersionMetadata::WhichCSN::CREATION);
|
||||
}
|
||||
|
||||
for (const auto & part : removing_parts)
|
||||
{
|
||||
part->version.removal_csn.store(csn);
|
||||
part->appendCSNToVersionMetadata(VersionMetadata::WhichCSN::REMOVAL);
|
||||
}
|
||||
|
||||
for (const auto & storage_and_mutation : mutations)
|
||||
storage_and_mutation.first->setMutationCSN(storage_and_mutation.second, csn);
|
||||
}
|
||||
|
||||
bool MergeTreeTransaction::rollback() noexcept
|
||||
{
|
||||
CSN expected = Tx::UnknownCSN;
|
||||
bool need_rollback = csn.compare_exchange_strong(expected, Tx::RolledBackCSN);
|
||||
|
||||
/// Check that it was not rolled back concurrently
|
||||
if (!need_rollback)
|
||||
return false;
|
||||
|
||||
/// It's not a problem if server crash at this point
|
||||
/// because on startup we will see that TID is not committed and will simply discard these changes.
|
||||
|
||||
RunningMutationsList mutations_to_kill;
|
||||
DataPartsVector parts_to_remove;
|
||||
DataPartsVector parts_to_activate;
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
mutations_to_kill = mutations;
|
||||
parts_to_remove = creating_parts;
|
||||
parts_to_activate = removing_parts;
|
||||
}
|
||||
|
||||
/// Forcefully stop related mutations if any
|
||||
for (const auto & table_and_mutation : mutations_to_kill)
|
||||
table_and_mutation.first->killMutation(table_and_mutation.second);
|
||||
|
||||
/// Discard changes in active parts set
|
||||
/// Remove parts that were created, restore parts that were removed (except parts that were created by this transaction too)
|
||||
for (const auto & part : parts_to_remove)
|
||||
{
|
||||
if (part->version.isRemovalTIDLocked())
|
||||
{
|
||||
/// Don't need to remove part from working set if it was created and removed by this transaction
|
||||
assert(part->version.removal_tid_lock == tid.getHash());
|
||||
continue;
|
||||
}
|
||||
/// FIXME do not lock removal_tid when rolling back part creation, it's ugly
|
||||
const_cast<MergeTreeData &>(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true);
|
||||
}
|
||||
|
||||
for (const auto & part : parts_to_activate)
|
||||
if (part->version.getCreationTID() != tid)
|
||||
const_cast<MergeTreeData &>(part->storage).restoreAndActivatePart(part);
|
||||
|
||||
/// Kind of optimization: cleanup thread can remove these parts immediately
|
||||
for (const auto & part : parts_to_remove)
|
||||
{
|
||||
part->version.creation_csn.store(Tx::RolledBackCSN);
|
||||
/// Write special RolledBackCSN, so we will be able to cleanup transaction log
|
||||
part->appendCSNToVersionMetadata(VersionMetadata::CREATION);
|
||||
}
|
||||
|
||||
for (const auto & part : parts_to_activate)
|
||||
{
|
||||
/// Clear removal_tid from version metadata file, so we will not need to distinguish TIDs that were not committed
|
||||
/// and TIDs that were committed long time ago and were removed from the log on log cleanup.
|
||||
part->appendRemovalTIDToVersionMetadata(/* clear */ true);
|
||||
part->version.unlockRemovalTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name});
|
||||
}
|
||||
|
||||
|
||||
assert([&]()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
assert(mutations_to_kill == mutations);
|
||||
assert(parts_to_remove == creating_parts);
|
||||
assert(parts_to_activate == removing_parts);
|
||||
return csn == Tx::RolledBackCSN;
|
||||
}());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void MergeTreeTransaction::onException()
|
||||
{
|
||||
TransactionLog::instance().rollbackTransaction(shared_from_this());
|
||||
}
|
||||
|
||||
String MergeTreeTransaction::dumpDescription() const
|
||||
{
|
||||
String res = fmt::format("{} state: {}, snapshot: {}", tid, getState(), snapshot);
|
||||
|
||||
if (isReadOnly())
|
||||
{
|
||||
res += ", readonly";
|
||||
return res;
|
||||
}
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
|
||||
res += fmt::format(", affects {} tables:", storages.size());
|
||||
|
||||
using ChangesInTable = std::tuple<Strings, Strings, Strings>;
|
||||
std::unordered_map<const IStorage *, ChangesInTable> storage_to_changes;
|
||||
|
||||
for (const auto & part : creating_parts)
|
||||
std::get<0>(storage_to_changes[&(part->storage)]).push_back(part->name);
|
||||
|
||||
for (const auto & part : removing_parts)
|
||||
{
|
||||
String info = fmt::format("{} (created by {}, {})", part->name, part->version.getCreationTID(), part->version.creation_csn);
|
||||
std::get<1>(storage_to_changes[&(part->storage)]).push_back(std::move(info));
|
||||
assert(!part->version.creation_csn || part->version.creation_csn <= snapshot);
|
||||
}
|
||||
|
||||
for (const auto & mutation : mutations)
|
||||
std::get<2>(storage_to_changes[mutation.first.get()]).push_back(mutation.second);
|
||||
|
||||
for (const auto & storage_changes : storage_to_changes)
|
||||
{
|
||||
res += fmt::format("\n\t{}:", storage_changes.first->getStorageID().getNameForLogs());
|
||||
const auto & creating_info = std::get<0>(storage_changes.second);
|
||||
const auto & removing_info = std::get<1>(storage_changes.second);
|
||||
const auto & mutations_info = std::get<2>(storage_changes.second);
|
||||
|
||||
if (!creating_info.empty())
|
||||
res += fmt::format("\n\t\tcreating parts:\n\t\t\t{}", fmt::join(creating_info, "\n\t\t\t"));
|
||||
if (!removing_info.empty())
|
||||
res += fmt::format("\n\t\tremoving parts:\n\t\t\t{}", fmt::join(removing_info, "\n\t\t\t"));
|
||||
if (!mutations_info.empty())
|
||||
res += fmt::format("\n\t\tmutations:\n\t\t\t{}", fmt::join(mutations_info, "\n\t\t\t"));
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
84
src/Interpreters/MergeTreeTransaction.h
Normal file
84
src/Interpreters/MergeTreeTransaction.h
Normal file
@ -0,0 +1,84 @@
|
||||
#pragma once
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <list>
|
||||
#include <unordered_set>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
||||
using DataPartsVector = std::vector<DataPartPtr>;
|
||||
|
||||
/// This object is responsible for tracking all changes that some transaction is making in MergeTree tables.
|
||||
/// It collects all changes that queries of current transaction made in data part sets of all MergeTree tables
|
||||
/// to ether make them visible when transaction commits or undo when transaction rolls back.
|
||||
class MergeTreeTransaction : public std::enable_shared_from_this<MergeTreeTransaction>, private boost::noncopyable
|
||||
{
|
||||
friend class TransactionLog;
|
||||
public:
|
||||
enum State
|
||||
{
|
||||
RUNNING,
|
||||
COMMITTED,
|
||||
ROLLED_BACK,
|
||||
};
|
||||
|
||||
CSN getSnapshot() const { return snapshot; }
|
||||
void setSnapshot(CSN new_snapshot);
|
||||
State getState() const;
|
||||
|
||||
const TransactionID tid;
|
||||
|
||||
MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id);
|
||||
|
||||
void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part);
|
||||
void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context);
|
||||
|
||||
void addMutation(const StoragePtr & table, const String & mutation_id);
|
||||
|
||||
static void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part, MergeTreeTransaction * txn);
|
||||
static void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn);
|
||||
static void addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn);
|
||||
|
||||
bool isReadOnly() const;
|
||||
|
||||
void onException();
|
||||
|
||||
String dumpDescription() const;
|
||||
|
||||
Float64 elapsedSeconds() const { return elapsed.elapsedSeconds(); }
|
||||
|
||||
private:
|
||||
scope_guard beforeCommit();
|
||||
void afterCommit(CSN assigned_csn) noexcept;
|
||||
bool rollback() noexcept;
|
||||
void checkIsNotCancelled() const;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Stopwatch elapsed;
|
||||
|
||||
/// Usually it's equal to tid.start_csn, but can be changed by SET SNAPSHOT query (for introspection purposes and time-traveling)
|
||||
CSN snapshot;
|
||||
std::list<CSN>::iterator snapshot_in_use_it;
|
||||
|
||||
/// Lists of changes made by transaction
|
||||
std::unordered_set<StoragePtr> storages;
|
||||
std::vector<TableLockHolder> table_read_locks_for_ordinary_db;
|
||||
DataPartsVector creating_parts;
|
||||
DataPartsVector removing_parts;
|
||||
using RunningMutationsList = std::vector<std::pair<StoragePtr, String>>;
|
||||
RunningMutationsList mutations;
|
||||
|
||||
std::atomic<CSN> csn;
|
||||
};
|
||||
|
||||
using MergeTreeTransactionPtr = std::shared_ptr<MergeTreeTransaction>;
|
||||
|
||||
}
|
84
src/Interpreters/MergeTreeTransactionHolder.cpp
Normal file
84
src/Interpreters/MergeTreeTransactionHolder.cpp
Normal file
@ -0,0 +1,84 @@
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_ = false, const Context * owned_by_session_context_)
|
||||
: txn(txn_)
|
||||
, autocommit(autocommit_)
|
||||
, owned_by_session_context(owned_by_session_context_)
|
||||
{
|
||||
assert(!txn || txn->getState() == MergeTreeTransaction::RUNNING);
|
||||
assert(!owned_by_session_context || owned_by_session_context == owned_by_session_context->getSessionContext().get());
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder::MergeTreeTransactionHolder(MergeTreeTransactionHolder && rhs) noexcept
|
||||
{
|
||||
*this = std::move(rhs);
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(MergeTreeTransactionHolder && rhs) noexcept
|
||||
{
|
||||
onDestroy();
|
||||
txn = NO_TRANSACTION_PTR;
|
||||
autocommit = false;
|
||||
owned_by_session_context = nullptr;
|
||||
std::swap(txn, rhs.txn);
|
||||
std::swap(autocommit, rhs.autocommit);
|
||||
std::swap(owned_by_session_context, rhs.owned_by_session_context);
|
||||
return *this;
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder::~MergeTreeTransactionHolder()
|
||||
{
|
||||
onDestroy();
|
||||
}
|
||||
|
||||
void MergeTreeTransactionHolder::onDestroy() noexcept
|
||||
{
|
||||
if (!txn)
|
||||
return;
|
||||
if (txn->getState() != MergeTreeTransaction::RUNNING)
|
||||
return;
|
||||
|
||||
if (autocommit && std::uncaught_exceptions() == 0)
|
||||
{
|
||||
try
|
||||
{
|
||||
TransactionLog::instance().commitTransaction(txn);
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
TransactionLog::instance().rollbackTransaction(txn);
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionHolder & rhs)
|
||||
{
|
||||
*this = rhs;
|
||||
}
|
||||
|
||||
MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder & rhs) // NOLINT
|
||||
{
|
||||
if (rhs.txn && !rhs.owned_by_session_context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Tried to copy non-empty MergeTreeTransactionHolder that is not owned by session context. It's a bug");
|
||||
assert(!txn);
|
||||
assert(!autocommit);
|
||||
assert(!owned_by_session_context);
|
||||
return *this;
|
||||
}
|
||||
|
||||
}
|
42
src/Interpreters/MergeTreeTransactionHolder.h
Normal file
42
src/Interpreters/MergeTreeTransactionHolder.h
Normal file
@ -0,0 +1,42 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
class MergeTreeTransaction;
|
||||
/// TODO maybe replace with raw pointer? It should not be shared, only MergeTreeTransactionHolder can own a transaction object
|
||||
using MergeTreeTransactionPtr = std::shared_ptr<MergeTreeTransaction>;
|
||||
|
||||
/// Owns a MergeTreeTransactionObject.
|
||||
/// Rolls back a transaction in dtor if it was not committed.
|
||||
/// If `autocommit` flag is true, then it commits transaction if dtor is called normally
|
||||
/// or rolls it back if dtor was called due to an exception.
|
||||
class MergeTreeTransactionHolder
|
||||
{
|
||||
public:
|
||||
MergeTreeTransactionHolder() = default;
|
||||
MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_, const Context * owned_by_session_context_ = nullptr);
|
||||
MergeTreeTransactionHolder(MergeTreeTransactionHolder && rhs) noexcept;
|
||||
MergeTreeTransactionHolder & operator=(MergeTreeTransactionHolder && rhs) noexcept;
|
||||
~MergeTreeTransactionHolder();
|
||||
|
||||
/// NOTE: We cannot make it noncopyable, because we use it as a field of Context.
|
||||
/// So the following copy constructor and operator does not copy anything,
|
||||
/// they just leave txn nullptr.
|
||||
MergeTreeTransactionHolder(const MergeTreeTransactionHolder & rhs);
|
||||
MergeTreeTransactionHolder & operator=(const MergeTreeTransactionHolder & rhs);
|
||||
|
||||
MergeTreeTransactionPtr getTransaction() const { return txn; }
|
||||
|
||||
private:
|
||||
void onDestroy() noexcept;
|
||||
|
||||
MergeTreeTransactionPtr txn;
|
||||
bool autocommit = false;
|
||||
const Context * owned_by_session_context = nullptr;
|
||||
};
|
||||
|
||||
}
|
@ -802,7 +802,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &
|
||||
/// e.g. ALTER referencing the same table in scalar subquery
|
||||
bool execute_scalar_subqueries = !dry_run;
|
||||
auto syntax_result = TreeRewriter(context).analyze(
|
||||
all_asts, all_columns, storage, storage->getStorageSnapshot(metadata_snapshot),
|
||||
all_asts, all_columns, storage, storage->getStorageSnapshot(metadata_snapshot, context),
|
||||
false, true, execute_scalar_subqueries);
|
||||
|
||||
if (execute_scalar_subqueries && context->hasQueryContext())
|
||||
|
@ -117,7 +117,9 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes()
|
||||
{"used_formats", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"used_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"used_storages", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"used_table_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}
|
||||
{"used_table_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
|
||||
{"transaction_id", getTransactionIDDataType()},
|
||||
};
|
||||
|
||||
}
|
||||
@ -257,6 +259,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
|
||||
fill_column(used_storages, column_storage_factory_objects);
|
||||
fill_column(used_table_functions, column_table_function_factory_objects);
|
||||
}
|
||||
|
||||
columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id});
|
||||
}
|
||||
|
||||
void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i)
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -85,6 +86,8 @@ struct QueryLogElement
|
||||
std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters;
|
||||
std::shared_ptr<Settings> query_settings;
|
||||
|
||||
TransactionID tid;
|
||||
|
||||
static std::string name() { return "QueryLog"; }
|
||||
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
@ -202,6 +203,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");
|
||||
processors_profile_log = createSystemLog<ProcessorsProfileLog>(global_context, "system", "processors_profile_log", config, "processors_profile_log");
|
||||
|
||||
if (query_log)
|
||||
@ -228,6 +231,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());
|
||||
if (processors_profile_log)
|
||||
logs.emplace_back(processors_profile_log.get());
|
||||
|
||||
@ -544,6 +549,7 @@ ASTPtr SystemLog<LogElement>::getCreateTableQuery()
|
||||
return create;
|
||||
}
|
||||
|
||||
|
||||
#define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog<ELEMENT>;
|
||||
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG)
|
||||
|
||||
|
@ -43,6 +43,7 @@ class OpenTelemetrySpanLog;
|
||||
class QueryViewsLog;
|
||||
class ZooKeeperLog;
|
||||
class SessionLog;
|
||||
class TransactionsInfoLog;
|
||||
class ProcessorsProfileLog;
|
||||
|
||||
/// System logs should be destroyed in destructor of the last Context and before tables,
|
||||
@ -71,6 +72,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;
|
||||
/// Used to log processors profiling
|
||||
std::shared_ptr<ProcessorsProfileLog> processors_profile_log;
|
||||
|
||||
|
484
src/Interpreters/TransactionLog.cpp
Normal file
484
src/Interpreters/TransactionLog.cpp
Normal file
@ -0,0 +1,484 @@
|
||||
#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>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
|
||||
/// It's used in critical places to exit on unexpected exceptions.
|
||||
/// SIGABRT is usually better that broken state in memory with unpredictable consequences.
|
||||
#define NOEXCEPT_SCOPE SCOPE_EXIT({ if (std::uncaught_exceptions()) { tryLogCurrentException("NOEXCEPT_SCOPE"); abort(); } })
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
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()
|
||||
: log(&Poco::Logger::get("TransactionLog"))
|
||||
{
|
||||
global_context = Context::getGlobalContextInstance();
|
||||
global_context->checkTransactionsAreAllowed();
|
||||
|
||||
zookeeper_path = global_context->getConfigRef().getString("transaction_log.zookeeper_path", "/clickhouse/txn");
|
||||
zookeeper_path_log = zookeeper_path + "/log";
|
||||
|
||||
loadLogFromZooKeeper();
|
||||
|
||||
updating_thread = ThreadFromGlobalPool(&TransactionLog::runUpdatingThread, this);
|
||||
}
|
||||
|
||||
TransactionLog::~TransactionLog()
|
||||
{
|
||||
shutdown();
|
||||
}
|
||||
|
||||
void TransactionLog::shutdown()
|
||||
{
|
||||
if (stop_flag.exchange(true))
|
||||
return;
|
||||
log_updated_event->set();
|
||||
latest_snapshot.notify_all();
|
||||
updating_thread.join();
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
/// This is required to... you'll never guess - avoid race condition inside Poco::Logger (Coordination::ZooKeeper::log)
|
||||
zookeeper.reset();
|
||||
}
|
||||
|
||||
ZooKeeperPtr TransactionLog::getZooKeeper() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return zookeeper;
|
||||
}
|
||||
|
||||
UInt64 TransactionLog::deserializeCSN(const String & csn_node_name)
|
||||
{
|
||||
ReadBufferFromString buf{csn_node_name};
|
||||
assertString("csn-", buf);
|
||||
UInt64 res;
|
||||
readText(res, buf);
|
||||
assertEOF(buf);
|
||||
return res;
|
||||
}
|
||||
|
||||
String TransactionLog::serializeCSN(CSN csn)
|
||||
{
|
||||
return zkutil::getSequentialNodeName("csn-", csn);
|
||||
}
|
||||
|
||||
TransactionID TransactionLog::deserializeTID(const String & csn_node_content)
|
||||
{
|
||||
TransactionID tid = Tx::EmptyTID;
|
||||
if (csn_node_content.empty())
|
||||
return tid;
|
||||
|
||||
ReadBufferFromString buf{csn_node_content};
|
||||
tid = TransactionID::read(buf);
|
||||
assertEOF(buf);
|
||||
return tid;
|
||||
}
|
||||
|
||||
String TransactionLog::serializeTID(const TransactionID & tid)
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
TransactionID::write(tid, buf);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
|
||||
void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_iterator end)
|
||||
{
|
||||
std::vector<std::future<Coordination::GetResponse>> futures;
|
||||
size_t entries_count = std::distance(beg, end);
|
||||
if (!entries_count)
|
||||
return;
|
||||
|
||||
String last_entry = *std::prev(end);
|
||||
LOG_TRACE(log, "Loading {} entries from {}: {}..{}", entries_count, zookeeper_path_log, *beg, last_entry);
|
||||
futures.reserve(entries_count);
|
||||
for (auto it = beg; it != end; ++it)
|
||||
futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path_log) / *it));
|
||||
|
||||
std::vector<std::pair<TIDHash, CSNEntry>> loaded;
|
||||
loaded.reserve(entries_count);
|
||||
auto it = beg;
|
||||
for (size_t i = 0; i < entries_count; ++i, ++it)
|
||||
{
|
||||
auto res = futures[i].get();
|
||||
CSN csn = deserializeCSN(*it);
|
||||
TransactionID tid = deserializeTID(res.data);
|
||||
loaded.emplace_back(tid.getHash(), CSNEntry{csn, tid});
|
||||
LOG_TEST(log, "Got entry {} -> {}", tid, csn);
|
||||
}
|
||||
futures.clear();
|
||||
|
||||
NOEXCEPT_SCOPE;
|
||||
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
|
||||
std::lock_guard lock{mutex};
|
||||
for (const auto & entry : loaded)
|
||||
{
|
||||
if (entry.first == Tx::EmptyTID.getHash())
|
||||
continue;
|
||||
|
||||
tid_to_csn.emplace(entry.first, entry.second);
|
||||
}
|
||||
last_loaded_entry = last_entry;
|
||||
latest_snapshot = loaded.back().second.csn;
|
||||
local_tid_counter = Tx::MaxReservedLocalTID;
|
||||
}
|
||||
|
||||
void TransactionLog::loadLogFromZooKeeper()
|
||||
{
|
||||
assert(!zookeeper);
|
||||
assert(tid_to_csn.empty());
|
||||
assert(last_loaded_entry.empty());
|
||||
zookeeper = global_context->getZooKeeper();
|
||||
|
||||
/// We do not write local_tid_counter to disk or zk and maintain it only in memory.
|
||||
/// Create empty entry to allocate new CSN to safely start counting from the beginning and avoid TID duplication.
|
||||
/// TODO It's possible to skip this step in come cases (especially for multi-host configuration).
|
||||
Coordination::Error code = zookeeper->tryCreate(zookeeper_path_log + "/csn-", "", zkutil::CreateMode::PersistentSequential);
|
||||
if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// Log probably does not exist, create it
|
||||
assert(code == Coordination::Error::ZNONODE);
|
||||
zookeeper->createAncestors(zookeeper_path_log);
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/tail_ptr", serializeCSN(Tx::MaxReservedCSN), zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path_log, "", zkutil::CreateMode::Persistent));
|
||||
|
||||
/// Fast-forward sequential counter to skip reserved CSNs
|
||||
for (size_t i = 0; i <= Tx::MaxReservedCSN; ++i)
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path_log + "/csn-", "", zkutil::CreateMode::PersistentSequential));
|
||||
Coordination::Responses res;
|
||||
code = zookeeper->tryMulti(ops, res);
|
||||
if (code != Coordination::Error::ZNODEEXISTS)
|
||||
zkutil::KeeperMultiException::check(code, ops, res);
|
||||
}
|
||||
|
||||
/// TODO Split log into "subdirectories" to:
|
||||
/// 1. fetch it more optimal way (avoid listing all CSNs on further incremental updates)
|
||||
/// 2. simplify log rotation
|
||||
/// 3. support 64-bit CSNs on top of Apache ZooKeeper (it uses Int32 for sequential numbers)
|
||||
Strings entries_list = zookeeper->getChildren(zookeeper_path_log, nullptr, log_updated_event);
|
||||
assert(!entries_list.empty());
|
||||
std::sort(entries_list.begin(), entries_list.end());
|
||||
loadEntries(entries_list.begin(), entries_list.end());
|
||||
assert(!last_loaded_entry.empty());
|
||||
assert(latest_snapshot == deserializeCSN(last_loaded_entry));
|
||||
local_tid_counter = Tx::MaxReservedLocalTID;
|
||||
|
||||
tail_ptr = deserializeCSN(zookeeper->get(zookeeper_path + "/tail_ptr"));
|
||||
}
|
||||
|
||||
void TransactionLog::runUpdatingThread()
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
try
|
||||
{
|
||||
log_updated_event->wait();
|
||||
if (stop_flag.load())
|
||||
return;
|
||||
|
||||
if (!zookeeper)
|
||||
{
|
||||
auto new_zookeeper = global_context->getZooKeeper();
|
||||
std::lock_guard lock{mutex};
|
||||
zookeeper = new_zookeeper;
|
||||
}
|
||||
|
||||
loadNewEntries();
|
||||
removeOldEntries();
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
/// TODO better backoff
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
|
||||
if (Coordination::isHardwareError(e.code))
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
zookeeper.reset();
|
||||
}
|
||||
log_updated_event->set();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
|
||||
log_updated_event->set();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void TransactionLog::loadNewEntries()
|
||||
{
|
||||
Strings entries_list = zookeeper->getChildren(zookeeper_path_log, nullptr, log_updated_event);
|
||||
assert(!entries_list.empty());
|
||||
std::sort(entries_list.begin(), entries_list.end());
|
||||
auto it = std::upper_bound(entries_list.begin(), entries_list.end(), last_loaded_entry);
|
||||
loadEntries(it, entries_list.end());
|
||||
assert(last_loaded_entry == entries_list.back());
|
||||
assert(latest_snapshot == deserializeCSN(last_loaded_entry));
|
||||
latest_snapshot.notify_all();
|
||||
}
|
||||
|
||||
void TransactionLog::removeOldEntries()
|
||||
{
|
||||
/// Try to update tail pointer. It's (almost) safe to set it to the oldest snapshot
|
||||
/// because if a transaction released snapshot, then CSN is already written into metadata.
|
||||
/// Why almost? Because on server startup we do not have the oldest snapshot (it's simply equal to the latest one),
|
||||
/// but it's possible that some CSNs are not written into data parts (and we will write them during startup).
|
||||
if (!global_context->isServerCompletelyStarted())
|
||||
return;
|
||||
|
||||
/// Also similar problem is possible if some table was not attached during startup (for example, if table is detached permanently).
|
||||
/// Also we write CSNs into data parts without fsync, so it's theoretically possible that we wrote CSN, finished transaction,
|
||||
/// removed its entry from the log, but after that server restarts and CSN is not actually saved to metadata on disk.
|
||||
/// We should store a bit more entries in ZK and keep outdated entries for a while.
|
||||
|
||||
/// TODO we will need a bit more complex logic for multiple hosts
|
||||
Coordination::Stat stat;
|
||||
CSN old_tail_ptr = deserializeCSN(zookeeper->get(zookeeper_path + "/tail_ptr", &stat));
|
||||
CSN new_tail_ptr = getOldestSnapshot();
|
||||
if (new_tail_ptr < old_tail_ptr)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got unexpected tail_ptr {}, oldest snapshot is {}, it's a bug", old_tail_ptr, new_tail_ptr);
|
||||
else if (new_tail_ptr == old_tail_ptr)
|
||||
return;
|
||||
|
||||
/// (it's not supposed to fail with ZBADVERSION while there is only one host)
|
||||
LOG_TRACE(log, "Updating tail_ptr from {} to {}", old_tail_ptr, new_tail_ptr);
|
||||
zookeeper->set(zookeeper_path + "/tail_ptr", serializeCSN(new_tail_ptr), stat.version);
|
||||
tail_ptr.store(new_tail_ptr);
|
||||
|
||||
/// Now we can find and remove old entries
|
||||
TIDMap tids;
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
tids = tid_to_csn;
|
||||
}
|
||||
|
||||
/// TODO support batching
|
||||
std::vector<TIDHash> removed_entries;
|
||||
CSN latest_entry_csn = latest_snapshot.load();
|
||||
for (const auto & elem : tids)
|
||||
{
|
||||
/// Definitely not safe to remove
|
||||
if (new_tail_ptr <= elem.second.tid.start_csn)
|
||||
continue;
|
||||
|
||||
/// Keep at least one node (the latest one we fetched)
|
||||
if (elem.second.csn == latest_entry_csn)
|
||||
continue;
|
||||
|
||||
LOG_TEST(log, "Removing entry {} -> {}", elem.second.tid, elem.second.csn);
|
||||
auto code = zookeeper->tryRemove(zookeeper_path_log + "/" + serializeCSN(elem.second.csn));
|
||||
if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)
|
||||
removed_entries.push_back(elem.first);
|
||||
}
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
for (const auto & tid_hash : removed_entries)
|
||||
tid_to_csn.erase(tid_hash);
|
||||
}
|
||||
|
||||
CSN TransactionLog::getLatestSnapshot() const
|
||||
{
|
||||
return latest_snapshot.load();
|
||||
}
|
||||
|
||||
MergeTreeTransactionPtr TransactionLog::beginTransaction()
|
||||
{
|
||||
MergeTreeTransactionPtr txn;
|
||||
{
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
CSN snapshot = latest_snapshot.load();
|
||||
LocalTID ltid = 1 + local_tid_counter.fetch_add(1);
|
||||
txn = std::make_shared<MergeTreeTransaction>(snapshot, ltid, ServerUUID::get());
|
||||
bool inserted = running_list.try_emplace(txn->tid.getHash(), txn).second;
|
||||
if (!inserted)
|
||||
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_TEST(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash());
|
||||
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::BEGIN, txn->tid);
|
||||
|
||||
return txn;
|
||||
}
|
||||
|
||||
CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn)
|
||||
{
|
||||
/// Some precommit checks, may throw
|
||||
auto committing_lock = txn->beforeCommit();
|
||||
|
||||
CSN new_csn;
|
||||
if (txn->isReadOnly())
|
||||
{
|
||||
/// Don't need to allocate CSN in ZK for readonly transactions, it's safe to use snapshot/start_csn as "commit" timestamp
|
||||
LOG_TEST(log, "Closing readonly transaction {}", txn->tid);
|
||||
new_csn = txn->snapshot;
|
||||
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::COMMIT, txn->tid, new_csn);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TEST(log, "Committing transaction {}", txn->dumpDescription());
|
||||
/// TODO handle connection loss
|
||||
/// TODO support batching
|
||||
auto current_zookeeper = getZooKeeper();
|
||||
String path_created = current_zookeeper->create(zookeeper_path_log + "/csn-", serializeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point
|
||||
NOEXCEPT_SCOPE;
|
||||
|
||||
/// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough for production use
|
||||
/// (overflow is possible in a several weeks/months of active usage)
|
||||
new_csn = deserializeCSN(path_created.substr(zookeeper_path_log.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 in this session will see the changes
|
||||
/// TODO it's optional, add a setting for this
|
||||
auto current_latest_snapshot = latest_snapshot.load();
|
||||
while (current_latest_snapshot < new_csn && !stop_flag)
|
||||
{
|
||||
latest_snapshot.wait(current_latest_snapshot);
|
||||
current_latest_snapshot = latest_snapshot.load();
|
||||
}
|
||||
}
|
||||
|
||||
/// Write allocated CSN, so we will be able to cleanup log in ZK. This method is noexcept.
|
||||
txn->afterCommit(new_csn);
|
||||
|
||||
{
|
||||
/// Finally we can remove transaction from the list and release the snapshot
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
bool removed = running_list.erase(txn->tid.getHash());
|
||||
if (!removed)
|
||||
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,
|
||||
std::uncaught_exceptions() ? fmt::format(" due to uncaught exception (code: {})", getCurrentExceptionCode()) : "");
|
||||
|
||||
if (!txn->rollback())
|
||||
{
|
||||
/// Transaction was cancelled concurrently, it's already rolled back.
|
||||
assert(txn->csn == Tx::RolledBackCSN);
|
||||
return;
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
bool removed = running_list.erase(txn->tid.getHash());
|
||||
if (!removed)
|
||||
abort();
|
||||
snapshots_in_use.erase(txn->snapshot_in_use_it);
|
||||
}
|
||||
|
||||
tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::ROLLBACK, txn->tid);
|
||||
}
|
||||
|
||||
MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & tid)
|
||||
{
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
auto it = running_list.find(tid);
|
||||
if (it == running_list.end())
|
||||
return NO_TRANSACTION_PTR;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
CSN TransactionLog::getCSN(const TransactionID & tid)
|
||||
{
|
||||
/// Avoid creation of the instance if transactions are not actually involved
|
||||
if (tid == Tx::PrehistoricTID)
|
||||
return Tx::PrehistoricCSN;
|
||||
return instance().getCSNImpl(tid.getHash());
|
||||
}
|
||||
|
||||
CSN TransactionLog::getCSN(const TIDHash & tid)
|
||||
{
|
||||
/// Avoid creation of the instance if transactions are not actually involved
|
||||
if (tid == Tx::PrehistoricTID.getHash())
|
||||
return Tx::PrehistoricCSN;
|
||||
return instance().getCSNImpl(tid);
|
||||
}
|
||||
|
||||
CSN TransactionLog::getCSNImpl(const TIDHash & tid_hash) const
|
||||
{
|
||||
assert(tid_hash);
|
||||
assert(tid_hash != Tx::EmptyTID.getHash());
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = tid_to_csn.find(tid_hash);
|
||||
if (it != tid_to_csn.end())
|
||||
return it->second.csn;
|
||||
|
||||
return Tx::UnknownCSN;
|
||||
}
|
||||
|
||||
void TransactionLog::assertTIDIsNotOutdated(const TransactionID & tid)
|
||||
{
|
||||
if (tid == Tx::PrehistoricTID)
|
||||
return;
|
||||
|
||||
/// Ensure that we are not trying to get CSN for TID that was already removed from the log
|
||||
CSN tail = instance().tail_ptr.load();
|
||||
if (tail <= tid.start_csn)
|
||||
return;
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get CSN for too old TID {}, current tail_ptr is {}, probably it's a bug", tid, tail);
|
||||
}
|
||||
|
||||
CSN TransactionLog::getOldestSnapshot() const
|
||||
{
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
if (snapshots_in_use.empty())
|
||||
return getLatestSnapshot();
|
||||
return snapshots_in_use.front();
|
||||
}
|
||||
|
||||
TransactionLog::TransactionsList TransactionLog::getTransactionsList() const
|
||||
{
|
||||
std::lock_guard lock{running_list_mutex};
|
||||
return running_list;
|
||||
}
|
||||
|
||||
}
|
192
src/Interpreters/TransactionLog.h
Normal file
192
src/Interpreters/TransactionLog.h
Normal file
@ -0,0 +1,192 @@
|
||||
#pragma once
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// We want to create a TransactionLog object lazily and avoid creation if it's not needed.
|
||||
/// But we also want to call shutdown() in a specific place to avoid race conditions.
|
||||
/// We cannot simply use return-static-variable pattern,
|
||||
/// because a call to shutdown() may construct unnecessary object in this case.
|
||||
template <typename Derived>
|
||||
class SingletonHelper : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
static Derived & instance()
|
||||
{
|
||||
Derived * ptr = instance_raw_ptr.load();
|
||||
if (likely(ptr))
|
||||
return *ptr;
|
||||
|
||||
return createInstanceOrThrow();
|
||||
}
|
||||
|
||||
static void shutdownIfAny()
|
||||
{
|
||||
std::lock_guard lock{instance_mutex};
|
||||
if (instance_holder)
|
||||
instance_holder->shutdown();
|
||||
}
|
||||
|
||||
private:
|
||||
static Derived & createInstanceOrThrow();
|
||||
|
||||
static inline std::atomic<Derived *> instance_raw_ptr;
|
||||
/// It was supposed to be std::optional, but gcc fails to compile it for some reason
|
||||
static inline std::shared_ptr<Derived> instance_holder;
|
||||
static inline std::mutex instance_mutex;
|
||||
};
|
||||
|
||||
class TransactionsInfoLog;
|
||||
using TransactionsInfoLogPtr = std::shared_ptr<TransactionsInfoLog>;
|
||||
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
|
||||
|
||||
/// This class maintains transaction log in ZooKeeper and a list of currently running transactions in memory.
|
||||
///
|
||||
/// Each transaction has unique ID (TID, see details below).
|
||||
/// TransactionID is allocated when transaction begins.
|
||||
///
|
||||
/// We use TransactionID to associate changes (created/removed data parts) with transaction that has made/is going to make these changes.
|
||||
/// To commit a transaction we create sequential node "/path_to_log/log/csn-" in ZK and write TID into this node.
|
||||
/// Allocated sequential number is a commit timestamp or Commit Sequence Number (CSN). It indicates a (logical) point in time
|
||||
/// when transaction is committed and all its changes became visible. So we have total order of all changes.
|
||||
///
|
||||
/// Also CSNs are used as snapshots: all changes that were made by a transaction that was committed with a CSN less or equal than some_csn
|
||||
/// are visible in some_csn snapshot.
|
||||
///
|
||||
/// TransactionID consists of three parts: (start_csn, local_tid, host_id)
|
||||
/// - start_csn is the newest CSN that existed when the transaction was started and also it's snapshot that is visible for this transaction
|
||||
/// - local_tid is local sequential number of the transaction, each server allocates local_tids independently without requests to ZK
|
||||
/// - host_id is persistent UUID of host that has started the transaction, it's kind of tie-breaker that makes ID unique across all servers
|
||||
///
|
||||
/// To check if some transaction is committed or not we fetch "csn-xxxxxx" nodes from ZK and construct TID -> CSN mapping,
|
||||
/// so for committed transactions we know commit timestamps.
|
||||
/// However, if we did not find a mapping for some TID, it means one of the following cases:
|
||||
/// 1. Transaction is not committed (yet)
|
||||
/// 2. Transaction is rolled back (quite similar to the first case, but it will never be committed)
|
||||
/// 3. Transactions was committed a long time ago and we removed its entry from the log
|
||||
/// To distinguish the third case we store a "tail pointer" in "/path_to_log/tail_ptr". It's a CSN such that it's safe to remove from log
|
||||
/// entries with tid.start_csn < tail_ptr, because CSNs for those TIDs are already written into data parts
|
||||
/// and we will not do a CSN lookup for those TIDs anymore.
|
||||
///
|
||||
/// (however, transactions involving multiple hosts and/or ReplicatedMergeTree tables are currently not supported)
|
||||
class TransactionLog final : public SingletonHelper<TransactionLog>
|
||||
{
|
||||
public:
|
||||
|
||||
TransactionLog();
|
||||
|
||||
~TransactionLog();
|
||||
|
||||
void shutdown();
|
||||
|
||||
/// Returns the newest snapshot available for reading
|
||||
CSN getLatestSnapshot() const;
|
||||
/// Returns the oldest snapshot that is visible for some running transaction
|
||||
CSN getOldestSnapshot() const;
|
||||
|
||||
/// Allocates TID, returns new transaction object
|
||||
MergeTreeTransactionPtr beginTransaction();
|
||||
|
||||
/// Tries to commit transaction. Returns Commit Sequence Number.
|
||||
/// Throw if transaction was concurrently killed or if some precommit check failed.
|
||||
/// May throw if ZK connection is lost. Transaction status is unknown in this case.
|
||||
CSN commitTransaction(const MergeTreeTransactionPtr & txn);
|
||||
|
||||
/// Releases locks that that were acquired by transaction, releases snapshot, removes transaction from the list of active transactions.
|
||||
/// Normally it should not throw, but if it does for some reason (global memory limit exceeded, disk failure, etc)
|
||||
/// then we should terminate server and reinitialize it to avoid corruption of data structures. That's why it's noexcept.
|
||||
void rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept;
|
||||
|
||||
/// Returns CSN if transaction with specified ID was committed and UnknownCSN if it was not.
|
||||
/// Returns PrehistoricCSN for PrehistoricTID without creating a TransactionLog instance as a special case.
|
||||
static CSN getCSN(const TransactionID & tid);
|
||||
static CSN getCSN(const TIDHash & tid);
|
||||
|
||||
/// Ensures that getCSN returned UnknownCSN because transaction is not committed and not because entry was removed from the log.
|
||||
static void assertTIDIsNotOutdated(const TransactionID & tid);
|
||||
|
||||
/// Returns a pointer to transaction object if it's running or nullptr.
|
||||
MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid);
|
||||
|
||||
using TransactionsList = std::unordered_map<TIDHash, MergeTreeTransactionPtr>;
|
||||
/// Returns copy of list of running transactions.
|
||||
TransactionsList getTransactionsList() const;
|
||||
|
||||
private:
|
||||
void loadLogFromZooKeeper();
|
||||
void runUpdatingThread();
|
||||
|
||||
void loadEntries(Strings::const_iterator beg, Strings::const_iterator end);
|
||||
void loadNewEntries();
|
||||
void removeOldEntries();
|
||||
|
||||
static UInt64 deserializeCSN(const String & csn_node_name);
|
||||
static String serializeCSN(CSN csn);
|
||||
static TransactionID deserializeTID(const String & csn_node_content);
|
||||
static String serializeTID(const TransactionID & tid);
|
||||
|
||||
ZooKeeperPtr getZooKeeper() const;
|
||||
|
||||
CSN getCSNImpl(const TIDHash & tid_hash) const;
|
||||
|
||||
ContextPtr global_context;
|
||||
Poco::Logger * log;
|
||||
|
||||
/// The newest snapshot available for reading
|
||||
std::atomic<CSN> latest_snapshot;
|
||||
|
||||
/// Local part of TransactionID number. We reset this counter for each new snapshot.
|
||||
std::atomic<LocalTID> local_tid_counter;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
/// Mapping from TransactionID to CSN for recently committed transactions.
|
||||
/// Allows to check if some transactions is committed.
|
||||
struct CSNEntry
|
||||
{
|
||||
CSN csn;
|
||||
TransactionID tid;
|
||||
};
|
||||
using TIDMap = std::unordered_map<TIDHash, CSNEntry>;
|
||||
TIDMap tid_to_csn;
|
||||
|
||||
mutable std::mutex running_list_mutex;
|
||||
/// Transactions that are currently processed
|
||||
TransactionsList running_list;
|
||||
/// Ordered list of snapshots that are currently used by some transactions. Needed for background cleanup.
|
||||
std::list<CSN> snapshots_in_use;
|
||||
|
||||
ZooKeeperPtr zookeeper;
|
||||
String zookeeper_path;
|
||||
|
||||
String zookeeper_path_log;
|
||||
/// Name of the newest entry that was loaded from log in ZK
|
||||
String last_loaded_entry;
|
||||
/// The oldest CSN such that we store in log entries with TransactionIDs containing this CSN.
|
||||
std::atomic<CSN> tail_ptr = Tx::UnknownCSN;
|
||||
|
||||
zkutil::EventPtr log_updated_event = std::make_shared<Poco::Event>();
|
||||
|
||||
std::atomic_bool stop_flag = false;
|
||||
ThreadFromGlobalPool updating_thread;
|
||||
};
|
||||
|
||||
template <typename Derived>
|
||||
Derived & SingletonHelper<Derived>::createInstanceOrThrow()
|
||||
{
|
||||
std::lock_guard lock{instance_mutex};
|
||||
if (!instance_holder)
|
||||
{
|
||||
instance_holder = std::make_shared<Derived>();
|
||||
instance_raw_ptr = instance_holder.get();
|
||||
}
|
||||
return *instance_holder;
|
||||
}
|
||||
|
||||
}
|
431
src/Interpreters/TransactionVersionMetadata.cpp
Normal file
431
src/Interpreters/TransactionVersionMetadata.cpp
Normal file
@ -0,0 +1,431 @@
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SERIALIZATION_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
}
|
||||
|
||||
inline static CSN getCSNAndAssert(TIDHash tid_hash, std::atomic<CSN> & csn, const TransactionID * tid = nullptr)
|
||||
{
|
||||
CSN maybe_csn = TransactionLog::getCSN(tid_hash);
|
||||
if (maybe_csn)
|
||||
return maybe_csn;
|
||||
|
||||
/// Either transaction is not committed (yet) or it was committed and then the CSN entry was cleaned up from the log.
|
||||
/// We should load CSN again to distinguish the second case.
|
||||
/// If entry was cleaned up, then CSN is already stored in VersionMetadata and we will get it.
|
||||
/// And for the first case we will get UnknownCSN again.
|
||||
maybe_csn = csn.load();
|
||||
if (maybe_csn)
|
||||
return maybe_csn;
|
||||
|
||||
if (tid)
|
||||
TransactionLog::assertTIDIsNotOutdated(*tid);
|
||||
|
||||
return Tx::UnknownCSN;
|
||||
}
|
||||
|
||||
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::getRemovalTID() const
|
||||
{
|
||||
TIDHash removal_lock = removal_tid_lock.load();
|
||||
if (removal_lock)
|
||||
{
|
||||
if (removal_lock == Tx::PrehistoricTID.getHash())
|
||||
return Tx::PrehistoricTID;
|
||||
if (auto txn = TransactionLog::instance().tryGetRunningTransaction(removal_lock))
|
||||
return txn->tid;
|
||||
}
|
||||
|
||||
if (removal_csn.load(std::memory_order_relaxed))
|
||||
{
|
||||
/// removal_tid cannot be changed since we have removal_csn, so it's readonly
|
||||
return removal_tid;
|
||||
}
|
||||
|
||||
return Tx::EmptyTID;
|
||||
}
|
||||
|
||||
void VersionMetadata::lockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context)
|
||||
{
|
||||
LOG_TEST(log, "Trying to lock removal_tid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name);
|
||||
TIDHash locked_by = 0;
|
||||
if (tryLockRemovalTID(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 {} from {}, "
|
||||
"but it's locked by another transaction (TID: {}, TIDH: {}) which is currently removing this part.",
|
||||
tid, part_desc, context.table.getNameForLogs(), getRemovalTID(), locked_by);
|
||||
}
|
||||
|
||||
bool VersionMetadata::tryLockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id)
|
||||
{
|
||||
assert(!tid.isEmpty());
|
||||
assert(!creation_tid.isEmpty());
|
||||
TIDHash removal_lock_value = tid.getHash();
|
||||
TIDHash expected_removal_lock_value = 0;
|
||||
bool locked = removal_tid_lock.compare_exchange_strong(expected_removal_lock_value, removal_lock_value);
|
||||
if (!locked)
|
||||
{
|
||||
if (tid == Tx::PrehistoricTID && expected_removal_lock_value == Tx::PrehistoricTID.getHash())
|
||||
{
|
||||
/// Don't need to lock part for queries without transaction
|
||||
LOG_TEST(log, "Assuming removal_tid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (locked_by_id)
|
||||
*locked_by_id = expected_removal_lock_value;
|
||||
return false;
|
||||
}
|
||||
|
||||
removal_tid = tid;
|
||||
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::LOCK_PART, tid, context);
|
||||
return true;
|
||||
}
|
||||
|
||||
void VersionMetadata::unlockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context)
|
||||
{
|
||||
LOG_TEST(log, "Unlocking removal_tid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name);
|
||||
assert(!tid.isEmpty());
|
||||
TIDHash removal_lock_value = tid.getHash();
|
||||
TIDHash locked_by = removal_tid_lock.load();
|
||||
|
||||
auto throw_cannot_unlock = [&]()
|
||||
{
|
||||
auto locked_by_txn = TransactionLog::instance().tryGetRunningTransaction(locked_by);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unlock removal_tid, it's a bug. Current: {} {}, actual: {} {}",
|
||||
removal_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID);
|
||||
};
|
||||
|
||||
if (locked_by != removal_lock_value)
|
||||
throw_cannot_unlock();
|
||||
|
||||
removal_tid = Tx::EmptyTID;
|
||||
bool unlocked = removal_tid_lock.compare_exchange_strong(locked_by, 0);
|
||||
if (!unlocked)
|
||||
throw_cannot_unlock();
|
||||
|
||||
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::UNLOCK_PART, tid, context);
|
||||
}
|
||||
|
||||
bool VersionMetadata::isRemovalTIDLocked() const
|
||||
{
|
||||
return removal_tid_lock.load() != 0;
|
||||
}
|
||||
|
||||
void VersionMetadata::setCreationTID(const TransactionID & tid, TransactionInfoContext * context)
|
||||
{
|
||||
/// NOTE ReplicatedMergeTreeBlockOutputStream may add one part multiple times
|
||||
assert(creation_tid.isEmpty() || creation_tid == tid);
|
||||
creation_tid = tid;
|
||||
if (context)
|
||||
tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, *context);
|
||||
}
|
||||
|
||||
bool VersionMetadata::isVisible(const MergeTreeTransaction & txn)
|
||||
{
|
||||
return isVisible(txn.getSnapshot(), txn.tid);
|
||||
}
|
||||
|
||||
bool VersionMetadata::isVisible(CSN snapshot_version, TransactionID current_tid)
|
||||
{
|
||||
assert(!creation_tid.isEmpty());
|
||||
CSN creation = creation_csn.load(std::memory_order_relaxed);
|
||||
TIDHash removal_lock = removal_tid_lock.load(std::memory_order_relaxed);
|
||||
CSN removal = removal_csn.load(std::memory_order_relaxed);
|
||||
|
||||
[[maybe_unused]] bool had_creation_csn = creation;
|
||||
[[maybe_unused]] bool had_removal_tid = removal_lock;
|
||||
[[maybe_unused]] bool had_removal_csn = removal;
|
||||
assert(!had_removal_csn || had_removal_tid);
|
||||
assert(!had_removal_csn || had_creation_csn);
|
||||
assert(creation == Tx::UnknownCSN || creation == Tx::PrehistoricCSN || Tx::MaxReservedCSN < creation);
|
||||
assert(removal == Tx::UnknownCSN || removal == Tx::PrehistoricCSN || Tx::MaxReservedCSN < removal);
|
||||
|
||||
/// Special snapshot for introspection purposes
|
||||
if (unlikely(snapshot_version == Tx::EverythingVisibleCSN))
|
||||
return true;
|
||||
|
||||
/// 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 (creation && snapshot_version < creation)
|
||||
return false;
|
||||
if (removal && removal <= snapshot_version)
|
||||
return false;
|
||||
if (!current_tid.isEmpty() && removal_lock && removal_lock == current_tid.getHash())
|
||||
return false;
|
||||
|
||||
/// Otherwise, part is definitely visible if:
|
||||
/// - creation was committed before we took the snapshot and nobody tried to remove the part
|
||||
/// - creation was committed before and removal was committed after
|
||||
/// - current transaction is creating it
|
||||
if (creation && creation <= snapshot_version && !removal_lock)
|
||||
return true;
|
||||
if (creation && creation <= snapshot_version && removal && snapshot_version < removal)
|
||||
return true;
|
||||
if (!current_tid.isEmpty() && creation_tid == current_tid)
|
||||
return true;
|
||||
|
||||
/// End of fast path.
|
||||
|
||||
/// Data part has creation_tid/removal_tid, but does not have creation_csn/removal_csn.
|
||||
/// It means that some transaction is creating/removing the part right now or has done it recently
|
||||
/// and we don't know if it was already committed or not.
|
||||
assert(!had_creation_csn || (had_removal_tid && !had_removal_csn));
|
||||
assert(current_tid.isEmpty() || (creation_tid != current_tid && removal_lock != current_tid.getHash()));
|
||||
|
||||
/// Before doing CSN lookup, let's check some extra conditions.
|
||||
/// 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.
|
||||
if (snapshot_version <= creation_tid.start_csn)
|
||||
return false;
|
||||
|
||||
/// Check if creation_tid/removal_tid transactions are committed and write CSNs
|
||||
/// TODO Transactions: we probably need more 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 committed/rolled back.
|
||||
creation = getCSNAndAssert(creation_tid.getHash(), creation_csn, &creation_tid);
|
||||
if (!creation)
|
||||
{
|
||||
return false; /// Part creation is not committed yet
|
||||
}
|
||||
|
||||
/// 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).
|
||||
creation_csn.store(creation, std::memory_order_relaxed);
|
||||
|
||||
if (removal_lock)
|
||||
{
|
||||
removal = getCSNAndAssert(removal_lock, removal_csn);
|
||||
if (removal)
|
||||
removal_csn.store(removal, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
return creation <= snapshot_version && (!removal || snapshot_version < removal);
|
||||
}
|
||||
|
||||
bool VersionMetadata::canBeRemoved()
|
||||
{
|
||||
if (creation_tid == Tx::PrehistoricTID)
|
||||
{
|
||||
/// Avoid access to Transaction log if transactions are not involved
|
||||
|
||||
TIDHash removal_lock = removal_tid_lock.load(std::memory_order_relaxed);
|
||||
if (!removal_lock)
|
||||
return false;
|
||||
|
||||
if (removal_lock == Tx::PrehistoricTID.getHash())
|
||||
return true;
|
||||
}
|
||||
|
||||
return canBeRemovedImpl(TransactionLog::instance().getOldestSnapshot());
|
||||
}
|
||||
|
||||
bool VersionMetadata::canBeRemovedImpl(CSN oldest_snapshot_version)
|
||||
{
|
||||
CSN creation = creation_csn.load(std::memory_order_relaxed);
|
||||
/// We can safely remove part if its creation was rolled back
|
||||
if (creation == Tx::RolledBackCSN)
|
||||
return true;
|
||||
|
||||
if (!creation)
|
||||
{
|
||||
/// Cannot remove part if its creation not committed yet
|
||||
creation = getCSNAndAssert(creation_tid.getHash(), creation_csn, &creation_tid);
|
||||
if (creation)
|
||||
creation_csn.store(creation, std::memory_order_relaxed);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Part is probably visible for some transactions (part is too new or the oldest snapshot is too old)
|
||||
if (oldest_snapshot_version < creation)
|
||||
return false;
|
||||
|
||||
TIDHash removal_lock = removal_tid_lock.load(std::memory_order_relaxed);
|
||||
/// Part is active
|
||||
if (!removal_lock)
|
||||
return false;
|
||||
|
||||
CSN removal = removal_csn.load(std::memory_order_relaxed);
|
||||
if (!removal)
|
||||
{
|
||||
/// Part removal is not committed yet
|
||||
removal = getCSNAndAssert(removal_lock, removal_csn);
|
||||
if (removal)
|
||||
removal_csn.store(removal, std::memory_order_relaxed);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
/// We can safely remove part if all running transactions were started after part removal was committed
|
||||
return removal <= oldest_snapshot_version;
|
||||
}
|
||||
|
||||
#define CREATION_TID_STR "creation_tid: "
|
||||
#define CREATION_CSN_STR "creation_csn: "
|
||||
#define REMOVAL_TID_STR "removal_tid: "
|
||||
#define REMOVAL_CSN_STR "removal_csn: "
|
||||
|
||||
|
||||
void VersionMetadata::writeCSN(WriteBuffer & buf, WhichCSN which_csn, bool internal /* = false*/) const
|
||||
{
|
||||
if (which_csn == CREATION)
|
||||
{
|
||||
if (CSN creation = creation_csn.load())
|
||||
{
|
||||
writeCString("\n" CREATION_CSN_STR, buf);
|
||||
writeText(creation, buf);
|
||||
}
|
||||
else if (!internal)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "writeCSN called for creation_csn = 0, it's a bug");
|
||||
}
|
||||
else /// if (which_csn == REMOVAL)
|
||||
{
|
||||
if (CSN removal = removal_csn.load())
|
||||
{
|
||||
writeCString("\n" REMOVAL_CSN_STR, buf);
|
||||
writeText(removal, buf);
|
||||
}
|
||||
else if (!internal)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "writeCSN called for removal_csn = 0, it's a bug");
|
||||
}
|
||||
}
|
||||
|
||||
void VersionMetadata::writeRemovalTID(WriteBuffer & buf, bool clear) const
|
||||
{
|
||||
writeCString("\n" REMOVAL_TID_STR, buf);
|
||||
if (clear)
|
||||
TransactionID::write(Tx::EmptyTID, buf);
|
||||
else
|
||||
TransactionID::write(removal_tid, buf);
|
||||
}
|
||||
|
||||
void VersionMetadata::write(WriteBuffer & buf) const
|
||||
{
|
||||
writeCString("version: 1", buf);
|
||||
writeCString("\n" CREATION_TID_STR, buf);
|
||||
TransactionID::write(creation_tid, buf);
|
||||
writeCSN(buf, CREATION, /* internal */ true);
|
||||
|
||||
if (removal_tid_lock)
|
||||
{
|
||||
assert(!removal_tid.isEmpty());
|
||||
assert(removal_tid.getHash() == removal_tid_lock);
|
||||
writeRemovalTID(buf);
|
||||
writeCSN(buf, REMOVAL, /* internal */ true);
|
||||
}
|
||||
}
|
||||
|
||||
void VersionMetadata::read(ReadBuffer & buf)
|
||||
{
|
||||
constexpr size_t size = sizeof(CREATION_TID_STR) - 1;
|
||||
static_assert(sizeof(CREATION_CSN_STR) - 1 == size);
|
||||
static_assert(sizeof(REMOVAL_TID_STR) - 1 == size);
|
||||
static_assert(sizeof(REMOVAL_CSN_STR) - 1 == size);
|
||||
|
||||
assertString("version: 1", buf);
|
||||
assertString("\n" CREATION_TID_STR, buf);
|
||||
creation_tid = TransactionID::read(buf);
|
||||
if (buf.eof())
|
||||
return;
|
||||
|
||||
String name;
|
||||
name.resize(size);
|
||||
|
||||
auto read_csn = [&]()
|
||||
{
|
||||
UInt64 val;
|
||||
readText(val, buf);
|
||||
return val;
|
||||
};
|
||||
|
||||
while (!buf.eof())
|
||||
{
|
||||
assertChar('\n', buf);
|
||||
buf.readStrict(name.data(), size);
|
||||
|
||||
if (name == CREATION_CSN_STR)
|
||||
{
|
||||
assert(!creation_csn);
|
||||
creation_csn = read_csn();
|
||||
}
|
||||
else if (name == REMOVAL_TID_STR)
|
||||
{
|
||||
/// NOTE Metadata file may actually contain multiple creation TIDs, we need the last one.
|
||||
removal_tid = TransactionID::read(buf);
|
||||
if (!removal_tid.isEmpty())
|
||||
removal_tid_lock = removal_tid.getHash();
|
||||
}
|
||||
else if (name == REMOVAL_CSN_STR)
|
||||
{
|
||||
if (removal_tid.isEmpty())
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Found removal_csn in metadata file, but removal_tid is {}", removal_tid);
|
||||
assert(!removal_csn);
|
||||
removal_csn = read_csn();
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Got unexpected content: {}", name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String VersionMetadata::toString(bool one_line) const
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
write(buf);
|
||||
String res = buf.str();
|
||||
if (one_line)
|
||||
std::replace(res.begin(), res.end(), '\n', ' ');
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
}
|
81
src/Interpreters/TransactionVersionMetadata.h
Normal file
81
src/Interpreters/TransactionVersionMetadata.h
Normal file
@ -0,0 +1,81 @@
|
||||
#pragma once
|
||||
#include <Common/TransactionID.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// This structure allows to pass more information about a part that transaction is trying to create/remove.
|
||||
/// It's useful for logging and for exception messages.
|
||||
struct TransactionInfoContext
|
||||
{
|
||||
/// To which table a part belongs
|
||||
StorageID table = StorageID::createEmpty();
|
||||
/// Name of a part that transaction is trying to create/remove
|
||||
String part_name;
|
||||
/// Optional: name of part that covers `part_name` if transaction is trying to remove `part_name`
|
||||
String covering_part;
|
||||
|
||||
TransactionInfoContext(StorageID id, String part) : table(std::move(id)), part_name(std::move(part)) {}
|
||||
};
|
||||
|
||||
/// This structure contains metadata of an object (currently it's used for data parts in MergeTree only)
|
||||
/// that allows to determine when and by which transaction it has been created/removed
|
||||
struct VersionMetadata
|
||||
{
|
||||
/// ID of transaction that has created/is trying to create this object
|
||||
TransactionID creation_tid = Tx::EmptyTID;
|
||||
/// ID of transaction that has removed/is trying to remove this object
|
||||
TransactionID removal_tid = Tx::EmptyTID;
|
||||
|
||||
/// Hash of removal_tid, used to lock an object for removal
|
||||
std::atomic<TIDHash> removal_tid_lock = 0;
|
||||
|
||||
/// CSN of transaction that has created this object
|
||||
std::atomic<CSN> creation_csn = Tx::UnknownCSN;
|
||||
/// CSN of transaction that has removed this object
|
||||
std::atomic<CSN> removal_csn = Tx::UnknownCSN;
|
||||
|
||||
/// Checks if an object is visible for transaction or not.
|
||||
bool isVisible(const MergeTreeTransaction & txn);
|
||||
bool isVisible(CSN snapshot_version, TransactionID current_tid = Tx::EmptyTID);
|
||||
|
||||
TransactionID getCreationTID() const { return creation_tid; }
|
||||
TransactionID getRemovalTID() const;
|
||||
|
||||
/// Looks an object for removal, throws if it's already locked by concurrent transaction
|
||||
bool tryLockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr);
|
||||
void lockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context);
|
||||
/// Unlocks an object for removal (when transaction is rolling back)
|
||||
void unlockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context);
|
||||
|
||||
bool isRemovalTIDLocked() const;
|
||||
|
||||
/// It can be called only from MergeTreeTransaction or on server startup
|
||||
void setCreationTID(const TransactionID & tid, TransactionInfoContext * context);
|
||||
|
||||
/// Checks if it's safe to remove outdated version of an object
|
||||
bool canBeRemoved();
|
||||
bool canBeRemovedImpl(CSN oldest_snapshot_version);
|
||||
|
||||
void write(WriteBuffer & buf) const;
|
||||
void read(ReadBuffer & buf);
|
||||
|
||||
enum WhichCSN { CREATION, REMOVAL };
|
||||
void writeCSN(WriteBuffer & buf, WhichCSN which_csn, bool internal = false) const;
|
||||
void writeRemovalTID(WriteBuffer & buf, bool clear = false) const;
|
||||
|
||||
String toString(bool one_line = true) const;
|
||||
|
||||
Poco::Logger * log;
|
||||
VersionMetadata();
|
||||
};
|
||||
|
||||
DataTypePtr getTransactionIDDataType();
|
||||
|
||||
}
|
111
src/Interpreters/TransactionsInfoLog.cpp
Normal file
111
src/Interpreters/TransactionsInfoLog.cpp
Normal file
@ -0,0 +1,111 @@
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/TransactionID.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Core/NamesAndTypes.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);
|
||||
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);
|
||||
}
|
||||
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
58
src/Interpreters/TransactionsInfoLog.h
Normal file
58
src/Interpreters/TransactionsInfoLog.h
Normal file
@ -0,0 +1,58 @@
|
||||
#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;
|
||||
};
|
||||
|
||||
|
||||
void tryWriteEventToSystemLog(Poco::Logger * log, TransactionsInfoLogElement::Type type,
|
||||
const TransactionID & tid, const TransactionInfoContext & context);
|
||||
|
||||
}
|
@ -63,6 +63,9 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, c
|
||||
|
||||
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, AccessRightsElements && query_requires_access)
|
||||
{
|
||||
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ON CLUSTER queries inside transactions are not supported");
|
||||
|
||||
/// Remove FORMAT <fmt> and INTO OUTFILE <file> if exists
|
||||
ASTPtr query_ptr = query_ptr_->clone();
|
||||
ASTQueryWithOutput::resetOutputASTIfExist(*query_ptr);
|
||||
|
@ -25,6 +25,8 @@
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <Parsers/ASTWatchQuery.h>
|
||||
#include <Parsers/ASTTransactionControl.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
#include <Parsers/Lexer.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
@ -49,6 +51,7 @@
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/SelectIntersectExceptQueryVisitor.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
@ -85,6 +88,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int INTO_OUTFILE_NOT_ALLOWED;
|
||||
extern const int QUERY_WAS_CANCELLED;
|
||||
extern const int INVALID_TRANSACTION;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
@ -176,10 +180,15 @@ static void logQuery(const String & query, ContextPtr context, bool internal)
|
||||
if (!comment.empty())
|
||||
comment = fmt::format(" (comment: {})", comment);
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}){} {}",
|
||||
String transaction_info;
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
transaction_info = fmt::format(" (TID: {}, TIDH: {})", txn->tid, txn->tid.getHash());
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}){}{} {}",
|
||||
client_info.current_address.toString(),
|
||||
(current_user != "default" ? ", user: " + current_user : ""),
|
||||
(!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()),
|
||||
transaction_info,
|
||||
comment,
|
||||
joinLines(query));
|
||||
|
||||
@ -294,6 +303,9 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr
|
||||
if (elem.log_comment.size() > settings.max_query_size)
|
||||
elem.log_comment.resize(settings.max_query_size);
|
||||
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
elem.tid = txn->tid;
|
||||
|
||||
if (settings.calculate_text_stack_trace)
|
||||
setExceptionStackTrace(elem);
|
||||
logException(context, elem);
|
||||
@ -428,6 +440,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
/// TODO: parser should fail early when max_query_size limit is reached.
|
||||
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
|
||||
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
{
|
||||
assert(txn->getState() != MergeTreeTransaction::COMMITTED);
|
||||
if (txn->getState() == MergeTreeTransaction::ROLLED_BACK && !ast->as<ASTTransactionControl>() && !ast->as<ASTExplainQuery>())
|
||||
throw Exception(ErrorCodes::INVALID_TRANSACTION, "Cannot execute query: transaction is rolled back");
|
||||
}
|
||||
|
||||
/// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter),
|
||||
/// to allow settings to take effect.
|
||||
if (const auto * select_query = ast->as<ASTSelectQuery>())
|
||||
@ -629,11 +648,18 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
const auto & table_id = insert_query->table_id;
|
||||
if (!table_id.empty())
|
||||
context->setInsertionTable(table_id);
|
||||
|
||||
if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts inside transactions are not supported");
|
||||
}
|
||||
else
|
||||
{
|
||||
interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal));
|
||||
|
||||
if (context->getCurrentTransaction() && !interpreter->supportsTransactions() &&
|
||||
context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID());
|
||||
|
||||
if (!interpreter->ignoreQuota())
|
||||
{
|
||||
quota = context->getQuota();
|
||||
@ -724,6 +750,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
elem.client_info = client_info;
|
||||
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
elem.tid = txn->tid;
|
||||
|
||||
bool log_queries = settings.log_queries && !internal;
|
||||
|
||||
/// Log into system table start of query execution, if need.
|
||||
@ -945,6 +974,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(),
|
||||
quota(quota), status_info_to_query_log] () mutable
|
||||
{
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
txn->onException();
|
||||
|
||||
if (quota)
|
||||
quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false);
|
||||
|
||||
@ -1001,6 +1033,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (auto txn = context->getCurrentTransaction())
|
||||
txn->onException();
|
||||
|
||||
if (!internal)
|
||||
{
|
||||
if (query_for_logging.empty())
|
||||
|
@ -19,6 +19,7 @@ public:
|
||||
QueryPipeline, /// 'EXPLAIN PIPELINE ...'
|
||||
QueryEstimates, /// 'EXPLAIN ESTIMATE ...'
|
||||
TableOverride, /// 'EXPLAIN TABLE OVERRIDE ...'
|
||||
CurrentTransaction, /// 'EXPLAIN CURRENT TRANSACTION'
|
||||
};
|
||||
|
||||
explicit ASTExplainQuery(ExplainKind kind_) : kind(kind_) {}
|
||||
@ -111,6 +112,7 @@ private:
|
||||
case QueryPipeline: return "EXPLAIN PIPELINE";
|
||||
case QueryEstimates: return "EXPLAIN ESTIMATE";
|
||||
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
|
||||
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
|
@ -24,6 +24,9 @@ void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatS
|
||||
case Type::PartMoveToShard:
|
||||
settings.ostr << "PART_MOVE_TO_SHARD";
|
||||
break;
|
||||
case Type::Transaction:
|
||||
settings.ostr << "TRANSACTION";
|
||||
break;
|
||||
}
|
||||
|
||||
formatOnCluster(settings);
|
||||
|
@ -14,6 +14,7 @@ public:
|
||||
Query, /// KILL QUERY
|
||||
Mutation, /// KILL MUTATION
|
||||
PartMoveToShard, /// KILL PART_MOVE_TO_SHARD
|
||||
Transaction, /// KILL TRANSACTION
|
||||
};
|
||||
|
||||
Type type = Type::Query;
|
||||
|
32
src/Parsers/ASTTransactionControl.cpp
Normal file
32
src/Parsers/ASTTransactionControl.cpp
Normal file
@ -0,0 +1,32 @@
|
||||
#include <Parsers/ASTTransactionControl.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void ASTTransactionControl::formatImpl(const FormatSettings & format /*state*/, FormatState &, FormatStateStacked /*frame*/) const
|
||||
{
|
||||
switch (action)
|
||||
{
|
||||
case BEGIN:
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "BEGIN TRANSACTION" << (format.hilite ? hilite_none : "");
|
||||
break;
|
||||
case COMMIT:
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "COMMIT" << (format.hilite ? hilite_none : "");
|
||||
break;
|
||||
case ROLLBACK:
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "ROLLBACK" << (format.hilite ? hilite_none : "");
|
||||
break;
|
||||
case SET_SNAPSHOT:
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "SET TRANSACTION SNAPSHOT " << (format.hilite ? hilite_none : "") << snapshot;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void ASTTransactionControl::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
hash_state.update(action);
|
||||
}
|
||||
|
||||
}
|
32
src/Parsers/ASTTransactionControl.h
Normal file
32
src/Parsers/ASTTransactionControl.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Common AST for TCL queries
|
||||
class ASTTransactionControl : public IAST
|
||||
{
|
||||
public:
|
||||
enum QueryType
|
||||
{
|
||||
BEGIN,
|
||||
COMMIT,
|
||||
ROLLBACK,
|
||||
SET_SNAPSHOT,
|
||||
};
|
||||
|
||||
QueryType action;
|
||||
|
||||
UInt64 snapshot; /// For SET TRANSACTION SNAPSHOT ...
|
||||
|
||||
ASTTransactionControl(QueryType action_) : action(action_) {}
|
||||
|
||||
String getID(char /*delimiter*/) const override { return "ASTTransactionControl"; }
|
||||
ASTPtr clone() const override { return std::make_shared<ASTTransactionControl>(*this); }
|
||||
|
||||
void formatImpl(const FormatSettings & format, FormatState & /*state*/, FormatStateStacked /*frame*/) const override;
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
};
|
||||
|
||||
}
|
@ -22,6 +22,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_plan("PLAN");
|
||||
ParserKeyword s_estimates("ESTIMATE");
|
||||
ParserKeyword s_table_override("TABLE OVERRIDE");
|
||||
ParserKeyword s_current_transaction("CURRENT TRANSACTION");
|
||||
|
||||
if (s_explain.ignore(pos, expected))
|
||||
{
|
||||
@ -39,6 +40,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
kind = ASTExplainQuery::ExplainKind::QueryEstimates; //-V1048
|
||||
else if (s_table_override.ignore(pos, expected))
|
||||
kind = ASTExplainQuery::ExplainKind::TableOverride;
|
||||
else if (s_current_transaction.ignore(pos, expected))
|
||||
kind = ASTExplainQuery::ExplainKind::CurrentTransaction;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
@ -79,6 +82,10 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
explain_query->setTableFunction(table_function);
|
||||
explain_query->setTableOverride(table_override);
|
||||
}
|
||||
else if (kind == ASTExplainQuery::ExplainKind::CurrentTransaction)
|
||||
{
|
||||
/// Nothing to parse
|
||||
}
|
||||
else if (select_p.parse(pos, query, expected) ||
|
||||
create_p.parse(pos, query, expected) ||
|
||||
insert_p.parse(pos, query, expected))
|
||||
|
@ -18,6 +18,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
|
||||
ParserKeyword p_query{"QUERY"};
|
||||
ParserKeyword p_mutation{"MUTATION"};
|
||||
ParserKeyword p_part_move_to_shard{"PART_MOVE_TO_SHARD"};
|
||||
ParserKeyword p_transaction{"TRANSACTION"};
|
||||
ParserKeyword p_on{"ON"};
|
||||
ParserKeyword p_test{"TEST"};
|
||||
ParserKeyword p_sync{"SYNC"};
|
||||
@ -34,6 +35,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
|
||||
query->type = ASTKillQueryQuery::Type::Mutation;
|
||||
else if (p_part_move_to_shard.ignore(pos, expected))
|
||||
query->type = ASTKillQueryQuery::Type::PartMoveToShard;
|
||||
else if (p_transaction.ignore(pos, expected))
|
||||
query->type = ASTKillQueryQuery::Type::Transaction;
|
||||
else
|
||||
return false;
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Parsers/ParserSystemQuery.h>
|
||||
#include <Parsers/ParserUseQuery.h>
|
||||
#include <Parsers/ParserExternalDDLQuery.h>
|
||||
#include <Parsers/ParserTransactionControl.h>
|
||||
|
||||
#include <Parsers/Access/ParserCreateQuotaQuery.h>
|
||||
#include <Parsers/Access/ParserCreateRoleQuery.h>
|
||||
@ -46,6 +47,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserGrantQuery grant_p;
|
||||
ParserSetRoleQuery set_role_p;
|
||||
ParserExternalDDLQuery external_ddl_p;
|
||||
ParserTransactionControl transaction_control_p;
|
||||
ParserBackupQuery backup_p;
|
||||
|
||||
bool res = query_with_output_p.parse(pos, node, expected)
|
||||
@ -64,6 +66,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|| drop_access_entity_p.parse(pos, node, expected)
|
||||
|| grant_p.parse(pos, node, expected)
|
||||
|| external_ddl_p.parse(pos, node, expected)
|
||||
|| transaction_control_p.parse(pos, node, expected)
|
||||
|| backup_p.parse(pos, node, expected);
|
||||
|
||||
return res;
|
||||
|
@ -53,6 +53,10 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
if (!s_set.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
/// Parse SET TRANSACTION ... queries using ParserTransactionControl
|
||||
if (ParserKeyword{"TRANSACTION"}.check(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
SettingsChanges changes;
|
||||
|
41
src/Parsers/ParserTransactionControl.cpp
Normal file
41
src/Parsers/ParserTransactionControl.cpp
Normal file
@ -0,0 +1,41 @@
|
||||
#include <Parsers/ParserTransactionControl.h>
|
||||
#include <Parsers/ASTTransactionControl.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTTransactionControl::QueryType action;
|
||||
UInt64 snapshot = 0;
|
||||
|
||||
if (ParserKeyword("BEGIN TRANSACTION").ignore(pos, expected))
|
||||
action = ASTTransactionControl::BEGIN;
|
||||
else if (ParserKeyword("COMMIT").ignore(pos, expected))
|
||||
action = ASTTransactionControl::COMMIT;
|
||||
else if (ParserKeyword("ROLLBACK").ignore(pos, expected))
|
||||
action = ASTTransactionControl::ROLLBACK;
|
||||
else if (ParserKeyword("SET TRANSACTION SNAPSHOT").ignore(pos, expected))
|
||||
{
|
||||
action = ASTTransactionControl::SET_SNAPSHOT;
|
||||
ASTPtr ast;
|
||||
if (!ParserNumber{}.parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
const auto & snapshot_num = ast->as<ASTLiteral>()->value;
|
||||
if (!snapshot_num.tryGet(snapshot))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
auto ast = std::make_shared<ASTTransactionControl>(action);
|
||||
ast->snapshot = snapshot;
|
||||
node = ast;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
14
src/Parsers/ParserTransactionControl.h
Normal file
14
src/Parsers/ParserTransactionControl.h
Normal file
@ -0,0 +1,14 @@
|
||||
#pragma once
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserTransactionControl : public IParserBase
|
||||
{
|
||||
public:
|
||||
const char * getName() const override { return "TCL query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -563,7 +563,7 @@ void RemoteQueryExecutor::sendExternalTables()
|
||||
{
|
||||
SelectQueryInfo query_info;
|
||||
auto metadata_snapshot = cur->getInMemoryMetadataPtr();
|
||||
auto storage_snapshot = cur->getStorageSnapshot(metadata_snapshot);
|
||||
auto storage_snapshot = cur->getStorageSnapshot(metadata_snapshot, context);
|
||||
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(
|
||||
context, QueryProcessingStage::Complete, storage_snapshot, query_info);
|
||||
|
||||
|
@ -679,7 +679,7 @@ bool StorageFileLog::streamToViews()
|
||||
throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
auto storage_snapshot = getStorageSnapshot(metadata_snapshot);
|
||||
auto storage_snapshot = getStorageSnapshot(metadata_snapshot, getContext());
|
||||
|
||||
auto max_streams_number = std::min<UInt64>(filelog_settings->max_threads.value, file_infos.file_names.size());
|
||||
/// No files to parse
|
||||
|
@ -161,6 +161,11 @@ public:
|
||||
/// Returns true if the storage supports reading of subcolumns of complex types.
|
||||
virtual bool supportsSubcolumns() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports transactions for SELECT, INSERT and ALTER queries.
|
||||
/// Storage may throw an exception later if some query kind is not fully supported.
|
||||
/// This method can return true for readonly engines that return the same rows for reading (such as SystemNumbers)
|
||||
virtual bool supportsTransactions() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports storing of dynamic subcolumns.
|
||||
/// For now it makes sense only for data type Object.
|
||||
virtual bool supportsDynamicSubcolumns() const { return false; }
|
||||
@ -483,6 +488,16 @@ public:
|
||||
throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
virtual void waitForMutation(const String & /*mutation_id*/)
|
||||
{
|
||||
throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
virtual void setMutationCSN(const String & /*mutation_id*/, UInt64 /*csn*/)
|
||||
{
|
||||
throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Cancel a part move to shard.
|
||||
virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/)
|
||||
{
|
||||
@ -553,11 +568,6 @@ public:
|
||||
/// Similar to above but checks for DETACH. It's only used for DICTIONARIES.
|
||||
virtual void checkTableCanBeDetached() const {}
|
||||
|
||||
/// Checks that Partition could be dropped right now
|
||||
/// Otherwise - throws an exception with detailed information.
|
||||
/// We do not use mutex because it is not very important that the size could change during the operation.
|
||||
virtual void checkPartitionCanBeDropped(const ASTPtr & /*partition*/) {}
|
||||
|
||||
/// Returns true if Storage may store some data on disk.
|
||||
/// NOTE: may not be equivalent to !getDataPaths().empty()
|
||||
virtual bool storesDataOnDisk() const { return false; }
|
||||
@ -610,15 +620,15 @@ public:
|
||||
virtual std::optional<UInt64> lifetimeBytes() const { return {}; }
|
||||
|
||||
/// Creates a storage snapshot from given metadata.
|
||||
virtual StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const
|
||||
virtual StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const
|
||||
{
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot);
|
||||
}
|
||||
|
||||
/// Creates a storage snapshot from given metadata and columns, which are used in query.
|
||||
virtual StorageSnapshotPtr getStorageSnapshotForQuery(const StorageMetadataPtr & metadata_snapshot, const ASTPtr & /*query*/) const
|
||||
virtual StorageSnapshotPtr getStorageSnapshotForQuery(const StorageMetadataPtr & metadata_snapshot, const ASTPtr & /*query*/, ContextPtr query_context) const
|
||||
{
|
||||
return getStorageSnapshot(metadata_snapshot);
|
||||
return getStorageSnapshot(metadata_snapshot, query_context);
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -647,7 +647,7 @@ bool StorageKafka::streamToViews()
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads};
|
||||
ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads);
|
||||
|
||||
auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr());
|
||||
auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext());
|
||||
|
||||
// Create an INSERT query for streaming data
|
||||
auto insert = std::make_shared<ASTInsertQuery>();
|
||||
|
@ -573,6 +573,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part =
|
||||
std::make_shared<MergeTreeDataPartInMemory>(data, part_name, volume);
|
||||
new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
|
||||
|
||||
for (auto i = 0ul; i < projections; ++i)
|
||||
{
|
||||
@ -601,7 +602,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
metadata_snapshot->projections.get(projection_name).metadata,
|
||||
block.getNamesAndTypesList(),
|
||||
{},
|
||||
CompressionCodecFactory::instance().get("NONE", {}));
|
||||
CompressionCodecFactory::instance().get("NONE", {}),
|
||||
NO_TRANSACTION_PTR);
|
||||
|
||||
part_out.write(block);
|
||||
part_out.finalizePart(new_projection_part, false);
|
||||
@ -625,7 +627,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
|
||||
MergedBlockOutputStream part_out(
|
||||
new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {},
|
||||
CompressionCodecFactory::instance().get("NONE", {}));
|
||||
CompressionCodecFactory::instance().get("NONE", {}), NO_TRANSACTION_PTR);
|
||||
|
||||
part_out.write(block);
|
||||
part_out.finalizePart(new_data_part, false);
|
||||
@ -753,6 +755,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
|
||||
assertEOF(in);
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
|
||||
new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->modification_time = time(nullptr);
|
||||
new_data_part->loadColumnsChecksumsIndexes(true, false);
|
||||
@ -842,6 +845,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta(
|
||||
assertEOF(in);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
|
||||
new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->modification_time = time(nullptr);
|
||||
new_data_part->loadColumnsChecksumsIndexes(true, false);
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include <Storages/MergeTree/PartMetadataManagerWithCache.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
@ -29,6 +28,8 @@
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -65,6 +66,12 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
|
||||
{
|
||||
size_t file_size = disk->getFileSize(path);
|
||||
return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size);
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager)
|
||||
{
|
||||
auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||
@ -466,6 +473,7 @@ SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & co
|
||||
|
||||
void IMergeTreeDataPart::removeIfNeeded()
|
||||
{
|
||||
assert(assertHasValidVersionMetadata());
|
||||
if (!is_temp && state != State::DeleteOnDestroy)
|
||||
return;
|
||||
|
||||
@ -790,10 +798,14 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const
|
||||
|
||||
NameSet result = {"checksums.txt", "columns.txt"};
|
||||
String default_codec_path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME;
|
||||
String txn_version_path = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
|
||||
if (volume->getDisk()->exists(default_codec_path))
|
||||
result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
|
||||
|
||||
if (volume->getDisk()->exists(txn_version_path))
|
||||
result.emplace(TXN_VERSION_METADATA_FILE_NAME);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -1223,6 +1235,218 @@ void IMergeTreeDataPart::loadColumns(bool require)
|
||||
setSerializationInfos(infos);
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const
|
||||
{
|
||||
TransactionID expected_tid = txn ? txn->tid : Tx::PrehistoricTID;
|
||||
if (version.creation_tid != expected_tid)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"CreationTID of part {} (table {}) is set to unexpected value {}, it's a bug. Current transaction: {}",
|
||||
name, storage.getStorageID().getNameForLogs(), version.creation_tid, txn ? txn->dumpDescription() : "<none>");
|
||||
|
||||
assert(!txn || storage.supportsTransactions());
|
||||
assert(!txn || volume->getDisk()->exists(fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME));
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::storeVersionMetadata() const
|
||||
{
|
||||
if (!wasInvolvedInTransaction())
|
||||
return;
|
||||
|
||||
LOG_TEST(storage.log, "Writing version for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid);
|
||||
assert(storage.supportsTransactions());
|
||||
|
||||
if (!isStoredOnDisk())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for in-memory parts (table: {}, part: {})",
|
||||
storage.getStorageID().getNameForLogs(), name);
|
||||
|
||||
String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
String tmp_version_file_name = version_file_name + ".tmp";
|
||||
DiskPtr disk = volume->getDisk();
|
||||
{
|
||||
/// TODO IDisk interface does not allow to open file with O_EXCL flag (for DiskLocal),
|
||||
/// so we create empty file at first (expecting that createFile throws if file already exists)
|
||||
/// and then overwrite it.
|
||||
disk->createFile(tmp_version_file_name);
|
||||
auto out = disk->writeFile(tmp_version_file_name, 256, WriteMode::Rewrite);
|
||||
version.write(*out);
|
||||
out->finalize();
|
||||
out->sync();
|
||||
}
|
||||
|
||||
SyncGuardPtr sync_guard;
|
||||
if (storage.getSettings()->fsync_part_directory)
|
||||
sync_guard = disk->getDirectorySyncGuard(getFullRelativePath());
|
||||
disk->replaceFile(tmp_version_file_name, version_file_name);
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::appendCSNToVersionMetadata(VersionMetadata::WhichCSN which_csn) const
|
||||
{
|
||||
assert(!version.creation_tid.isEmpty());
|
||||
assert(!(which_csn == VersionMetadata::WhichCSN::CREATION && version.creation_tid.isPrehistoric()));
|
||||
assert(!(which_csn == VersionMetadata::WhichCSN::CREATION && version.creation_csn == 0));
|
||||
assert(!(which_csn == VersionMetadata::WhichCSN::REMOVAL && (version.removal_tid.isPrehistoric() || version.removal_tid.isEmpty())));
|
||||
assert(!(which_csn == VersionMetadata::WhichCSN::REMOVAL && version.removal_csn == 0));
|
||||
assert(isStoredOnDisk());
|
||||
|
||||
/// Small enough appends to file are usually atomic,
|
||||
/// so we append new metadata instead of rewriting file to reduce number of fsyncs.
|
||||
/// We don't need to do fsync when writing CSN, because in case of hard restart
|
||||
/// we will be able to restore CSN from transaction log in Keeper.
|
||||
|
||||
String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
DiskPtr disk = volume->getDisk();
|
||||
auto out = disk->writeFile(version_file_name, 256, WriteMode::Append);
|
||||
version.writeCSN(*out, which_csn);
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::appendRemovalTIDToVersionMetadata(bool clear) const
|
||||
{
|
||||
assert(!version.creation_tid.isEmpty());
|
||||
assert(version.removal_csn == 0);
|
||||
assert(!version.removal_tid.isEmpty());
|
||||
assert(isStoredOnDisk());
|
||||
|
||||
if (version.creation_tid.isPrehistoric() && !clear)
|
||||
{
|
||||
/// Metadata file probably does not exist, because it was not written on part creation, because it was created without a transaction.
|
||||
/// Let's create it (if needed). Concurrent writes are not possible, because creation_csn is prehistoric and we own removal_tid_lock.
|
||||
storeVersionMetadata();
|
||||
return;
|
||||
}
|
||||
|
||||
if (clear)
|
||||
LOG_TEST(storage.log, "Clearing removal TID for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid);
|
||||
else
|
||||
LOG_TEST(storage.log, "Appending removal TID for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid);
|
||||
|
||||
String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
DiskPtr disk = volume->getDisk();
|
||||
auto out = disk->writeFile(version_file_name, 256, WriteMode::Append);
|
||||
version.writeRemovalTID(*out, clear);
|
||||
out->finalize();
|
||||
|
||||
/// fsync is not required when we clearing removal TID, because after hard restart we will fix metadata
|
||||
if (!clear)
|
||||
out->sync();
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::loadVersionMetadata() const
|
||||
try
|
||||
{
|
||||
String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
String tmp_version_file_name = version_file_name + ".tmp";
|
||||
DiskPtr disk = volume->getDisk();
|
||||
|
||||
auto remove_tmp_file = [&]()
|
||||
{
|
||||
auto last_modified = disk->getLastModified(tmp_version_file_name);
|
||||
auto buf = openForReading(disk, tmp_version_file_name);
|
||||
String content;
|
||||
readStringUntilEOF(content, *buf);
|
||||
LOG_WARNING(storage.log, "Found file {} that was last modified on {}, has size {} and the following content: {}",
|
||||
tmp_version_file_name, last_modified.epochTime(), content.size(), content);
|
||||
disk->removeFile(tmp_version_file_name);
|
||||
};
|
||||
|
||||
if (disk->exists(version_file_name))
|
||||
{
|
||||
auto buf = openForReading(disk, version_file_name);
|
||||
version.read(*buf);
|
||||
if (disk->exists(tmp_version_file_name))
|
||||
remove_tmp_file();
|
||||
return;
|
||||
}
|
||||
|
||||
/// Four (?) cases are possible:
|
||||
/// 1. Part was created without transactions.
|
||||
/// 2. Version metadata file was not renamed from *.tmp on part creation.
|
||||
/// 3. Version metadata were written to *.tmp file, but hard restart happened before fsync.
|
||||
/// 4. Fsyncs in storeVersionMetadata() work incorrectly.
|
||||
|
||||
if (!disk->exists(tmp_version_file_name))
|
||||
{
|
||||
/// Case 1.
|
||||
/// 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.
|
||||
/// NOTE It might be Case 3, but version metadata file is written on part creation before other files,
|
||||
/// so it's not Case 3 if part is not broken.
|
||||
version.setCreationTID(Tx::PrehistoricTID, nullptr);
|
||||
version.creation_csn = Tx::PrehistoricCSN;
|
||||
return;
|
||||
}
|
||||
|
||||
/// 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.
|
||||
version.setCreationTID(Tx::DummyTID, nullptr);
|
||||
version.creation_csn = Tx::RolledBackCSN;
|
||||
remove_tmp_file();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("While loading version metadata from table {} part {}", storage.getStorageID().getNameForLogs(), name);
|
||||
throw;
|
||||
}
|
||||
|
||||
bool IMergeTreeDataPart::wasInvolvedInTransaction() const
|
||||
{
|
||||
assert(!version.creation_tid.isEmpty() || (state == State::Temporary /* && std::uncaught_exceptions() */));
|
||||
bool created_by_transaction = !version.creation_tid.isPrehistoric();
|
||||
bool removed_by_transaction = version.isRemovalTIDLocked() && version.removal_tid_lock != Tx::PrehistoricTID.getHash();
|
||||
return created_by_transaction || removed_by_transaction;
|
||||
}
|
||||
|
||||
bool IMergeTreeDataPart::assertHasValidVersionMetadata() const
|
||||
{
|
||||
/// We don't have many tests with server restarts and it's really inconvenient to write such tests.
|
||||
/// So we use debug assertions to ensure that part version is written correctly.
|
||||
/// This method is not supposed to be called in release builds.
|
||||
|
||||
if (isProjectionPart())
|
||||
return true;
|
||||
|
||||
if (!wasInvolvedInTransaction())
|
||||
return true;
|
||||
|
||||
if (!isStoredOnDisk())
|
||||
return false;
|
||||
|
||||
if (part_is_probably_removed_from_disk)
|
||||
return true;
|
||||
|
||||
DiskPtr disk = volume->getDisk();
|
||||
if (!disk->exists(getFullRelativePath()))
|
||||
return true;
|
||||
|
||||
String content;
|
||||
String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME;
|
||||
try
|
||||
{
|
||||
auto buf = openForReading(disk, version_file_name);
|
||||
readStringUntilEOF(content, *buf);
|
||||
ReadBufferFromString str_buf{content};
|
||||
VersionMetadata file;
|
||||
file.read(str_buf);
|
||||
bool valid_creation_tid = version.creation_tid == file.creation_tid;
|
||||
bool valid_removal_tid = version.removal_tid == file.removal_tid || version.removal_tid == Tx::PrehistoricTID;
|
||||
bool valid_creation_csn = version.creation_csn == file.creation_csn || version.creation_csn == Tx::RolledBackCSN;
|
||||
bool valid_removal_csn = version.removal_csn == file.removal_csn || version.removal_csn == Tx::PrehistoricCSN;
|
||||
if (!valid_creation_tid || !valid_removal_tid || !valid_creation_csn || !valid_removal_csn)
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid version metadata file");
|
||||
return true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
WriteBufferFromOwnString expected;
|
||||
version.write(expected);
|
||||
tryLogCurrentException(storage.log, fmt::format("File {} contains:\n{}\nexpected:\n{}", version_file_name, content, expected.str()));
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IMergeTreeDataPart::appendFilesOfColumns(Strings & files)
|
||||
{
|
||||
files.push_back("columns.txt");
|
||||
@ -1337,6 +1561,9 @@ void IMergeTreeDataPart::initializePartMetadataManager()
|
||||
|
||||
void IMergeTreeDataPart::remove() const
|
||||
{
|
||||
assert(assertHasValidVersionMetadata());
|
||||
part_is_probably_removed_from_disk = true;
|
||||
|
||||
std::optional<bool> keep_shared_data = keepSharedDataInDecoupledStorage();
|
||||
if (!keep_shared_data.has_value())
|
||||
return;
|
||||
@ -1444,6 +1671,7 @@ void IMergeTreeDataPart::remove() const
|
||||
|
||||
request.emplace_back(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, true);
|
||||
request.emplace_back(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, true);
|
||||
request.emplace_back(fs::path(to) / TXN_VERSION_METADATA_FILE_NAME, true);
|
||||
|
||||
disk->removeSharedFiles(request, *keep_shared_data);
|
||||
disk->removeDirectory(to);
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreeIOSettings.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
#include <Storages/MergeTree/IPartMetadataManager.h>
|
||||
|
||||
@ -40,6 +41,7 @@ class IMergeTreeReader;
|
||||
class IMergeTreeDataPartWriter;
|
||||
class MarkCache;
|
||||
class UncompressedCache;
|
||||
class MergeTreeTransaction;
|
||||
|
||||
/// Description of the data part.
|
||||
class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPart>
|
||||
@ -327,6 +329,8 @@ public:
|
||||
|
||||
CompressionCodecPtr default_codec;
|
||||
|
||||
mutable VersionMetadata version;
|
||||
|
||||
/// For data in RAM ('index')
|
||||
UInt64 getIndexSizeInBytes() const;
|
||||
UInt64 getIndexSizeInAllocatedBytes() const;
|
||||
@ -414,6 +418,8 @@ public:
|
||||
/// (number of rows, number of rows with default values, etc).
|
||||
static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.json";
|
||||
|
||||
static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt";
|
||||
|
||||
/// One of part files which is used to check how many references (I'd like
|
||||
/// to say hardlinks, but it will confuse even more) we have for the part
|
||||
/// for zero copy replication. Sadly it's very complex.
|
||||
@ -435,12 +441,38 @@ public:
|
||||
/// Required for distinguish different copies of the same part on remote FS.
|
||||
String getUniqueId() const;
|
||||
|
||||
/// Ensures that creation_tid was correctly set after part creation.
|
||||
void assertHasVersionMetadata(MergeTreeTransaction * txn) const;
|
||||
|
||||
/// [Re]writes file with transactional metadata on disk
|
||||
void storeVersionMetadata() const;
|
||||
|
||||
/// Appends the corresponding CSN to file on disk (without fsync)
|
||||
void appendCSNToVersionMetadata(VersionMetadata::WhichCSN which_csn) const;
|
||||
|
||||
/// Appends removal TID to file on disk (with fsync)
|
||||
void appendRemovalTIDToVersionMetadata(bool clear = false) const;
|
||||
|
||||
/// Loads transactional metadata from disk
|
||||
void loadVersionMetadata() const;
|
||||
|
||||
/// Returns true if part was created or removed by a transaction
|
||||
bool wasInvolvedInTransaction() const;
|
||||
|
||||
/// Moar hardening: this method is supposed to be used for debug assertions
|
||||
bool assertHasValidVersionMetadata() const;
|
||||
|
||||
/// Return hardlink count for part.
|
||||
/// Required for keep data on remote FS when part has shadow copies.
|
||||
UInt32 getNumberOfRefereneces() const;
|
||||
|
||||
/// Get checksums of metadata file in part directory
|
||||
IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const;
|
||||
|
||||
/// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true)
|
||||
std::unordered_map<String, uint128> checkMetadata() const;
|
||||
|
||||
|
||||
protected:
|
||||
|
||||
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
|
||||
@ -551,6 +583,9 @@ private:
|
||||
CompressionCodecPtr detectDefaultCompressionCodec() const;
|
||||
|
||||
mutable State state{State::Temporary};
|
||||
|
||||
/// This ugly flag is needed for debug assertions only
|
||||
mutable bool part_is_probably_removed_from_disk = false;
|
||||
};
|
||||
|
||||
using MergeTreeDataPartState = IMergeTreeDataPart::State;
|
||||
|
@ -227,7 +227,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
future_merged_part,
|
||||
settings);
|
||||
|
||||
transaction_ptr = std::make_unique<MergeTreeData::Transaction>(storage);
|
||||
transaction_ptr = std::make_unique<MergeTreeData::Transaction>(storage, NO_TRANSACTION_RAW);
|
||||
stopwatch_ptr = std::make_unique<Stopwatch>();
|
||||
|
||||
merge_task = storage.merger_mutator.mergePartsToTemporaryPart(
|
||||
@ -241,7 +241,8 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
reserved_space,
|
||||
entry.deduplicate,
|
||||
entry.deduplicate_by_columns,
|
||||
storage.merging_params);
|
||||
storage.merging_params,
|
||||
NO_TRANSACTION_PTR);
|
||||
|
||||
|
||||
/// Adjust priority
|
||||
@ -264,7 +265,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite
|
||||
/// Task is not needed
|
||||
merge_task.reset();
|
||||
|
||||
storage.merger_mutator.renameMergedTemporaryPart(part, parts, transaction_ptr.get());
|
||||
storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, transaction_ptr.get());
|
||||
|
||||
try
|
||||
{
|
||||
|
@ -39,10 +39,13 @@ struct MergeMutateSelectedEntry
|
||||
FutureMergedMutatedPartPtr future_part;
|
||||
CurrentlyMergingPartsTaggerPtr tagger;
|
||||
MutationCommandsConstPtr commands;
|
||||
MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, MutationCommandsConstPtr commands_)
|
||||
MergeTreeTransactionPtr txn;
|
||||
MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_,
|
||||
MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR)
|
||||
: future_part(future_part_)
|
||||
, tagger(std::move(tagger_))
|
||||
, commands(commands_)
|
||||
, txn(txn_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
@ -107,14 +107,15 @@ void MergePlainMergeTreeTask::prepare()
|
||||
merge_mutate_entry->tagger->reserved_space,
|
||||
deduplicate,
|
||||
deduplicate_by_columns,
|
||||
storage.merging_params);
|
||||
storage.merging_params,
|
||||
txn);
|
||||
}
|
||||
|
||||
|
||||
void MergePlainMergeTreeTask::finish()
|
||||
{
|
||||
new_part = merge_task->getFuture().get();
|
||||
storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, nullptr);
|
||||
storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, txn, nullptr);
|
||||
write_part_log({});
|
||||
}
|
||||
|
||||
|
@ -39,6 +39,12 @@ public:
|
||||
StorageID getStorageID() override;
|
||||
UInt64 getPriority() override { return priority; }
|
||||
|
||||
void setCurrentTransaction(MergeTreeTransactionHolder && txn_holder_, MergeTreeTransactionPtr && txn_)
|
||||
{
|
||||
txn_holder = std::move(txn_holder_);
|
||||
txn = std::move(txn_);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
void prepare();
|
||||
@ -73,6 +79,9 @@ private:
|
||||
std::function<void(const ExecutionStatus &)> write_part_log;
|
||||
IExecutableTask::TaskResultCallback task_result_callback;
|
||||
MergeTaskPtr merge_task{nullptr};
|
||||
|
||||
MergeTreeTransactionHolder txn_holder;
|
||||
MergeTreeTransactionPtr txn;
|
||||
};
|
||||
|
||||
|
||||
|
@ -260,6 +260,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
||||
global_ctx->merging_columns,
|
||||
MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()),
|
||||
ctx->compression_codec,
|
||||
global_ctx->txn,
|
||||
/*reset_columns=*/ true,
|
||||
ctx->blocks_are_granules_size,
|
||||
global_ctx->context->getWriteSettings());
|
||||
@ -593,6 +594,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c
|
||||
projection_merging_params,
|
||||
global_ctx->new_data_part.get(),
|
||||
".proj",
|
||||
NO_TRANSACTION_PTR,
|
||||
global_ctx->data,
|
||||
global_ctx->mutator,
|
||||
global_ctx->merges_blocker,
|
||||
|
@ -60,6 +60,7 @@ public:
|
||||
MergeTreeData::MergingParams merging_params_,
|
||||
const IMergeTreeDataPart * parent_part_,
|
||||
String suffix_,
|
||||
MergeTreeTransactionPtr txn,
|
||||
MergeTreeData * data_,
|
||||
MergeTreeDataMergerMutator * mutator_,
|
||||
ActionBlocker * merges_blocker_,
|
||||
@ -83,6 +84,7 @@ public:
|
||||
global_ctx->mutator = std::move(mutator_);
|
||||
global_ctx->merges_blocker = std::move(merges_blocker_);
|
||||
global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_);
|
||||
global_ctx->txn = std::move(txn);
|
||||
|
||||
auto prepare_stage_ctx = std::make_shared<ExecuteAndFinalizeHorizontalPartRuntimeContext>();
|
||||
|
||||
@ -164,6 +166,8 @@ private:
|
||||
std::promise<MergeTreeData::MutableDataPartPtr> promise{};
|
||||
|
||||
IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{};
|
||||
|
||||
MergeTreeTransactionPtr txn;
|
||||
};
|
||||
|
||||
using GlobalRuntimeContextPtr = std::shared_ptr<GlobalRuntimeContext>;
|
||||
|
@ -30,8 +30,10 @@
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -49,8 +51,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MergeTree/localBackup.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
@ -1320,51 +1320,155 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
for (auto & part : duplicate_parts_to_remove)
|
||||
part->remove();
|
||||
|
||||
auto deactivate_part = [&] (DataPartIteratorByStateAndInfo it)
|
||||
{
|
||||
|
||||
(*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed);
|
||||
auto creation_csn = (*it)->version.creation_csn.load(std::memory_order_relaxed);
|
||||
if (creation_csn != Tx::RolledBackCSN && creation_csn != Tx::PrehistoricCSN && !(*it)->version.isRemovalTIDLocked())
|
||||
{
|
||||
/// It's possible that covering part was created without transaction,
|
||||
/// but if covered part was created with transaction (i.e. creation_tid is not prehistoric),
|
||||
/// then it must have removal tid in metadata file.
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Data part {} is Outdated and has creation TID {} and CSN {}, "
|
||||
"but does not have removal tid. It's a bug or a result of manual intervention.",
|
||||
(*it)->name, (*it)->version.creation_tid, creation_csn);
|
||||
}
|
||||
modifyPartState(it, DataPartState::Outdated);
|
||||
removePartContributionToDataVolume(*it);
|
||||
};
|
||||
|
||||
/// All parts are in "Active" state after loading
|
||||
assert(std::find_if(data_parts_by_state_and_info.begin(), data_parts_by_state_and_info.end(),
|
||||
[](const auto & part)
|
||||
{
|
||||
return part->getState() != DataPartState::Active;
|
||||
}) == data_parts_by_state_and_info.end());
|
||||
|
||||
bool have_parts_with_version_metadata = false;
|
||||
auto iter = data_parts_by_state_and_info.begin();
|
||||
while (iter != data_parts_by_state_and_info.end() && (*iter)->getState() == DataPartState::Active)
|
||||
{
|
||||
const DataPartPtr & part = *iter;
|
||||
part->loadVersionMetadata();
|
||||
VersionMetadata & version = part->version;
|
||||
if (part->wasInvolvedInTransaction())
|
||||
{
|
||||
have_parts_with_version_metadata = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
++iter;
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Check if CSNs were witten after committing transaction, update and write if needed.
|
||||
bool version_updated = false;
|
||||
assert(!version.creation_tid.isEmpty());
|
||||
if (!part->version.creation_csn)
|
||||
{
|
||||
auto min = TransactionLog::getCSN(version.creation_tid);
|
||||
if (!min)
|
||||
{
|
||||
/// Transaction that created this part was not committed. Remove part.
|
||||
TransactionLog::assertTIDIsNotOutdated(version.creation_tid);
|
||||
min = Tx::RolledBackCSN;
|
||||
}
|
||||
LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has creation_tid={}, setting creation_csn={}",
|
||||
part->name, version.creation_tid, min);
|
||||
version.creation_csn = min;
|
||||
version_updated = true;
|
||||
}
|
||||
if (!version.removal_tid.isEmpty() && !part->version.removal_csn)
|
||||
{
|
||||
auto max = TransactionLog::getCSN(version.removal_tid);
|
||||
if (max)
|
||||
{
|
||||
LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has removal_tid={}, setting removal_csn={}",
|
||||
part->name, version.removal_tid, max);
|
||||
version.removal_csn = max;
|
||||
}
|
||||
else
|
||||
{
|
||||
TransactionLog::assertTIDIsNotOutdated(version.removal_tid);
|
||||
/// Transaction that tried to remove this part was not committed. Clear removal_tid.
|
||||
LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: clearing removal_tid={}",
|
||||
part->name, version.removal_tid);
|
||||
version.unlockRemovalTID(version.removal_tid, TransactionInfoContext{getStorageID(), part->name});
|
||||
}
|
||||
version_updated = true;
|
||||
}
|
||||
|
||||
/// Sanity checks
|
||||
bool csn_order = !version.removal_csn || version.creation_csn <= version.removal_csn || version.removal_csn == Tx::PrehistoricCSN;
|
||||
bool min_start_csn_order = version.creation_tid.start_csn <= version.creation_csn;
|
||||
bool max_start_csn_order = version.removal_tid.start_csn <= version.removal_csn;
|
||||
bool creation_csn_known = version.creation_csn;
|
||||
if (!csn_order || !min_start_csn_order || !max_start_csn_order || !creation_csn_known)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} has invalid version metadata: {}", part->name, version.toString());
|
||||
|
||||
if (version_updated)
|
||||
part->storeVersionMetadata();
|
||||
|
||||
/// Deactivate part if creation was not committed or if removal was.
|
||||
if (version.creation_csn == Tx::RolledBackCSN || version.removal_csn)
|
||||
{
|
||||
auto next_it = std::next(iter);
|
||||
deactivate_part(iter);
|
||||
iter = next_it;
|
||||
}
|
||||
else
|
||||
{
|
||||
++iter;
|
||||
}
|
||||
}
|
||||
|
||||
if (have_parts_with_version_metadata)
|
||||
transactions_enabled.store(true);
|
||||
|
||||
/// Delete from the set of current parts those parts that are covered by another part (those parts that
|
||||
/// were merged), but that for some reason are still not deleted from the filesystem.
|
||||
/// Deletion of files will be performed later in the clearOldParts() method.
|
||||
|
||||
if (data_parts_indexes.size() >= 2)
|
||||
auto active_parts_range = getDataPartsStateRange(DataPartState::Active);
|
||||
auto prev_it = active_parts_range.begin();
|
||||
auto end_it = active_parts_range.end();
|
||||
|
||||
bool less_than_two_active_parts = prev_it == end_it || std::next(prev_it) == end_it;
|
||||
|
||||
if (!less_than_two_active_parts)
|
||||
{
|
||||
/// Now all parts are committed, so data_parts_by_state_and_info == committed_parts_range
|
||||
auto prev_jt = data_parts_by_state_and_info.begin();
|
||||
auto curr_jt = std::next(prev_jt);
|
||||
(*prev_it)->assertState({DataPartState::Active});
|
||||
auto curr_it = std::next(prev_it);
|
||||
|
||||
auto deactivate_part = [&] (DataPartIteratorByStateAndInfo it)
|
||||
while (curr_it != data_parts_by_state_and_info.end() && (*curr_it)->getState() == DataPartState::Active)
|
||||
{
|
||||
(*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed);
|
||||
modifyPartState(it, DataPartState::Outdated);
|
||||
removePartContributionToDataVolume(*it);
|
||||
};
|
||||
(*curr_it)->assertState({DataPartState::Active});
|
||||
|
||||
(*prev_jt)->assertState({DataPartState::Active});
|
||||
|
||||
while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Active)
|
||||
{
|
||||
/// Don't consider data parts belonging to different partitions.
|
||||
if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id)
|
||||
if ((*curr_it)->info.partition_id != (*prev_it)->info.partition_id)
|
||||
{
|
||||
++prev_jt;
|
||||
++curr_jt;
|
||||
++prev_it;
|
||||
++curr_it;
|
||||
continue;
|
||||
}
|
||||
|
||||
if ((*curr_jt)->contains(**prev_jt))
|
||||
if ((*curr_it)->contains(**prev_it))
|
||||
{
|
||||
deactivate_part(prev_jt);
|
||||
prev_jt = curr_jt;
|
||||
++curr_jt;
|
||||
deactivate_part(prev_it);
|
||||
prev_it = curr_it;
|
||||
++curr_it;
|
||||
}
|
||||
else if ((*prev_jt)->contains(**curr_jt))
|
||||
else if ((*prev_it)->contains(**curr_it))
|
||||
{
|
||||
auto next = std::next(curr_jt);
|
||||
deactivate_part(curr_jt);
|
||||
curr_jt = next;
|
||||
auto next = std::next(curr_it);
|
||||
deactivate_part(curr_it);
|
||||
curr_it = next;
|
||||
}
|
||||
else
|
||||
{
|
||||
++prev_jt;
|
||||
++curr_jt;
|
||||
++prev_it;
|
||||
++curr_it;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1482,12 +1586,20 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force)
|
||||
{
|
||||
const DataPartPtr & part = *it;
|
||||
|
||||
/// Do not remove outdated part if it may be visible for some transaction
|
||||
if (!part->version.canBeRemoved())
|
||||
continue;
|
||||
|
||||
auto part_remove_time = part->remove_time.load(std::memory_order_relaxed);
|
||||
|
||||
if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example).
|
||||
((part_remove_time < now &&
|
||||
now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force
|
||||
|| isInMemoryPart(part))) /// Remove in-memory parts immediately to not store excessive data in RAM
|
||||
/// Grab only parts that are not used by anyone (SELECTs for example).
|
||||
if (!part.unique())
|
||||
continue;
|
||||
|
||||
if ((part_remove_time < now && now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds())
|
||||
|| force
|
||||
|| isInMemoryPart(part) /// Remove in-memory parts immediately to not store excessive data in RAM
|
||||
|| (part->version.creation_csn == Tx::RolledBackCSN && getSettings()->remove_rolled_back_parts_immediately))
|
||||
{
|
||||
parts_to_delete.emplace_back(it);
|
||||
}
|
||||
@ -1578,7 +1690,7 @@ void MergeTreeData::flushAllInMemoryPartsIfNeeded()
|
||||
return;
|
||||
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
DataPartsVector parts = getDataPartsVector();
|
||||
DataPartsVector parts = getDataPartsVectorForInternalUsage();
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if (auto part_in_memory = asInMemoryPart(part))
|
||||
@ -1641,7 +1753,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
|
||||
|
||||
size_t MergeTreeData::clearOldWriteAheadLogs()
|
||||
{
|
||||
DataPartsVector parts = getDataPartsVector();
|
||||
DataPartsVector parts = getDataPartsVectorForInternalUsage();
|
||||
std::vector<std::pair<Int64, Int64>> all_block_numbers_on_disk;
|
||||
std::vector<std::pair<Int64, Int64>> block_numbers_on_disk;
|
||||
|
||||
@ -1706,14 +1818,19 @@ size_t MergeTreeData::clearEmptyParts()
|
||||
return 0;
|
||||
|
||||
size_t cleared_count = 0;
|
||||
auto parts = getDataPartsVector();
|
||||
auto parts = getDataPartsVectorForInternalUsage();
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if (part->rows_count == 0)
|
||||
{
|
||||
dropPartNoWaitNoThrow(part->name);
|
||||
++cleared_count;
|
||||
}
|
||||
if (part->rows_count != 0)
|
||||
continue;
|
||||
|
||||
/// Do not try to drop uncommitted parts.
|
||||
if (!part->version.getCreationTID().isPrehistoric() && !part->version.isVisible(TransactionLog::instance().getLatestSnapshot()))
|
||||
continue;
|
||||
|
||||
LOG_TRACE(log, "Will drop empty part {}", part->name);
|
||||
dropPartNoWaitNoThrow(part->name);
|
||||
++cleared_count;
|
||||
}
|
||||
return cleared_count;
|
||||
}
|
||||
@ -2215,7 +2332,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & part : getDataPartsVector())
|
||||
for (const auto & part : getDataPartsVectorForInternalUsage())
|
||||
{
|
||||
bool at_least_one_column_rest = false;
|
||||
for (const auto & column : part->getColumns())
|
||||
@ -2500,6 +2617,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace(
|
||||
|
||||
bool MergeTreeData::renameTempPartAndAdd(
|
||||
MutableDataPartPtr & part,
|
||||
MergeTreeTransaction * txn,
|
||||
SimpleIncrement * increment,
|
||||
Transaction * out_transaction,
|
||||
MergeTreeDeduplicationLog * deduplication_log,
|
||||
@ -2512,7 +2630,7 @@ bool MergeTreeData::renameTempPartAndAdd(
|
||||
DataPartsVector covered_parts;
|
||||
{
|
||||
auto lock = lockParts();
|
||||
if (!renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts, deduplication_log, deduplication_token))
|
||||
if (!renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts, deduplication_log, deduplication_token))
|
||||
return false;
|
||||
}
|
||||
if (!covered_parts.empty())
|
||||
@ -2525,6 +2643,7 @@ bool MergeTreeData::renameTempPartAndAdd(
|
||||
|
||||
bool MergeTreeData::renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part,
|
||||
MergeTreeTransaction * txn,
|
||||
SimpleIncrement * increment,
|
||||
Transaction * out_transaction,
|
||||
std::unique_lock<std::mutex> & lock,
|
||||
@ -2536,6 +2655,9 @@ bool MergeTreeData::renameTempPartAndReplace(
|
||||
throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (txn)
|
||||
transactions_enabled.store(true);
|
||||
|
||||
part->assertState({DataPartState::Temporary});
|
||||
|
||||
MergeTreePartInfo part_info = part->info;
|
||||
@ -2578,7 +2700,6 @@ bool MergeTreeData::renameTempPartAndReplace(
|
||||
|
||||
DataPartPtr covering_part;
|
||||
DataPartsVector covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock);
|
||||
DataPartsVector covered_parts_in_memory;
|
||||
|
||||
if (covering_part)
|
||||
{
|
||||
@ -2614,6 +2735,9 @@ bool MergeTreeData::renameTempPartAndReplace(
|
||||
part->renameTo(part_name, true);
|
||||
|
||||
auto part_it = data_parts_indexes.insert(part).first;
|
||||
/// FIXME Transactions: it's not the best place for checking and setting removal_tid,
|
||||
/// because it's too optimistic. We should lock removal_tid of covered parts at the beginning of operation.
|
||||
MergeTreeTransaction::addNewPartAndRemoveCovered(shared_from_this(), part, covered_parts, txn);
|
||||
|
||||
if (out_transaction)
|
||||
{
|
||||
@ -2669,7 +2793,8 @@ bool MergeTreeData::renameTempPartAndReplace(
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, MergeTreeDeduplicationLog * deduplication_log)
|
||||
MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment,
|
||||
Transaction * out_transaction, MergeTreeDeduplicationLog * deduplication_log)
|
||||
{
|
||||
if (out_transaction && &out_transaction->data != this)
|
||||
throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.",
|
||||
@ -2678,18 +2803,25 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
DataPartsVector covered_parts;
|
||||
{
|
||||
auto lock = lockParts();
|
||||
renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts, deduplication_log);
|
||||
renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts, deduplication_log);
|
||||
}
|
||||
return covered_parts;
|
||||
}
|
||||
|
||||
void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & acquired_lock)
|
||||
void MergeTreeData::removePartsFromWorkingSet(MergeTreeTransaction * txn, const MergeTreeData::DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & acquired_lock)
|
||||
|
||||
{
|
||||
if (txn)
|
||||
transactions_enabled.store(true);
|
||||
|
||||
auto remove_time = clear_without_timeout ? 0 : time(nullptr);
|
||||
bool removed_active_part = false;
|
||||
|
||||
for (const DataPartPtr & part : remove)
|
||||
{
|
||||
if (part->version.creation_csn != Tx::RolledBackCSN)
|
||||
MergeTreeTransaction::removeOldPart(shared_from_this(), part, txn);
|
||||
|
||||
if (part->getState() == IMergeTreeDataPart::State::Active)
|
||||
{
|
||||
removePartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
@ -2714,7 +2846,6 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect
|
||||
void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove)
|
||||
{
|
||||
auto lock = lockParts();
|
||||
bool removed_active_part = false;
|
||||
|
||||
for (const auto & part : remove)
|
||||
{
|
||||
@ -2722,19 +2853,16 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con
|
||||
if (it_part == data_parts_by_info.end())
|
||||
throw Exception("Part " + part->getNameWithState() + " not found in data_parts", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (part->getState() == IMergeTreeDataPart::State::Active)
|
||||
removed_active_part = true;
|
||||
assert(part->getState() == IMergeTreeDataPart::State::PreActive);
|
||||
|
||||
modifyPartState(part, IMergeTreeDataPart::State::Temporary);
|
||||
/// Erase immediately
|
||||
data_parts_indexes.erase(it_part);
|
||||
}
|
||||
|
||||
if (removed_active_part)
|
||||
resetObjectColumnsFromActiveParts(lock);
|
||||
}
|
||||
|
||||
void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock)
|
||||
void MergeTreeData::removePartsFromWorkingSet(
|
||||
MergeTreeTransaction * txn, const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock)
|
||||
{
|
||||
auto lock = (acquired_lock) ? DataPartsLock() : lockParts();
|
||||
|
||||
@ -2746,11 +2874,12 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo
|
||||
part->assertState({DataPartState::PreActive, DataPartState::Active, DataPartState::Outdated});
|
||||
}
|
||||
|
||||
removePartsFromWorkingSet(remove, clear_without_timeout, lock);
|
||||
removePartsFromWorkingSet(txn, remove, clear_without_timeout, lock);
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(const MergeTreePartInfo & drop_range, bool clear_without_timeout,
|
||||
DataPartsLock & lock)
|
||||
MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(
|
||||
MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range,
|
||||
bool clear_without_timeout, DataPartsLock & lock)
|
||||
{
|
||||
DataPartsVector parts_to_remove;
|
||||
|
||||
@ -2815,15 +2944,34 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c
|
||||
part->name, drop_range.getPartName());
|
||||
}
|
||||
|
||||
if (part->getState() != DataPartState::Deleting)
|
||||
parts_to_remove.emplace_back(part);
|
||||
if (part->getState() == DataPartState::Deleting)
|
||||
continue;
|
||||
|
||||
/// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice
|
||||
if (txn)
|
||||
{
|
||||
if (!part->version.isVisible(*txn))
|
||||
continue;
|
||||
}
|
||||
|
||||
parts_to_remove.emplace_back(part);
|
||||
}
|
||||
|
||||
removePartsFromWorkingSet(parts_to_remove, clear_without_timeout, lock);
|
||||
removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock);
|
||||
|
||||
return parts_to_remove;
|
||||
}
|
||||
|
||||
void MergeTreeData::restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock)
|
||||
{
|
||||
auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); //-V1018
|
||||
if (part->getState() == DataPartState::Active)
|
||||
return;
|
||||
addPartContributionToColumnAndSecondaryIndexSizes(part);
|
||||
addPartContributionToDataVolume(part);
|
||||
modifyPartState(part, DataPartState::Active);
|
||||
}
|
||||
|
||||
void MergeTreeData::forgetPartAndMoveToDetached(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered)
|
||||
{
|
||||
if (prefix.empty())
|
||||
@ -3238,9 +3386,23 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String &
|
||||
return getActiveContainingPart(part_info);
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(MergeTreeData::DataPartState state, const String & partition_id) const
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const
|
||||
{
|
||||
DataPartStateAndPartitionID state_with_partition{state, partition_id};
|
||||
if (const auto * txn = local_context->getCurrentTransaction().get())
|
||||
{
|
||||
DataPartStateAndPartitionID active_parts{MergeTreeDataPartState::Active, partition_id};
|
||||
DataPartStateAndPartitionID outdated_parts{MergeTreeDataPartState::Outdated, partition_id};
|
||||
DataPartsVector res;
|
||||
{
|
||||
auto lock = lockParts();
|
||||
res.insert(res.end(), data_parts_by_state_and_info.lower_bound(active_parts), data_parts_by_state_and_info.upper_bound(active_parts));
|
||||
res.insert(res.end(), data_parts_by_state_and_info.lower_bound(outdated_parts), data_parts_by_state_and_info.upper_bound(outdated_parts));
|
||||
}
|
||||
filterVisibleDataParts(res, txn->getSnapshot(), txn->tid);
|
||||
return res;
|
||||
}
|
||||
|
||||
DataPartStateAndPartitionID state_with_partition{MergeTreeDataPartState::Active, partition_id};
|
||||
|
||||
auto lock = lockParts();
|
||||
return DataPartsVector(
|
||||
@ -3248,19 +3410,37 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(Merg
|
||||
data_parts_by_state_and_info.upper_bound(state_with_partition));
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartitions(MergeTreeData::DataPartState state, const std::unordered_set<String> & partition_ids) const
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set<String> & partition_ids) const
|
||||
{
|
||||
auto lock = lockParts();
|
||||
auto txn = local_context->getCurrentTransaction();
|
||||
DataPartsVector res;
|
||||
for (const auto & partition_id : partition_ids)
|
||||
{
|
||||
DataPartStateAndPartitionID state_with_partition{state, partition_id};
|
||||
insertAtEnd(
|
||||
res,
|
||||
DataPartsVector(
|
||||
data_parts_by_state_and_info.lower_bound(state_with_partition),
|
||||
data_parts_by_state_and_info.upper_bound(state_with_partition)));
|
||||
auto lock = lockParts();
|
||||
for (const auto & partition_id : partition_ids)
|
||||
{
|
||||
DataPartStateAndPartitionID active_parts{MergeTreeDataPartState::Active, partition_id};
|
||||
insertAtEnd(
|
||||
res,
|
||||
DataPartsVector(
|
||||
data_parts_by_state_and_info.lower_bound(active_parts),
|
||||
data_parts_by_state_and_info.upper_bound(active_parts)));
|
||||
|
||||
if (txn)
|
||||
{
|
||||
DataPartStateAndPartitionID outdated_parts{MergeTreeDataPartState::Active, partition_id};
|
||||
|
||||
insertAtEnd(
|
||||
res,
|
||||
DataPartsVector(
|
||||
data_parts_by_state_and_info.lower_bound(outdated_parts),
|
||||
data_parts_by_state_and_info.upper_bound(outdated_parts)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (txn)
|
||||
filterVisibleDataParts(res, txn->getSnapshot(), txn->tid);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -3292,6 +3472,8 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
|
||||
|
||||
part->loadColumnsChecksumsIndexes(false, true);
|
||||
part->modification_time = disk->getLastModified(full_part_path).epochTime();
|
||||
disk->removeFileIfExists(fs::path(full_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
|
||||
disk->removeFileIfExists(fs::path(full_part_path) / IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME);
|
||||
}
|
||||
|
||||
void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl()
|
||||
@ -3398,16 +3580,16 @@ void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
|
||||
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context)
|
||||
{
|
||||
DataPartsVector parts_to_remove;
|
||||
const auto * partition_ast = partition->as<ASTPartition>();
|
||||
if (partition_ast && partition_ast->all)
|
||||
parts_to_remove = getDataPartsVector();
|
||||
parts_to_remove = getVisibleDataPartsVector(local_context);
|
||||
else
|
||||
{
|
||||
const String partition_id = getPartitionIDFromQuery(partition, getContext());
|
||||
parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
const String partition_id = getPartitionIDFromQuery(partition, local_context);
|
||||
parts_to_remove = getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
}
|
||||
UInt64 partition_size = 0;
|
||||
|
||||
@ -3446,7 +3628,7 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String &
|
||||
throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART);
|
||||
}
|
||||
else
|
||||
parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
parts = getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
|
||||
auto disk = getStoragePolicy()->getDiskByName(name);
|
||||
parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr)
|
||||
@ -3488,7 +3670,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
|
||||
throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART);
|
||||
}
|
||||
else
|
||||
parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
parts = getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
|
||||
auto volume = getStoragePolicy()->getVolumeByName(name);
|
||||
if (!volume)
|
||||
@ -3560,7 +3742,7 @@ Pipe MergeTreeData::alterPartition(
|
||||
}
|
||||
else
|
||||
{
|
||||
checkPartitionCanBeDropped(command.partition);
|
||||
checkPartitionCanBeDropped(command.partition, query_context);
|
||||
dropPartition(command.partition, command.detach, query_context);
|
||||
}
|
||||
}
|
||||
@ -3609,7 +3791,7 @@ Pipe MergeTreeData::alterPartition(
|
||||
case PartitionCommand::REPLACE_PARTITION:
|
||||
{
|
||||
if (command.replace)
|
||||
checkPartitionCanBeDropped(command.partition);
|
||||
checkPartitionCanBeDropped(command.partition, query_context);
|
||||
String from_database = query_context->resolveDatabase(command.from_database);
|
||||
auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context);
|
||||
replacePartitionFrom(from_storage, command.partition, command.replace, query_context);
|
||||
@ -3668,9 +3850,9 @@ BackupEntries MergeTreeData::backupData(ContextPtr local_context, const ASTs & p
|
||||
{
|
||||
DataPartsVector data_parts;
|
||||
if (partitions.empty())
|
||||
data_parts = getDataPartsVector();
|
||||
data_parts = getVisibleDataPartsVector(local_context);
|
||||
else
|
||||
data_parts = getDataPartsVectorInPartitions(MergeTreeDataPartState::Active, getPartitionIDsFromQuery(partitions, local_context));
|
||||
data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(partitions, local_context));
|
||||
return backupDataParts(data_parts);
|
||||
}
|
||||
|
||||
@ -3807,8 +3989,11 @@ private:
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
|
||||
auto part = storage->createPart(part_name, part_info, single_disk_volume, relative_temp_part_dir);
|
||||
/// TODO Transactions: Decide what to do with version metadata (if any). Let's just remove it for now.
|
||||
disk->removeFileIfExists(fs::path(temp_part_dir) / IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME);
|
||||
part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
|
||||
part->loadColumnsChecksumsIndexes(false, true);
|
||||
storage->renameTempPartAndAdd(part, increment);
|
||||
storage->renameTempPartAndAdd(part, NO_TRANSACTION_RAW, increment);
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -3928,6 +4113,86 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
|
||||
return partition_id;
|
||||
}
|
||||
|
||||
|
||||
DataPartsVector MergeTreeData::getVisibleDataPartsVector(ContextPtr local_context) const
|
||||
{
|
||||
DataPartsVector res;
|
||||
if (const auto * txn = local_context->getCurrentTransaction().get())
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated});
|
||||
filterVisibleDataParts(res, txn->getSnapshot(), txn->tid);
|
||||
}
|
||||
else
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
DataPartsVector MergeTreeData::getVisibleDataPartsVectorUnlocked(ContextPtr local_context, const DataPartsLock & lock) const
|
||||
{
|
||||
DataPartsVector res;
|
||||
if (const auto * txn = local_context->getCurrentTransaction().get())
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated}, lock);
|
||||
filterVisibleDataParts(res, txn->getSnapshot(), txn->tid);
|
||||
}
|
||||
else
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage({DataPartState::Active}, lock);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const
|
||||
{
|
||||
DataPartsVector res;
|
||||
if (txn)
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated});
|
||||
filterVisibleDataParts(res, txn->getSnapshot(), txn->tid);
|
||||
}
|
||||
else
|
||||
{
|
||||
res = getDataPartsVectorForInternalUsage();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(CSN snapshot_version, TransactionID current_tid) const
|
||||
{
|
||||
auto res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated});
|
||||
filterVisibleDataParts(res, snapshot_version, current_tid);
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts, CSN snapshot_version, TransactionID current_tid) const
|
||||
{
|
||||
[[maybe_unused]] size_t total_size = maybe_visible_parts.size();
|
||||
|
||||
auto need_remove_pred = [snapshot_version, ¤t_tid] (const DataPartPtr & part) -> bool
|
||||
{
|
||||
return !part->version.isVisible(snapshot_version, current_tid);
|
||||
};
|
||||
|
||||
auto new_end_it = std::remove_if(maybe_visible_parts.begin(), maybe_visible_parts.end(), need_remove_pred);
|
||||
maybe_visible_parts.erase(new_end_it, maybe_visible_parts.end());
|
||||
[[maybe_unused]] size_t visible_size = maybe_visible_parts.size();
|
||||
|
||||
|
||||
auto get_part_names = [&maybe_visible_parts]() -> Strings
|
||||
{
|
||||
Strings visible_part_names;
|
||||
for (const auto & p : maybe_visible_parts)
|
||||
visible_part_names.push_back(p->name);
|
||||
return visible_part_names;
|
||||
};
|
||||
|
||||
LOG_TEST(log, "Got {} parts (of {}) visible in snapshot {} (TID {}): {}",
|
||||
visible_size, total_size, snapshot_version, current_tid, fmt::join(get_part_names(), ", "));
|
||||
}
|
||||
|
||||
|
||||
std::unordered_set<String> MergeTreeData::getPartitionIDsFromQuery(const ASTs & asts, ContextPtr local_context) const
|
||||
{
|
||||
std::unordered_set<String> partition_ids;
|
||||
@ -3958,7 +4223,7 @@ std::set<String> MergeTreeData::getPartitionIdsAffectedByCommands(
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorUnlocked(
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage(
|
||||
const DataPartStates & affordable_states,
|
||||
const DataPartsLock & /*lock*/,
|
||||
DataPartStateVector * out_states,
|
||||
@ -4005,13 +4270,13 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorUnlocked(
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage(
|
||||
const DataPartStates & affordable_states,
|
||||
DataPartStateVector * out_states,
|
||||
bool require_projection_parts) const
|
||||
{
|
||||
auto lock = lockParts();
|
||||
return getDataPartsVectorUnlocked(affordable_states, lock, out_states, require_projection_parts);
|
||||
return getDataPartsVectorForInternalUsage(affordable_states, lock, out_states, require_projection_parts);
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector
|
||||
@ -4371,14 +4636,14 @@ MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affo
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeData::DataParts MergeTreeData::getDataParts() const
|
||||
MergeTreeData::DataParts MergeTreeData::getDataPartsForInternalUsage() const
|
||||
{
|
||||
return getDataParts({DataPartState::Active});
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage() const
|
||||
{
|
||||
return getDataPartsVector({DataPartState::Active});
|
||||
return getDataPartsVectorForInternalUsage({DataPartState::Active});
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition(
|
||||
@ -4422,7 +4687,19 @@ void MergeTreeData::Transaction::rollback()
|
||||
buf << ".";
|
||||
LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str());
|
||||
|
||||
data.removePartsFromWorkingSet(
|
||||
if (!txn)
|
||||
{
|
||||
auto lock = data.lockParts();
|
||||
for (const auto & part : precommitted_parts)
|
||||
{
|
||||
DataPartPtr covering_part;
|
||||
DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, lock);
|
||||
for (auto & covered : covered_parts)
|
||||
covered->version.unlockRemovalTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name});
|
||||
}
|
||||
}
|
||||
|
||||
data.removePartsFromWorkingSet(txn,
|
||||
DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()),
|
||||
/* clear_without_timeout = */ true);
|
||||
}
|
||||
@ -5413,7 +5690,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
|
||||
const MergeTreeData::DataPartPtr & src_part,
|
||||
const String & tmp_part_prefix,
|
||||
const MergeTreePartInfo & dst_part_info,
|
||||
const StorageMetadataPtr & metadata_snapshot)
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeTransactionPtr & txn)
|
||||
{
|
||||
/// Check that the storage policy contains the disk where the src_part is located.
|
||||
bool does_storage_policy_allow_same_disk = false;
|
||||
@ -5431,6 +5709,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String dst_part_name = src_part->getNewName(dst_part_info);
|
||||
assert(!tmp_part_prefix.empty());
|
||||
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
|
||||
|
||||
auto reservation = reserveSpace(src_part->getBytesOnDisk(), src_part->volume->getDisk());
|
||||
@ -5451,12 +5730,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path));
|
||||
localBackup(disk, src_part_path, dst_part_path);
|
||||
localBackup(disk, src_part_path, dst_part_path, /* make_source_readonly */ false);
|
||||
disk->removeFileIfExists(fs::path(dst_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
|
||||
disk->removeFileIfExists(fs::path(dst_part_path) / IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME);
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
|
||||
auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name);
|
||||
|
||||
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
|
||||
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
|
||||
dst_data_part->version.setCreationTID(tid, nullptr);
|
||||
dst_data_part->storeVersionMetadata();
|
||||
|
||||
dst_data_part->is_temp = true;
|
||||
|
||||
dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
||||
@ -5505,7 +5790,7 @@ void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) con
|
||||
if (data_part->volume && data_part->volume->getDisk()->isBroken())
|
||||
{
|
||||
auto disk = data_part->volume->getDisk();
|
||||
auto parts = getDataParts();
|
||||
auto parts = getDataPartsForInternalUsage();
|
||||
LOG_WARNING(log, "Scanning parts to recover on broken disk {}.", disk->getName() + "@" + disk->getPath());
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
@ -5580,7 +5865,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
|
||||
const String shadow_path = "shadow/";
|
||||
|
||||
/// Acquire a snapshot of active data parts to prevent removing while doing backup.
|
||||
const auto data_parts = getDataParts();
|
||||
const auto data_parts = getVisibleDataPartsVector(local_context);
|
||||
|
||||
String backup_name = (!with_name.empty() ? escapeForFileName(with_name) : toString(increment));
|
||||
String backup_path = fs::path(shadow_path) / backup_name / "";
|
||||
@ -6323,12 +6608,12 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart
|
||||
DB::updateObjectColumns(object_columns, part->getColumns());
|
||||
}
|
||||
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const
|
||||
{
|
||||
auto snapshot_data = std::make_unique<SnapshotData>();
|
||||
|
||||
auto lock = lockParts();
|
||||
snapshot_data->parts = getDataPartsVectorUnlocked({DataPartState::Active}, lock);
|
||||
snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock);
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
|
||||
}
|
||||
|
||||
|
@ -49,6 +49,7 @@ class MutationCommands;
|
||||
class Context;
|
||||
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
||||
struct JobAndPool;
|
||||
class MergeTreeTransaction;
|
||||
struct ZeroCopyLock;
|
||||
|
||||
/// Auxiliary struct holding information about the future merged or mutated part.
|
||||
@ -247,7 +248,7 @@ public:
|
||||
class Transaction : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
explicit Transaction(MergeTreeData & data_) : data(data_) {}
|
||||
Transaction(MergeTreeData & data_, MergeTreeTransaction * txn_) : data(data_), txn(txn_) {}
|
||||
|
||||
DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr);
|
||||
|
||||
@ -276,6 +277,7 @@ public:
|
||||
friend class MergeTreeData;
|
||||
|
||||
MergeTreeData & data;
|
||||
MergeTreeTransaction * txn;
|
||||
DataParts precommitted_parts;
|
||||
|
||||
void clear() { precommitted_parts.clear(); }
|
||||
@ -436,7 +438,7 @@ public:
|
||||
DataPartsVector parts;
|
||||
};
|
||||
|
||||
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;
|
||||
|
||||
/// Load the set of data parts from disk. Call once - immediately after the object is created.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
@ -445,10 +447,11 @@ public:
|
||||
|
||||
Int64 getMaxBlockNumber() const;
|
||||
|
||||
|
||||
/// Returns a copy of the list so that the caller shouldn't worry about locks.
|
||||
DataParts getDataParts(const DataPartStates & affordable_states) const;
|
||||
|
||||
DataPartsVector getDataPartsVectorUnlocked(
|
||||
DataPartsVector getDataPartsVectorForInternalUsage(
|
||||
const DataPartStates & affordable_states,
|
||||
const DataPartsLock & lock,
|
||||
DataPartStateVector * out_states = nullptr,
|
||||
@ -456,31 +459,28 @@ public:
|
||||
|
||||
/// Returns sorted list of the parts with specified states
|
||||
/// out_states will contain snapshot of each part state
|
||||
DataPartsVector getDataPartsVector(
|
||||
const DataPartStates & affordable_states,
|
||||
DataPartStateVector * out_states = nullptr,
|
||||
bool require_projection_parts = false) const;
|
||||
DataPartsVector getDataPartsVectorForInternalUsage(
|
||||
const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr, bool require_projection_parts = false) const;
|
||||
|
||||
|
||||
/// Returns absolutely all parts (and snapshot of their states)
|
||||
DataPartsVector getAllDataPartsVector(
|
||||
DataPartStateVector * out_states = nullptr,
|
||||
bool require_projection_parts = false) const;
|
||||
|
||||
/// Returns all detached parts
|
||||
DetachedPartsInfo getDetachedParts() const;
|
||||
/// Returns parts in Active state
|
||||
DataParts getDataPartsForInternalUsage() const;
|
||||
DataPartsVector getDataPartsVectorForInternalUsage() const;
|
||||
|
||||
static void validateDetachedPartName(const String & name);
|
||||
void filterVisibleDataParts(DataPartsVector & maybe_visible_parts, CSN snapshot_version, TransactionID current_tid) const;
|
||||
|
||||
void dropDetached(const ASTPtr & partition, bool part, ContextPtr local_context);
|
||||
/// Returns parts that visible with current snapshot
|
||||
DataPartsVector getVisibleDataPartsVector(ContextPtr local_context) const;
|
||||
DataPartsVector getVisibleDataPartsVectorUnlocked(ContextPtr local_context, const DataPartsLock & lock) const;
|
||||
DataPartsVector getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const;
|
||||
DataPartsVector getVisibleDataPartsVector(CSN snapshot_version, TransactionID current_tid) const;
|
||||
|
||||
MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part,
|
||||
ContextPtr context, PartsTemporaryRename & renamed_parts);
|
||||
|
||||
/// Returns Active parts
|
||||
DataParts getDataParts() const;
|
||||
DataPartsVector getDataPartsVector() const;
|
||||
|
||||
/// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr.
|
||||
/// Returns a part in Active state with the given name or a part containing it. If there is no such part, returns nullptr.
|
||||
DataPartPtr getActiveContainingPart(const String & part_name) const;
|
||||
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info) const;
|
||||
DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock & lock) const;
|
||||
@ -490,8 +490,8 @@ public:
|
||||
void swapActivePart(MergeTreeData::DataPartPtr part_copy);
|
||||
|
||||
/// Returns all parts in specified partition
|
||||
DataPartsVector getDataPartsVectorInPartition(DataPartState state, const String & partition_id) const;
|
||||
DataPartsVector getDataPartsVectorInPartitions(DataPartState state, const std::unordered_set<String> & partition_ids) const;
|
||||
DataPartsVector getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const;
|
||||
DataPartsVector getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set<String> & partition_ids) const;
|
||||
|
||||
/// Returns the part with the given name and state or nullptr if no such part.
|
||||
DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states);
|
||||
@ -511,6 +511,18 @@ public:
|
||||
/// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition.
|
||||
std::optional<Int64> getMinPartDataVersion() const;
|
||||
|
||||
|
||||
/// Returns all detached parts
|
||||
DetachedPartsInfo getDetachedParts() const;
|
||||
|
||||
static void validateDetachedPartName(const String & name);
|
||||
|
||||
void dropDetached(const ASTPtr & partition, bool part, ContextPtr context);
|
||||
|
||||
MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part,
|
||||
ContextPtr context, PartsTemporaryRename & renamed_parts);
|
||||
|
||||
|
||||
/// If the table contains too many active parts, sleep for a while to give them time to merge.
|
||||
/// If until is non-null, wake up from the sleep earlier if the event happened.
|
||||
void delayInsertOrThrowIfNeeded(Poco::Event * until = nullptr) const;
|
||||
@ -524,6 +536,7 @@ public:
|
||||
/// Returns true if part was added. Returns false if part is covered by bigger part.
|
||||
bool renameTempPartAndAdd(
|
||||
MutableDataPartPtr & part,
|
||||
MergeTreeTransaction * txn,
|
||||
SimpleIncrement * increment = nullptr,
|
||||
Transaction * out_transaction = nullptr,
|
||||
MergeTreeDeduplicationLog * deduplication_log = nullptr,
|
||||
@ -533,11 +546,14 @@ public:
|
||||
/// Returns all parts covered by the added part (in ascending order).
|
||||
/// If out_transaction == nullptr, marks covered parts as Outdated.
|
||||
DataPartsVector renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr);
|
||||
MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment = nullptr,
|
||||
Transaction * out_transaction = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr);
|
||||
|
||||
/// Low-level version of previous one, doesn't lock mutex
|
||||
/// FIXME Transactions: remove add_to_txn flag, maybe merge MergeTreeTransaction and Transaction
|
||||
bool renameTempPartAndReplace(
|
||||
MutableDataPartPtr & part,
|
||||
MergeTreeTransaction * txn,
|
||||
SimpleIncrement * increment,
|
||||
Transaction * out_transaction,
|
||||
DataPartsLock & lock,
|
||||
@ -554,15 +570,18 @@ public:
|
||||
/// Parts in add must already be in data_parts with PreActive, Active, or Outdated states.
|
||||
/// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to
|
||||
/// clearOldParts (ignoring old_parts_lifetime).
|
||||
void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock = nullptr);
|
||||
void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & acquired_lock);
|
||||
void removePartsFromWorkingSet(MergeTreeTransaction * txn, const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock = nullptr);
|
||||
void removePartsFromWorkingSet(MergeTreeTransaction * txn, const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock & acquired_lock);
|
||||
|
||||
/// Removes all parts from the working set parts
|
||||
/// for which (partition_id = drop_range.partition_id && min_block >= drop_range.min_block && max_block <= drop_range.max_block).
|
||||
/// Used in REPLACE PARTITION command;
|
||||
DataPartsVector removePartsInRangeFromWorkingSet(const MergeTreePartInfo & drop_range, bool clear_without_timeout,
|
||||
DataPartsVector removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, bool clear_without_timeout,
|
||||
DataPartsLock & lock);
|
||||
|
||||
/// Restores Outdated part and adds it to working set
|
||||
void restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock = nullptr);
|
||||
|
||||
/// Renames the part to detached/<prefix>_<part> and removes it from data_parts,
|
||||
//// so it will not be deleted in clearOldParts.
|
||||
/// If restore_covered is true, adds to the working set inactive parts, which were merged into the deleted part.
|
||||
@ -697,7 +716,10 @@ public:
|
||||
/// Moves partition to specified Volume
|
||||
void movePartitionToVolume(const ASTPtr & partition, const String & name, bool moving_part, ContextPtr context);
|
||||
|
||||
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
|
||||
/// Checks that Partition could be dropped right now
|
||||
/// Otherwise - throws an exception with detailed information.
|
||||
/// We do not use mutex because it is not very important that the size could change during the operation.
|
||||
void checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context);
|
||||
|
||||
void checkPartCanBeDropped(const String & part_name);
|
||||
|
||||
@ -743,7 +765,8 @@ public:
|
||||
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk(
|
||||
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot);
|
||||
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info,
|
||||
const StorageMetadataPtr & metadata_snapshot, const MergeTreeTransactionPtr & txn);
|
||||
|
||||
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;
|
||||
|
||||
@ -978,6 +1001,9 @@ protected:
|
||||
mutable std::shared_mutex pinned_part_uuids_mutex;
|
||||
PinnedPartUUIDsPtr pinned_part_uuids;
|
||||
|
||||
/// True if at least one part was created/removed with transaction.
|
||||
mutable std::atomic_bool transactions_enabled = false;
|
||||
|
||||
/// Work with data parts
|
||||
|
||||
struct TagByInfo{};
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeProgress.h>
|
||||
#include <Storages/MergeTree/MergeTask.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
|
||||
#include <Processors/Transforms/TTLTransform.h>
|
||||
#include <Processors/Transforms/TTLCalcTransform.h>
|
||||
@ -29,6 +30,7 @@
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/interpolate.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -52,6 +54,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ABORTED;
|
||||
}
|
||||
|
||||
/// Do not start to merge parts, if free space is less than sum size of parts times specified coefficient.
|
||||
@ -124,9 +127,70 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
size_t max_total_size_to_merge,
|
||||
const AllowedMergingPredicate & can_merge_callback,
|
||||
bool merge_with_ttl_allowed,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
String * out_disable_reason)
|
||||
{
|
||||
MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
|
||||
MergeTreeData::DataPartsVector data_parts;
|
||||
if (txn)
|
||||
{
|
||||
/// Merge predicate (for simple MergeTree) allows to merge two parts only if both parts are visible for merge transaction.
|
||||
/// So at the first glance we could just get all active parts.
|
||||
/// Active parts include uncommitted parts, but it's ok and merge predicate handles it.
|
||||
/// However, it's possible that some transaction is trying to remove a part in the middle, for example, all_2_2_0.
|
||||
/// If parts all_1_1_0 and all_3_3_0 are active and visible for merge transaction, then we would try to merge them.
|
||||
/// But it's wrong, because all_2_2_0 may become active again if transaction will roll back.
|
||||
/// That's why we must include some outdated parts into `data_part`, more precisely, such parts that removal is not committed.
|
||||
MergeTreeData::DataPartsVector active_parts;
|
||||
MergeTreeData::DataPartsVector outdated_parts;
|
||||
|
||||
{
|
||||
auto lock = data.lockParts();
|
||||
active_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Active}, lock);
|
||||
outdated_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Outdated}, lock);
|
||||
}
|
||||
|
||||
ActiveDataPartSet active_parts_set{data.format_version};
|
||||
for (const auto & part : active_parts)
|
||||
active_parts_set.add(part->name);
|
||||
|
||||
for (const auto & part : outdated_parts)
|
||||
{
|
||||
/// We don't need rolled back parts.
|
||||
/// NOTE When rolling back a transaction we set creation_csn to RolledBackCSN at first
|
||||
/// and then remove part from working set, so there's no race condition
|
||||
if (part->version.creation_csn == Tx::RolledBackCSN)
|
||||
continue;
|
||||
|
||||
/// We don't need parts that are finally removed.
|
||||
/// NOTE There's a minor race condition: we may get UnknownCSN if a transaction has been just committed concurrently.
|
||||
/// But it's not a problem if we will add such part to `data_parts`.
|
||||
if (part->version.removal_csn != Tx::UnknownCSN)
|
||||
continue;
|
||||
|
||||
active_parts_set.add(part->name);
|
||||
}
|
||||
|
||||
/// Restore "active" parts set from selected active and outdated parts
|
||||
auto remove_pred = [&](const MergeTreeData::DataPartPtr & part) -> bool
|
||||
{
|
||||
return active_parts_set.getContainingPart(part->info) != part->name;
|
||||
};
|
||||
|
||||
auto new_end_it = std::remove_if(active_parts.begin(), active_parts.end(), remove_pred);
|
||||
active_parts.erase(new_end_it, active_parts.end());
|
||||
|
||||
new_end_it = std::remove_if(outdated_parts.begin(), outdated_parts.end(), remove_pred);
|
||||
outdated_parts.erase(new_end_it, outdated_parts.end());
|
||||
|
||||
std::merge(active_parts.begin(), active_parts.end(),
|
||||
outdated_parts.begin(), outdated_parts.end(),
|
||||
std::back_inserter(data_parts), MergeTreeData::LessDataPart());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Simply get all active parts
|
||||
data_parts = data.getDataPartsVectorForInternalUsage();
|
||||
}
|
||||
const auto data_settings = data.getSettings();
|
||||
auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||
|
||||
@ -172,7 +236,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
* So we have to check if this part is currently being inserted with quorum and so on and so forth.
|
||||
* Obviously we have to check it manually only for the first part
|
||||
* of each partition because it will be automatically checked for a pair of parts. */
|
||||
if (!can_merge_callback(nullptr, part, nullptr))
|
||||
if (!can_merge_callback(nullptr, part, txn.get(), nullptr))
|
||||
continue;
|
||||
|
||||
/// This part can be merged only with next parts (no prev part exists), so start
|
||||
@ -184,7 +248,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
{
|
||||
/// If we cannot merge with previous part we had to start new parts
|
||||
/// interval (in the same partition)
|
||||
if (!can_merge_callback(*prev_part, part, nullptr))
|
||||
if (!can_merge_callback(*prev_part, part, txn.get(), nullptr))
|
||||
{
|
||||
/// Now we have no previous part
|
||||
prev_part = nullptr;
|
||||
@ -196,7 +260,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
/// for example, merge is already assigned for such parts, or they participate in quorum inserts
|
||||
/// and so on.
|
||||
/// Also we don't start new interval here (maybe all next parts cannot be merged and we don't want to have empty interval)
|
||||
if (!can_merge_callback(nullptr, part, nullptr))
|
||||
if (!can_merge_callback(nullptr, part, txn.get(), nullptr))
|
||||
continue;
|
||||
|
||||
/// Starting new interval in the same partition
|
||||
@ -307,6 +371,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
String * out_disable_reason,
|
||||
bool optimize_skip_merged_partitions)
|
||||
{
|
||||
@ -343,7 +408,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti
|
||||
while (it != parts.end())
|
||||
{
|
||||
/// For the case of one part, we check that it can be merged "with itself".
|
||||
if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, out_disable_reason))
|
||||
if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, txn.get(), out_disable_reason))
|
||||
{
|
||||
return SelectPartsDecision::CANNOT_SELECT;
|
||||
}
|
||||
@ -390,7 +455,7 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::selectAllPartsFromPar
|
||||
{
|
||||
MergeTreeData::DataPartsVector parts_from_partition;
|
||||
|
||||
MergeTreeData::DataParts data_parts = data.getDataParts();
|
||||
MergeTreeData::DataParts data_parts = data.getDataPartsForInternalUsage();
|
||||
|
||||
for (const auto & current_part : data_parts)
|
||||
{
|
||||
@ -416,6 +481,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart(
|
||||
bool deduplicate,
|
||||
const Names & deduplicate_by_columns,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
const IMergeTreeDataPart * parent_part,
|
||||
const String & suffix)
|
||||
{
|
||||
@ -432,6 +498,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart(
|
||||
merging_params,
|
||||
parent_part,
|
||||
suffix,
|
||||
txn,
|
||||
&data,
|
||||
this,
|
||||
&merges_blocker,
|
||||
@ -446,6 +513,7 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart(
|
||||
MergeListEntry * merge_entry,
|
||||
time_t time_of_mutation,
|
||||
ContextPtr context,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
ReservationSharedPtr space_reservation,
|
||||
TableLockHolder & holder)
|
||||
{
|
||||
@ -458,6 +526,7 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart(
|
||||
context,
|
||||
space_reservation,
|
||||
holder,
|
||||
txn,
|
||||
data,
|
||||
*this,
|
||||
merges_blocker
|
||||
@ -508,10 +577,16 @@ MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm(
|
||||
MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart(
|
||||
MergeTreeData::MutableDataPartPtr & new_data_part,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
MergeTreeData::Transaction * out_transaction)
|
||||
{
|
||||
/// Some of source parts was possibly created in transaction, so non-transactional merge may break isolation.
|
||||
if (data.transactions_enabled.load(std::memory_order_relaxed) && !txn)
|
||||
throw Exception(ErrorCodes::ABORTED, "Cancelling merge, because it was done without starting transaction,"
|
||||
"but transactions were enabled for this table");
|
||||
|
||||
/// Rename new part, add to the set and remove original parts.
|
||||
auto replaced_parts = data.renameTempPartAndReplace(new_data_part, nullptr, out_transaction);
|
||||
auto replaced_parts = data.renameTempPartAndReplace(new_data_part, txn.get(), nullptr, out_transaction);
|
||||
|
||||
/// Let's check that all original parts have been deleted and only them.
|
||||
if (replaced_parts.size() != parts.size())
|
||||
|
@ -40,7 +40,10 @@ enum class ExecuteTTLType
|
||||
class MergeTreeDataMergerMutator
|
||||
{
|
||||
public:
|
||||
using AllowedMergingPredicate = std::function<bool (const MergeTreeData::DataPartPtr &, const MergeTreeData::DataPartPtr &, String *)>;
|
||||
using AllowedMergingPredicate = std::function<bool (const MergeTreeData::DataPartPtr &,
|
||||
const MergeTreeData::DataPartPtr &,
|
||||
const MergeTreeTransaction *,
|
||||
String *)>;
|
||||
|
||||
MergeTreeDataMergerMutator(MergeTreeData & data_, size_t max_tasks_count_);
|
||||
|
||||
@ -72,6 +75,7 @@ public:
|
||||
size_t max_total_size_to_merge,
|
||||
const AllowedMergingPredicate & can_merge,
|
||||
bool merge_with_ttl_allowed,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
String * out_disable_reason = nullptr);
|
||||
|
||||
/** Select all the parts in the specified partition for merge, if possible.
|
||||
@ -85,6 +89,7 @@ public:
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
String * out_disable_reason = nullptr,
|
||||
bool optimize_skip_merged_partitions = false);
|
||||
|
||||
@ -107,6 +112,7 @@ public:
|
||||
bool deduplicate,
|
||||
const Names & deduplicate_by_columns,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
const IMergeTreeDataPart * parent_part = nullptr,
|
||||
const String & suffix = "");
|
||||
|
||||
@ -118,12 +124,14 @@ public:
|
||||
MergeListEntry * merge_entry,
|
||||
time_t time_of_mutation,
|
||||
ContextPtr context,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
ReservationSharedPtr space_reservation,
|
||||
TableLockHolder & table_lock_holder);
|
||||
|
||||
MergeTreeData::DataPartPtr renameMergedTemporaryPart(
|
||||
MergeTreeData::MutableDataPartPtr & new_data_part,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
MergeTreeData::Transaction * out_transaction = nullptr);
|
||||
|
||||
|
||||
|
@ -91,7 +91,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri
|
||||
|
||||
auto compression_codec = storage.getContext()->chooseCompressionCodec(0, 0);
|
||||
auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices());
|
||||
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec);
|
||||
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec, NO_TRANSACTION_PTR);
|
||||
out.write(block);
|
||||
const auto & projections = metadata_snapshot->getProjections();
|
||||
for (const auto & [projection_name, projection] : projection_parts)
|
||||
@ -122,7 +122,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri
|
||||
auto projection_indices = MergeTreeIndexFactory::instance().getMany(desc.metadata->getSecondaryIndices());
|
||||
MergedBlockOutputStream projection_out(
|
||||
projection_data_part, desc.metadata, projection_part->columns, projection_indices,
|
||||
projection_compression_codec);
|
||||
projection_compression_codec, NO_TRANSACTION_PTR);
|
||||
|
||||
projection_out.write(projection_part->block);
|
||||
projection_out.finalizePart(projection_data_part, false);
|
||||
|
@ -130,9 +130,11 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
||||
return std::make_unique<QueryPlan>();
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
const auto & metadata_for_reading = storage_snapshot->getMetadataForQuery();
|
||||
|
||||
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
|
||||
|
||||
const auto & parts = snapshot_data.parts;
|
||||
|
||||
if (!query_info.projection)
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
@ -433,7 +434,9 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart(
|
||||
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
auto out = std::make_unique<MergedBlockOutputStream>(new_data_part, metadata_snapshot, columns,
|
||||
index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, false, false, context->getWriteSettings());
|
||||
index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec,
|
||||
context->getCurrentTransaction(), false, false, context->getWriteSettings());
|
||||
|
||||
|
||||
out->writeWithPermutation(block, perm_ptr);
|
||||
|
||||
@ -565,7 +568,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
|
||||
metadata_snapshot,
|
||||
columns,
|
||||
MergeTreeIndices{},
|
||||
compression_codec);
|
||||
compression_codec,
|
||||
NO_TRANSACTION_PTR);
|
||||
|
||||
out->writeWithPermutation(block, perm_ptr);
|
||||
auto finalizer = out->finalizePartAsync(new_data_part, false);
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
|
||||
#include <utility>
|
||||
|
||||
@ -43,13 +44,15 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse mutation version from file name, expected 'mutation_<UInt64>.txt', got '{}'", file_name_);
|
||||
}
|
||||
|
||||
MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const WriteSettings & settings)
|
||||
MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number,
|
||||
const TransactionID & tid_, const WriteSettings & settings)
|
||||
: create_time(time(nullptr))
|
||||
, commands(std::move(commands_))
|
||||
, disk(std::move(disk_))
|
||||
, path_prefix(path_prefix_)
|
||||
, file_name("tmp_mutation_" + toString(tmp_number) + ".txt")
|
||||
, is_temp(true)
|
||||
, tid(tid_)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -59,6 +62,16 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP
|
||||
*out << "commands: ";
|
||||
commands.writeText(*out);
|
||||
*out << "\n";
|
||||
if (tid.isPrehistoric())
|
||||
{
|
||||
csn = Tx::PrehistoricCSN;
|
||||
}
|
||||
else
|
||||
{
|
||||
*out << "tid: ";
|
||||
TransactionID::write(tid, *out);
|
||||
*out << "\n";
|
||||
}
|
||||
out->sync();
|
||||
}
|
||||
catch (...)
|
||||
@ -90,6 +103,14 @@ void MergeTreeMutationEntry::removeFile()
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeMutationEntry::writeCSN(CSN csn_)
|
||||
{
|
||||
csn = csn_;
|
||||
auto out = disk->writeFile(path_prefix + file_name, 256, WriteMode::Append);
|
||||
*out << "csn: " << csn << "\n";
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_)
|
||||
: disk(std::move(disk_))
|
||||
, path_prefix(path_prefix_)
|
||||
@ -111,6 +132,23 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat
|
||||
commands.readText(*buf);
|
||||
*buf >> "\n";
|
||||
|
||||
if (buf->eof())
|
||||
{
|
||||
tid = Tx::PrehistoricTID;
|
||||
csn = Tx::PrehistoricCSN;
|
||||
}
|
||||
else
|
||||
{
|
||||
*buf >> "tid: ";
|
||||
tid = TransactionID::read(*buf);
|
||||
*buf >> "\n";
|
||||
|
||||
if (!buf->eof())
|
||||
{
|
||||
*buf >> "csn: " >> csn >> "\n";
|
||||
}
|
||||
}
|
||||
|
||||
assertEOF(*buf);
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Common/TransactionID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -28,8 +29,15 @@ struct MergeTreeMutationEntry
|
||||
time_t latest_fail_time = 0;
|
||||
String latest_fail_reason;
|
||||
|
||||
/// ID of transaction which has created mutation.
|
||||
TransactionID tid = Tx::PrehistoricTID;
|
||||
/// CSN of transaction which has created mutation
|
||||
/// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction.
|
||||
CSN csn = Tx::UnknownCSN;
|
||||
|
||||
/// Create a new entry and write it to a temporary file.
|
||||
MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, const WriteSettings & settings);
|
||||
MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number,
|
||||
const TransactionID & tid_, const WriteSettings & settings);
|
||||
MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete;
|
||||
MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default;
|
||||
|
||||
@ -38,6 +46,8 @@ struct MergeTreeMutationEntry
|
||||
|
||||
void removeFile();
|
||||
|
||||
void writeCSN(CSN csn_);
|
||||
|
||||
static String versionToFileName(UInt64 block_number_);
|
||||
static UInt64 tryParseFileName(const String & file_name_);
|
||||
static UInt64 parseFileName(const String & file_name_);
|
||||
|
@ -94,7 +94,7 @@ bool MergeTreePartsMover::selectPartsForMove(
|
||||
unsigned parts_to_move_by_ttl_rules = 0;
|
||||
double parts_to_move_total_size_bytes = 0.0;
|
||||
|
||||
MergeTreeData::DataPartsVector data_parts = data->getDataPartsVector();
|
||||
MergeTreeData::DataPartsVector data_parts = data->getDataPartsVectorForInternalUsage();
|
||||
|
||||
if (data_parts.empty())
|
||||
return false;
|
||||
@ -231,6 +231,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt
|
||||
LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getFullPath());
|
||||
|
||||
cloned_part->loadColumnsChecksumsIndexes(true, true);
|
||||
cloned_part->loadVersionMetadata();
|
||||
cloned_part->modification_time = disk->getLastModified(cloned_part->getFullRelativePath()).epochTime();
|
||||
return cloned_part;
|
||||
|
||||
|
@ -61,6 +61,7 @@ struct Settings;
|
||||
M(UInt64, merge_selecting_sleep_ms, 5000, "Sleep time for merge selecting when no part selected, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \
|
||||
M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \
|
||||
M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \
|
||||
M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \
|
||||
\
|
||||
/** Inserts settings. */ \
|
||||
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
|
||||
|
@ -50,7 +50,7 @@ struct MergeTreeSink::DelayedChunk
|
||||
void MergeTreeSink::consume(Chunk chunk)
|
||||
{
|
||||
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot);
|
||||
auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, context);
|
||||
|
||||
storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
||||
@ -134,7 +134,7 @@ void MergeTreeSink::finishDelayedChunk()
|
||||
auto & part = partition.temp_part.part;
|
||||
|
||||
/// Part can be deduplicated, so increment counters and add to part log only if it's really added
|
||||
if (storage.renameTempPartAndAdd(part, &storage.increment, nullptr, storage.getDeduplicationLog(), partition.block_dedup_token))
|
||||
if (storage.renameTempPartAndAdd(part, context->getCurrentTransaction().get(), &storage.increment, nullptr, storage.getDeduplicationLog(), partition.block_dedup_token))
|
||||
{
|
||||
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns);
|
||||
|
||||
|
@ -197,7 +197,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
|
||||
metadata_snapshot,
|
||||
block.getNamesAndTypesList(),
|
||||
{},
|
||||
CompressionCodecFactory::instance().get("NONE", {}));
|
||||
CompressionCodecFactory::instance().get("NONE", {}),
|
||||
NO_TRANSACTION_PTR);
|
||||
|
||||
part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
part->partition.create(metadata_snapshot, block, 0, context);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
|
||||
@ -18,6 +19,7 @@ MergedBlockOutputStream::MergedBlockOutputStream(
|
||||
const NamesAndTypesList & columns_list_,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
CompressionCodecPtr default_codec_,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
bool reset_columns_,
|
||||
bool blocks_are_granules_size,
|
||||
const WriteSettings & write_settings)
|
||||
@ -36,6 +38,13 @@ MergedBlockOutputStream::MergedBlockOutputStream(
|
||||
if (!part_path.empty())
|
||||
volume->getDisk()->createDirectories(part_path);
|
||||
|
||||
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
|
||||
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
|
||||
/// NOTE do not pass context for writing to system.transactions_info_log,
|
||||
/// because part may have temporary name (with temporary block numbers). Will write it later.
|
||||
data_part->version.setCreationTID(tid, nullptr);
|
||||
data_part->storeVersionMetadata();
|
||||
|
||||
writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings);
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@ public:
|
||||
const NamesAndTypesList & columns_list_,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
CompressionCodecPtr default_codec_,
|
||||
const MergeTreeTransactionPtr & txn,
|
||||
bool reset_columns_ = false,
|
||||
bool blocks_are_granules_size = false,
|
||||
const WriteSettings & write_settings = {});
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user