From f596906f47667c2160957540be4814d20672caee Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 Mar 2021 20:55:04 +0300 Subject: [PATCH 001/372] add transaction counters --- src/Common/ErrorCodes.cpp | 2 + src/Common/TransactionMetadata.cpp | 18 +++++ src/Common/TransactionMetadata.h | 41 +++++++++++ .../FunctionsTransactionCounters.cpp | 58 +++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Interpreters/Context.cpp | 15 ++++ src/Interpreters/Context.h | 8 +++ src/Interpreters/InterpreterFactory.cpp | 6 ++ .../InterpreterTransactionControlQuery.cpp | 72 +++++++++++++++++++ .../InterpreterTransactionControlQuery.h | 31 ++++++++ src/Interpreters/MergeTreeTransaction.cpp | 14 ++++ src/Interpreters/MergeTreeTransaction.h | 35 +++++++++ src/Interpreters/QueryLog.cpp | 6 +- src/Interpreters/QueryLog.h | 3 + src/Interpreters/TransactionLog.cpp | 44 ++++++++++++ src/Interpreters/TransactionLog.h | 30 ++++++++ src/Interpreters/executeQuery.cpp | 7 ++ src/Parsers/ASTTransactionControl.cpp | 34 +++++++++ src/Parsers/ASTTransactionControl.h | 29 ++++++++ src/Parsers/ParserQuery.cpp | 5 +- src/Parsers/ParserTransactionControl.cpp | 25 +++++++ src/Parsers/ParserTransactionControl.h | 14 ++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 14 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 11 ++- src/Storages/System/StorageSystemParts.cpp | 22 +++++- 25 files changed, 542 insertions(+), 4 deletions(-) create mode 100644 src/Common/TransactionMetadata.cpp create mode 100644 src/Common/TransactionMetadata.h create mode 100644 src/Functions/FunctionsTransactionCounters.cpp create mode 100644 src/Interpreters/InterpreterTransactionControlQuery.cpp create mode 100644 src/Interpreters/InterpreterTransactionControlQuery.h create mode 100644 src/Interpreters/MergeTreeTransaction.cpp create mode 100644 src/Interpreters/MergeTreeTransaction.h create mode 100644 src/Interpreters/TransactionLog.cpp create mode 100644 src/Interpreters/TransactionLog.h create mode 100644 src/Parsers/ASTTransactionControl.cpp create mode 100644 src/Parsers/ASTTransactionControl.h create mode 100644 src/Parsers/ParserTransactionControl.cpp create mode 100644 src/Parsers/ParserTransactionControl.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0f85ad5c792..55745ffc211 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -548,6 +548,8 @@ M(578, INVALID_FORMAT_INSERT_QUERY_WITH_DATA) \ M(579, INCORRECT_PART_TYPE) \ M(580, CANNOT_SET_ROUNDING_MODE) \ + M(581, INVALID_TRANSACTION) \ + M(582, SERIALIZATION_ERROR) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp new file mode 100644 index 00000000000..430ea5cda29 --- /dev/null +++ b/src/Common/TransactionMetadata.cpp @@ -0,0 +1,18 @@ +#include +#include +#include +#include + +namespace DB +{ + +DataTypePtr TransactionID::getDataType() +{ + DataTypes types; + types.push_back(std::make_shared()); + types.push_back(std::make_shared()); + types.push_back(std::make_shared()); + return std::make_shared(std::move(types)); +} + +} diff --git a/src/Common/TransactionMetadata.h b/src/Common/TransactionMetadata.h new file mode 100644 index 00000000000..34eaaf2d245 --- /dev/null +++ b/src/Common/TransactionMetadata.h @@ -0,0 +1,41 @@ +#pragma once +#include +#include + +namespace DB +{ + +class IDataType; +using DataTypePtr = std::shared_ptr; + +/// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough +using CSN = UInt64; +using Snapshot = CSN; +using LocalTID = UInt64; + +struct TransactionID +{ + CSN start_csn = 0; + LocalTID local_tid = 0; + UUID host_id = UUIDHelpers::Nil; /// Depends on #17278, leave it Nil for now. + + static DataTypePtr getDataType(); +}; + +namespace Tx +{ + +const CSN UnknownCSN = 0; +const CSN PrehistoricCSN = 1; + +const LocalTID PrehistoricLocalTID = 1; + +const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; +const TransactionID PrehistoricTID = {0, PrehistoricLocalTID, UUIDHelpers::Nil}; + +/// So far, that changes will never become visible +const CSN RolledBackCSN = std::numeric_limits::max(); + +} + +} diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp new file mode 100644 index 00000000000..93a2961fb74 --- /dev/null +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -0,0 +1,58 @@ +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +class FunctionTransactionID : public IFunction +{ +public: + static constexpr auto name = "transactionID"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getCurrentTransaction()); + } + + explicit FunctionTransactionID(MergeTreeTransactionPtr && txn_) : txn(txn_) + { + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes &) const override + { + return TransactionID::getDataType(); + } + + bool isDeterministic() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override + { + 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 result_type->createColumnConst(input_rows_count, res); + } + +private: + MergeTreeTransactionPtr txn; +}; + +} + +void registerFunctionsTransactionCounters(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index d00d4a42db0..ab415740be3 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -73,6 +73,7 @@ void registerFunctionFile(FunctionFactory & factory); void registerFunctionConnectionId(FunctionFactory & factory); void registerFunctionPartitionId(FunctionFactory & factory); void registerFunctionIsIPAddressContainedIn(FunctionFactory &); +void registerFunctionsTransactionCounters(FunctionFactory & factory); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -146,6 +147,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionConnectionId(factory); registerFunctionPartitionId(factory); registerFunctionIsIPAddressContainedIn(factory); + registerFunctionsTransactionCounters(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8eec5d099e2..5c6afb38ff4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2581,6 +2581,21 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const return metadata_transaction; } +void Context::setCurrentTransaction(MergeTreeTransactionPtr txn) +{ + assert(!merge_tree_transaction || !txn); + assert(this == session_context || this == query_context); + int enable_mvcc_test_helper = getConfigRef().getInt("_enable_mvcc_test_helper_dev", 0); + if (enable_mvcc_test_helper != 42) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); + merge_tree_transaction = std::move(txn); +} + +MergeTreeTransactionPtr Context::getCurrentTransaction() const +{ + return merge_tree_transaction; +} + PartUUIDsPtr Context::getPartUUIDs() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5c98a2ba64a..e035031f993 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -122,6 +122,8 @@ struct BackgroundTaskSchedulingSettings; class ZooKeeperMetadataTransaction; using ZooKeeperMetadataTransactionPtr = std::shared_ptr; +class MergeTreeTransaction; +using MergeTreeTransactionPtr = std::shared_ptr; #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; @@ -290,6 +292,8 @@ 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. + /// Use copy constructor or createGlobal() instead Context(); @@ -757,6 +761,10 @@ public: /// Returns context of current distributed DDL query or nullptr. ZooKeeperMetadataTransactionPtr getZooKeeperMetadataTransaction() const; + void setCurrentTransaction(MergeTreeTransactionPtr txn); + MergeTreeTransactionPtr getCurrentTransaction() const; + + struct MySQLWireContext { uint8_t sequence_id = 0; diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 15e4c52f040..5684150fed1 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include @@ -66,6 +67,7 @@ #include #include #include +#include #include #include @@ -264,6 +266,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else { throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp new file mode 100644 index 00000000000..369f5edd1da --- /dev/null +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -0,0 +1,72 @@ +#include +#include +#include +#include + +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"); + + Context & session_context = query_context.getSessionContext(); + const auto & tcl = query_ptr->as(); + + switch (tcl.action) + { + case ASTTransactionControl::BEGIN: + return executeBegin(session_context); + case ASTTransactionControl::COMMIT: + return executeCommit(session_context); + case ASTTransactionControl::ROLLBACK: + return executeRollback(session_context); + } +} + +BlockIO InterpreterTransactionControlQuery::executeBegin(Context & context) +{ + if (context.getCurrentTransaction()) + throw Exception(ErrorCodes::INVALID_TRANSACTION, "Nested transactions are not supported"); + + auto txn = TransactionLog::instance().beginTransaction(); + context.setCurrentTransaction(txn); + query_context.setCurrentTransaction(txn); + return {}; +} + +BlockIO InterpreterTransactionControlQuery::executeCommit(Context & context) +{ + auto txn = 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); + context.setCurrentTransaction(nullptr); + return {}; +} + +BlockIO InterpreterTransactionControlQuery::executeRollback(Context & context) +{ + auto txn = 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); + context.getSessionContext().setCurrentTransaction(nullptr); + return {}; +} + +} diff --git a/src/Interpreters/InterpreterTransactionControlQuery.h b/src/Interpreters/InterpreterTransactionControlQuery.h new file mode 100644 index 00000000000..4c455d22a24 --- /dev/null +++ b/src/Interpreters/InterpreterTransactionControlQuery.h @@ -0,0 +1,31 @@ +#pragma once +#include +#include + +namespace DB +{ + +class InterpreterTransactionControlQuery : public IInterpreter +{ +public: + InterpreterTransactionControlQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_) + , query_context(context_) + { + } + + BlockIO execute() override; + + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } +private: + BlockIO executeBegin(Context & context); + BlockIO executeCommit(Context & context); + BlockIO executeRollback(Context & context); + +private: + ASTPtr query_ptr; + Context & query_context; +}; + +} diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp new file mode 100644 index 00000000000..d1af1d61ba7 --- /dev/null +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id) + : tid({snapshot_, local_tid_, host_id}) + , snapshot(snapshot_) + , state(RUNNING) +{ +} + + +} diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h new file mode 100644 index 00000000000..95c4a81c756 --- /dev/null +++ b/src/Interpreters/MergeTreeTransaction.h @@ -0,0 +1,35 @@ +#pragma once +#include + +namespace DB +{ + +class MergeTreeTransaction +{ + friend class TransactionLog; +public: + enum State + { + RUNNING, + COMMITTED, + ROLLED_BACK, + }; + + Snapshot getSnapshot() const { return snapshot; } + State getState() const { return state; } + + const TransactionID tid; + + MergeTreeTransaction() = delete; + MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id); + +private: + Snapshot snapshot; + State state; + + CSN csn = Tx::UnknownCSN; +}; + +using MergeTreeTransactionPtr = std::shared_ptr; + +} diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index b6902468242..cee1d2b5412 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -108,7 +108,9 @@ Block QueryLogElement::createBlock() {std::make_shared(std::make_shared()), "used_formats"}, {std::make_shared(std::make_shared()), "used_functions"}, {std::make_shared(std::make_shared()), "used_storages"}, - {std::make_shared(std::make_shared()), "used_table_functions"} + {std::make_shared(std::make_shared()), "used_table_functions"}, + + {TransactionID::getDataType(), "transaction_id"} }; } @@ -237,6 +239,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) diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 8617a8d1cbc..7532f2d14f2 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -2,6 +2,7 @@ #include #include +#include namespace ProfileEvents @@ -80,6 +81,8 @@ struct QueryLogElement std::shared_ptr profile_counters; std::shared_ptr query_settings; + TransactionID tid; + static std::string name() { return "QueryLog"; } static Block createBlock(); diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp new file mode 100644 index 00000000000..00dbb3352b4 --- /dev/null +++ b/src/Interpreters/TransactionLog.cpp @@ -0,0 +1,44 @@ +#include + +namespace DB +{ + +TransactionLog & TransactionLog::instance() +{ + static TransactionLog inst; + return inst; +} + +TransactionLog::TransactionLog() +{ + csn_counter = 1; + local_tid_counter = 1; +} + +Snapshot TransactionLog::getLatestSnapshot() const +{ + return csn_counter.load(); +} + +MergeTreeTransactionPtr TransactionLog::beginTransaction() +{ + Snapshot snapshot = csn_counter.load(); + LocalTID ltid = 1 + local_tid_counter.fetch_add(1); + return std::make_shared(snapshot, ltid, UUIDHelpers::Nil); +} + +CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) +{ + txn->csn = 1 + csn_counter.fetch_add(1); + /// TODO Transactions: reset local_tid_counter + txn->state = MergeTreeTransaction::COMMITTED; + return txn->csn; +} + +void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) +{ + txn->csn = Tx::RolledBackCSN; + txn->state = MergeTreeTransaction::ROLLED_BACK; +} + +} diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h new file mode 100644 index 00000000000..073d2a2929d --- /dev/null +++ b/src/Interpreters/TransactionLog.h @@ -0,0 +1,30 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class TransactionLog final : private boost::noncopyable +{ +public: + static TransactionLog & instance(); + + TransactionLog(); + + Snapshot getLatestSnapshot() const; + + /// Allocated TID, returns transaction object + MergeTreeTransactionPtr beginTransaction(); + + CSN commitTransaction(const MergeTreeTransactionPtr & txn); + + void rollbackTransaction(const MergeTreeTransactionPtr & txn); + +private: + std::atomic csn_counter; + std::atomic local_tid_counter; +}; + +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a5c21405ff1..a00f8301622 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -50,6 +50,7 @@ #include #include #include +#include #include #include @@ -266,6 +267,9 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c 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); @@ -634,6 +638,9 @@ static std::tuple executeQueryImpl( elem.client_info = context.getClientInfo(); + 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. diff --git a/src/Parsers/ASTTransactionControl.cpp b/src/Parsers/ASTTransactionControl.cpp new file mode 100644 index 00000000000..d12c9d6d6e4 --- /dev/null +++ b/src/Parsers/ASTTransactionControl.cpp @@ -0,0 +1,34 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +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; + } +} + +void ASTTransactionControl::updateTreeHashImpl(SipHash & hash_state) const +{ + hash_state.update(action); +} + +} diff --git a/src/Parsers/ASTTransactionControl.h b/src/Parsers/ASTTransactionControl.h new file mode 100644 index 00000000000..c01c4172627 --- /dev/null +++ b/src/Parsers/ASTTransactionControl.h @@ -0,0 +1,29 @@ +#pragma once +#include + +namespace DB +{ + +/// Common AST for TCL queries +class ASTTransactionControl : public IAST +{ +public: + enum QueryType + { + BEGIN, + COMMIT, + ROLLBACK, + }; + + QueryType action; + + ASTTransactionControl(QueryType action_) : action(action_) {} + + String getID(char /*delimiter*/) const override { return "ASTTransactionControl"; } + ASTPtr clone() const override { return std::make_shared(*this); } + + void formatImpl(const FormatSettings & format, FormatState & /*state*/, FormatStateStacked /*frame*/) const override; + void updateTreeHashImpl(SipHash & hash_state) const override; +}; + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 4550bdc8a75..f9c11f1c870 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -40,6 +41,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; bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) @@ -54,7 +56,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) - || external_ddl_p.parse(pos, node, expected); + || external_ddl_p.parse(pos, node, expected) + || transaction_control_p.parse(pos, node, expected); return res; } diff --git a/src/Parsers/ParserTransactionControl.cpp b/src/Parsers/ParserTransactionControl.cpp new file mode 100644 index 00000000000..a5591a0447e --- /dev/null +++ b/src/Parsers/ParserTransactionControl.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTTransactionControl::QueryType action; + + 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 + return false; + + node = std::make_shared(action); + return true; +} + +} diff --git a/src/Parsers/ParserTransactionControl.h b/src/Parsers/ParserTransactionControl.h new file mode 100644 index 00000000000..157c088624c --- /dev/null +++ b/src/Parsers/ParserTransactionControl.h @@ -0,0 +1,14 @@ +#pragma once +#include + +namespace DB +{ + +class ParserTransactionControl : public IParserBase +{ +public: + const char * getName() const override { return "TCL query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 03f6564788a..c990bdcc65b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -301,6 +302,19 @@ public: CompressionCodecPtr default_codec; + struct VersionMetadata + { + TransactionID mintid = Tx::EmptyTID; + TransactionID maxtid = Tx::EmptyTID; + + bool maybe_visible = false; + + CSN mincsn = Tx::UnknownCSN; + CSN maxcsn = Tx::UnknownCSN; + }; + + mutable VersionMetadata versions; + /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d61de13b604..7610239662e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -993,6 +993,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); modifyPartState(it, DataPartState::Outdated); + (*it)->versions.maxtid = Tx::PrehistoricTID; + (*it)->versions.maxcsn = Tx::PrehistoricCSN; removePartContributionToDataVolume(*it); }; @@ -1000,6 +1002,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Committed) { + /// 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. + /// TODO Transactions: distinguish "prehistoric" parts from uncommitted parts in case of hard restart + (*curr_jt)->versions.mintid = Tx::PrehistoricTID; + (*curr_jt)->versions.mincsn = Tx::PrehistoricCSN; + (*curr_jt)->versions.maybe_visible = true; + /// Don't consider data parts belonging to different partitions. if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id) { @@ -1030,7 +1040,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) calculateColumnSizesImpl(); - LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 6a643dbe1b9..f100ecdfa9b 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -75,7 +76,12 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"rows_where_ttl_info.expression", std::make_shared(std::make_shared())}, {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.max", std::make_shared(std::make_shared())} + {"rows_where_ttl_info.max", std::make_shared(std::make_shared())}, + + {"mintid", TransactionID::getDataType()}, + {"maxtid", TransactionID::getDataType()}, + {"mincsn", std::make_shared()}, + {"maxcsn", std::make_shared()}, } ) { @@ -257,6 +263,20 @@ void StorageSystemParts::processNextStorage( /// Do not use part->getState*, it can be changed from different thread if (has_state_column) columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state)); + + auto get_tid_as_field = [](const TransactionID & tid) -> Field + { + return Tuple{tid.start_csn, tid.local_tid, tid.host_id}; + }; + + if (columns_mask[src_index++]) + columns[res_index++]->insert(get_tid_as_field(part->versions.mintid)); + if (columns_mask[src_index++]) + columns[res_index++]->insert(get_tid_as_field(part->versions.maxtid)); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->versions.mincsn); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->versions.maxcsn); } } From 3422bd1742238ec85e5328bbf6f0c53a13449515 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Apr 2021 20:20:45 +0300 Subject: [PATCH 002/372] check parts visibility for select --- src/Common/TransactionMetadata.cpp | 154 ++++++++++++++++++ src/Common/TransactionMetadata.h | 60 +++++++ src/IO/WriteHelpers.h | 16 ++ src/Interpreters/InterpreterSelectQuery.cpp | 3 +- src/Interpreters/MergeTreeTransaction.cpp | 79 +++++++++ src/Interpreters/MergeTreeTransaction.h | 20 +++ src/Interpreters/TransactionLog.cpp | 81 ++++++++- src/Interpreters/TransactionLog.h | 14 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 12 +- .../MergeTree/MergeTreeBlockOutputStream.cpp | 2 +- .../MergeTree/MergeTreeBlockOutputStream.h | 9 +- src/Storages/MergeTree/MergeTreeData.cpp | 70 ++++++-- src/Storages/MergeTree/MergeTreeData.h | 12 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 10 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 +- src/Storages/System/StorageSystemParts.cpp | 6 +- .../01174_select_insert_isolation.reference | 1 + .../01174_select_insert_isolation.sh | 60 +++++++ 21 files changed, 574 insertions(+), 53 deletions(-) create mode 100644 tests/queries/0_stateless/01174_select_insert_isolation.reference create mode 100755 tests/queries/0_stateless/01174_select_insert_isolation.sh diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp index 430ea5cda29..a4dcebbf3ef 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Common/TransactionMetadata.cpp @@ -1,11 +1,19 @@ #include +#include #include #include #include +#include namespace DB { +namespace ErrorCodes +{ +extern const int SERIALIZATION_ERROR; +extern const int LOGICAL_ERROR; +} + DataTypePtr TransactionID::getDataType() { DataTypes types; @@ -15,4 +23,150 @@ DataTypePtr TransactionID::getDataType() return std::make_shared(std::move(types)); } +TIDHash TransactionID::getHash() const +{ + SipHash hash; + hash.update(start_csn); + hash.update(local_tid); + hash.update(host_id); + return hash.get64(); +} + +/// It can be used fro introspection purposes only +TransactionID VersionMetadata::getMaxTID() const +{ + TIDHash max_lock = maxtid_lock.load(); + if (max_lock) + { + if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) + return txn->tid; + } + + if (maxcsn.load(std::memory_order_relaxed)) + { + /// maxtid cannot be changed since we have maxcsn, so it's readonly + return maxtid; + } + + return Tx::EmptyTID; +} + +void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context) +{ + TIDHash max_lock_value = tid.getHash(); + TIDHash expected_max_lock_value = 0; + bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); + if (!locked) + { + throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: " + "Transaction {} tried to remove data part, " + "but it's locked ({}) by another transaction {} which is currently removing this part. {}", + tid, expected_max_lock_value, getMaxTID(), error_context); + } + + maxtid = tid; +} + +void VersionMetadata::unlockMaxTID(const TransactionID & tid) +{ + TIDHash max_lock_value = tid.getHash(); + TIDHash locked_by = maxtid_lock.load(); + + auto throw_cannot_unlock = [&]() + { + auto locked_by_txn = TransactionLog::instance().tryGetRunningTransaction(locked_by); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unlock maxtid, it's a bug. Current: {} {}, actual: {} {}", + max_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID); + }; + + if (locked_by != max_lock_value) + throw_cannot_unlock(); + + maxtid = Tx::EmptyTID; + bool unlocked = maxtid_lock.compare_exchange_strong(locked_by, 0); + if (!unlocked) + throw_cannot_unlock(); +} + +void VersionMetadata::setMinTID(const TransactionID & tid) +{ + /// TODO Transactions: initialize it in constructor on part creation and remove this method + assert(!mintid); + const_cast(mintid) = tid; +} + +bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) +{ + Snapshot snapshot_version = txn.getSnapshot(); + assert(mintid); + CSN min = mincsn.load(std::memory_order_relaxed); + TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); + CSN max = maxcsn.load(std::memory_order_relaxed); + + [[maybe_unused]] bool had_mincsn = min; + [[maybe_unused]] bool had_maxtid = max_lock; + [[maybe_unused]] bool had_maxcsn = max; + assert(!had_maxcsn || had_maxtid); + assert(!had_maxcsn || had_mincsn); + + /// Fast path: + + /// Part is definitely not visible if: + /// - creation was committed after we took the snapshot + /// - removal was committed before we took the snapshot + /// - current transaction is removing it + if (min && snapshot_version < min) + return false; + if (max && max <= snapshot_version) + return false; + if (max_lock && max_lock == txn.tid.getHash()) + return false; + + /// Otherwise, part is definitely visible if: + /// - creation was committed after we took the snapshot and nobody tried to remove the part + /// - current transaction is creating it + if (!max_lock && min && min <= snapshot_version) + return true; + if (mintid == txn.tid) + return true; + + /// End of fast path. + + /// Data part has mintid/maxtid, but does not have mincsn/maxcsn. + /// It means that some transaction is creating/removing the part right now or has done it recently + /// and we don't know if it was already committed ot not. + assert(!had_mincsn || (had_maxtid && !had_maxcsn)); + assert(mintid != txn.tid && max_lock != txn.tid.getHash()); + + /// Before doing CSN lookup, let's check some extra conditions. + /// If snapshot_version <= some_tid.start_csn, then changes of transaction with some_tid + /// are definitely not visible for us, so we don't need to check if it was committed. + if (snapshot_version <= mintid.start_csn) + return false; + + /// Check if mintid/maxtid transactions are committed and write CSNs + /// TODO Transactions: we probably need some optimizations here + /// to avoid some CSN lookups or make the lookups cheaper. + /// NOTE: Old enough committed parts always have written CSNs, + /// so we can determine their visibility through fast path. + /// But for long-running writing transactions we will always do + /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. + min = TransactionLog::instance().getCSN(mintid); + if (!min) + return false; /// Part creation is not committed yet + + /// We don't need to check if CSNs are already writen or not, + /// because once writen CSN cannot be changed, so it's safe to overwrite it (with tha same value). + mincsn.store(min, std::memory_order_relaxed); + + if (max_lock) + { + max = TransactionLog::instance().getCSN(max_lock); + if (max) + maxcsn.store(max, std::memory_order_relaxed); + } + + return min <= snapshot_version && (!max || snapshot_version < max); +} + } diff --git a/src/Common/TransactionMetadata.h b/src/Common/TransactionMetadata.h index 34eaaf2d245..6abf7c2b4b3 100644 --- a/src/Common/TransactionMetadata.h +++ b/src/Common/TransactionMetadata.h @@ -1,17 +1,21 @@ #pragma once #include #include +#include +#include namespace DB { class IDataType; using DataTypePtr = std::shared_ptr; +class MergeTreeTransaction; /// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough using CSN = UInt64; using Snapshot = CSN; using LocalTID = UInt64; +using TIDHash = UInt64; struct TransactionID { @@ -20,6 +24,24 @@ struct TransactionID UUID host_id = UUIDHelpers::Nil; /// Depends on #17278, leave it Nil for now. static DataTypePtr getDataType(); + + 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); + } + + operator bool() const + { + assert(local_tid || (start_csn == 0 && host_id == UUIDHelpers::Nil)); + return local_tid; + } + + TIDHash getHash() const; }; namespace Tx @@ -38,4 +60,42 @@ const CSN RolledBackCSN = std::numeric_limits::max(); } +struct VersionMetadata +{ + const TransactionID mintid = Tx::EmptyTID; + TransactionID maxtid = Tx::EmptyTID; + + std::atomic maxtid_lock = 0; + + std::atomic mincsn = Tx::UnknownCSN; + std::atomic maxcsn = Tx::UnknownCSN; + + bool isVisible(const MergeTreeTransaction & txn); + + TransactionID getMinTID() const { return mintid; } + TransactionID getMaxTID() const; + + void lockMaxTID(const TransactionID & tid, const String & error_context = {}); + void unlockMaxTID(const TransactionID & tid); + + /// It can be called only from MergeTreeTransaction or on server startup + void setMinTID(const TransactionID & tid); +}; + } + +template<> +struct fmt::formatter +{ + template + constexpr auto parse(ParseContext & context) + { + return context.begin(); + } + + template + auto format(const DB::TransactionID & tid, FormatContext & context) + { + return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id); + } +}; diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index b9497b6f87e..8400d676018 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1182,3 +1182,19 @@ struct PcgSerializer void writePointerHex(const void * ptr, WriteBuffer & buf); } + +template<> +struct fmt::formatter +{ + template + constexpr auto parse(ParseContext & context) + { + return context.begin(); + } + + template + auto format(const DB::UUID & uuid, FormatContext & context) + { + return fmt::format_to(context.out(), "{}", toString(uuid)); + } +}; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1f6b0c37437..42103ec15b4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1370,7 +1370,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) - && typeid_cast(query_analyzer->aggregates()[0].function.get()); + && typeid_cast(query_analyzer->aggregates()[0].function.get()) + && !context->getCurrentTransaction(); if (optimize_trivial_count) { diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index d1af1d61ba7..5b267e45f83 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -1,4 +1,6 @@ #include +#include +#include namespace DB { @@ -10,5 +12,82 @@ MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_ti { } +void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part, MergeTreeTransaction * txn) +{ + TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + + new_part->versions.setMinTID(tid); + if (txn) + txn->addNewPart(new_part); +} + +void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) +{ + TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + String error_context = fmt::format("Table: {}, part name: {}", + part_to_remove->storage.getStorageID().getNameForLogs(), + part_to_remove->name); + part_to_remove->versions.lockMaxTID(tid, error_context); + if (txn) + txn->removeOldPart(part_to_remove); +} + +void MergeTreeTransaction::addNewPartAndRemoveCovered(const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) +{ + TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + + new_part->versions.setMinTID(tid); + if (txn) + txn->addNewPart(new_part); + + String error_context = fmt::format("Table: {}, covering part name: {}", + new_part->storage.getStorageID().getNameForLogs(), + new_part->name); + error_context += ", part_name: {}"; + for (auto covered : covered_parts) + { + covered->versions.lockMaxTID(tid, fmt::format(error_context, covered->name)); + if (txn) + txn->removeOldPart(covered); + } +} + +void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part) +{ + creating_parts.push_back(new_part); +} + +void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove) +{ + removing_parts.push_back(part_to_remove); +} + +bool MergeTreeTransaction::isReadOnly() const +{ + return creating_parts.empty() && removing_parts.empty(); +} + +void MergeTreeTransaction::beforeCommit() +{ + assert(state == RUNNING); +} + +void MergeTreeTransaction::afterCommit() +{ + assert(state == COMMITTED); + for (const auto & part : creating_parts) + part->versions.mincsn.store(csn); + for (const auto & part : removing_parts) + part->versions.maxcsn.store(csn); +} + +void MergeTreeTransaction::rollback() +{ + assert(state == RUNNING); + for (const auto & part : creating_parts) + part->versions.mincsn.store(Tx::RolledBackCSN); + for (const auto & part : removing_parts) + part->versions.unlockMaxTID(tid); +} } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 95c4a81c756..c085fa96b0f 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -4,6 +4,10 @@ namespace DB { +class IMergeTreeDataPart; +using DataPartPtr = std::shared_ptr; +using DataPartsVector = std::vector; + class MergeTreeTransaction { friend class TransactionLog; @@ -23,10 +27,26 @@ public: MergeTreeTransaction() = delete; MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id); + void addNewPart(const DataPartPtr & new_part); + void removeOldPart(const DataPartPtr & part_to_remove); + + static void addNewPart(const DataPartPtr & new_part, MergeTreeTransaction * txn); + static void removeOldPart(const DataPartPtr & part_to_remove, MergeTreeTransaction * txn); + static void addNewPartAndRemoveCovered(const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn); + + bool isReadOnly() const; + private: + void beforeCommit(); + void afterCommit(); + void rollback(); + Snapshot snapshot; State state; + DataPartsVector creating_parts; + DataPartsVector removing_parts; + CSN csn = Tx::UnknownCSN; }; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 00dbb3352b4..4b4f893f791 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -1,8 +1,15 @@ #include +#include +#include namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + TransactionLog & TransactionLog::instance() { static TransactionLog inst; @@ -11,27 +18,59 @@ TransactionLog & TransactionLog::instance() TransactionLog::TransactionLog() { + latest_snapshot = 1; csn_counter = 1; local_tid_counter = 1; } Snapshot TransactionLog::getLatestSnapshot() const { - return csn_counter.load(); + return latest_snapshot.load(); } MergeTreeTransactionPtr TransactionLog::beginTransaction() { - Snapshot snapshot = csn_counter.load(); + Snapshot snapshot = latest_snapshot.load(); LocalTID ltid = 1 + local_tid_counter.fetch_add(1); - return std::make_shared(snapshot, ltid, UUIDHelpers::Nil); + auto txn = std::make_shared(snapshot, ltid, UUIDHelpers::Nil); + { + std::lock_guard lock{running_list_mutex}; + bool inserted = running_list.try_emplace(txn->tid.getHash(), txn).second; /// Commit point + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); + } + return txn; } CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) { - txn->csn = 1 + csn_counter.fetch_add(1); + txn->beforeCommit(); + + CSN new_csn; /// TODO Transactions: reset local_tid_counter + if (txn->isReadOnly()) + { + new_csn = txn->snapshot; + } + else + { + std::lock_guard lock{commit_mutex}; + new_csn = 1 + csn_counter.fetch_add(1); + bool inserted = tid_to_csn.try_emplace(txn->tid.getHash(), new_csn).second; /// Commit point + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); + latest_snapshot.store(new_csn, std::memory_order_relaxed); + } + + txn->csn = new_csn; txn->state = MergeTreeTransaction::COMMITTED; + txn->afterCommit(); + { + 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); + } return txn->csn; } @@ -39,6 +78,40 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) { txn->csn = Tx::RolledBackCSN; txn->state = MergeTreeTransaction::ROLLED_BACK; + { + 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); + } +} + +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 nullptr; + return it->second; +} + +CSN TransactionLog::getCSN(const TransactionID & tid) const +{ + return getCSN(tid.getHash()); +} + +CSN TransactionLog::getCSN(const TIDHash & tid) const +{ + assert(tid); + assert(tid != Tx::EmptyTID.getHash()); + if (tid == Tx::PrehistoricTID.getHash()) + return Tx::PrehistoricCSN; + + std::lock_guard lock{commit_mutex}; + auto it = tid_to_csn.find(tid); + if (it == tid_to_csn.end()) + return Tx::UnknownCSN; + return it->second; } } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 073d2a2929d..8766ab24251 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -22,9 +23,22 @@ public: void rollbackTransaction(const MergeTreeTransactionPtr & txn); + CSN getCSN(const TransactionID & tid) const; + CSN getCSN(const TIDHash & tid) const; + + MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); + private: + std::atomic latest_snapshot; std::atomic csn_counter; std::atomic local_tid_counter; + + /// FIXME Transactions: it's probably a bad idea to use global mutex here + mutable std::mutex commit_mutex; + std::unordered_map tid_to_csn; + + mutable std::mutex running_list_mutex; + std::unordered_map running_list; }; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c990bdcc65b..7b481a3719e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -44,6 +44,7 @@ class IMergeTreeReader; class IMergeTreeDataPartWriter; class MarkCache; class UncompressedCache; +class MergeTreeTransaction; namespace ErrorCodes @@ -302,17 +303,6 @@ public: CompressionCodecPtr default_codec; - struct VersionMetadata - { - TransactionID mintid = Tx::EmptyTID; - TransactionID maxtid = Tx::EmptyTID; - - bool maybe_visible = false; - - CSN mincsn = Tx::UnknownCSN; - CSN maxcsn = Tx::UnknownCSN; - }; - mutable VersionMetadata versions; /// For data in RAM ('index') diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index bb5644567ae..81b67021651 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -35,7 +35,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) if (!part) continue; - storage.renameTempPartAndAdd(part, &storage.increment); + storage.renameTempPartAndAdd(part, context.getCurrentTransaction().get(), &storage.increment); PartLog::addNewPart(storage.global_context, part, watch.elapsed()); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 8caa53382dc..0000696a2d4 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -14,11 +14,17 @@ class StorageMergeTree; class MergeTreeBlockOutputStream : public IBlockOutputStream { public: - MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_, bool optimize_on_insert_) + MergeTreeBlockOutputStream( + StorageMergeTree & storage_, + const StorageMetadataPtr metadata_snapshot_, + size_t max_parts_per_block_, + bool optimize_on_insert_, + const Context & context_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , max_parts_per_block(max_parts_per_block_) , optimize_on_insert(optimize_on_insert_) + , context(context_) { } @@ -31,6 +37,7 @@ private: StorageMetadataPtr metadata_snapshot; size_t max_parts_per_block; bool optimize_on_insert; + const Context & context; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7610239662e..b452f2a7c45 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -979,6 +980,16 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part->renameToDetached(""); + for (auto it = data_parts_by_state_and_info.begin(); it != data_parts_by_state_and_info.end(); ++it) + { + /// 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. + /// TODO Transactions: distinguish "prehistoric" parts from uncommitted parts in case of hard restart + (*it)->versions.setMinTID(Tx::PrehistoricTID); + (*it)->versions.mincsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); + } + /// 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. @@ -994,7 +1005,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); modifyPartState(it, DataPartState::Outdated); (*it)->versions.maxtid = Tx::PrehistoricTID; - (*it)->versions.maxcsn = Tx::PrehistoricCSN; + (*it)->versions.maxcsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); removePartContributionToDataVolume(*it); }; @@ -1002,14 +1013,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Committed) { - /// 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. - /// TODO Transactions: distinguish "prehistoric" parts from uncommitted parts in case of hard restart - (*curr_jt)->versions.mintid = Tx::PrehistoricTID; - (*curr_jt)->versions.mincsn = Tx::PrehistoricCSN; - (*curr_jt)->versions.maybe_visible = true; - /// Don't consider data parts belonging to different partitions. if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id) { @@ -2031,7 +2034,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace( } -bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction) +bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment, Transaction * out_transaction) { if (out_transaction && &out_transaction->data != this) throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", @@ -2040,7 +2043,7 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem DataPartsVector covered_parts; { auto lock = lockParts(); - if (!renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts)) + if (!renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts)) return false; } if (!covered_parts.empty()) @@ -2052,7 +2055,7 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem bool MergeTreeData::renameTempPartAndReplace( - MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, + MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment, Transaction * out_transaction, std::unique_lock & lock, DataPartsVector * out_covered_parts) { if (out_transaction && &out_transaction->data != this) @@ -2100,7 +2103,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) { @@ -2108,6 +2110,10 @@ bool MergeTreeData::renameTempPartAndReplace( return false; } + /// FIXME Transactions: it's not the best place for checking and setting maxtid, + /// because it's too optimistic. We should lock maxtid of covered parts at the beginning of operation. + MergeTreeTransaction::addNewPartAndRemoveCovered(part, covered_parts, txn); + /// All checks are passed. Now we can rename the part on disk. /// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts /// @@ -2164,7 +2170,7 @@ bool MergeTreeData::renameTempPartAndReplace( } MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( - MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction) + MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment, Transaction * out_transaction) { if (out_transaction && &out_transaction->data != this) throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", @@ -2173,7 +2179,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( DataPartsVector covered_parts; { auto lock = lockParts(); - renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts); + renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts); } return covered_parts; } @@ -3072,6 +3078,40 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context return partition_id; } +DataPartsVector MergeTreeData::getDataPartsVector(const Context & context) const +{ + if (auto txn = context.getCurrentTransaction()) + return getVisibleDataPartsVector(*txn); + else + return getDataPartsVector(); +} + +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const MergeTreeTransaction & txn) const +{ + DataPartsVector maybe_visible_parts = getDataPartsVector({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); + if (maybe_visible_parts.empty()) + return maybe_visible_parts; + + auto it = maybe_visible_parts.begin(); + auto it_last = maybe_visible_parts.end() - 1; + while (it <= it_last) + { + if ((*it)->versions.isVisible(txn)) + { + ++it; + } + else + { + std::swap(*it, *it_last); + --it_last; + } + } + + size_t new_size = it_last - maybe_visible_parts.begin() + 1; + maybe_visible_parts.resize(new_size); + return maybe_visible_parts; +} + MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states) const { DataPartsVector res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 63d776a838c..d9e9bb9bcec 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -40,6 +40,7 @@ class MergeTreePartsMover; class MutationCommands; class Context; struct JobAndPool; +class MergeTreeTransaction; /// Auxiliary struct holding information about the future merged or mutated part. struct EmergingPartInfo @@ -406,6 +407,9 @@ public: DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; + DataPartsVector getDataPartsVector(const Context & context) const; + DataPartsVector getVisibleDataPartsVector(const MergeTreeTransaction & txn) const; + /// Returns a committed part 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; @@ -447,17 +451,19 @@ public: /// active set later with out_transaction->commit()). /// Else, commits the part immediately. /// Returns true if part was added. Returns false if part is covered by bigger part. - bool renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); + bool renameTempPartAndAdd(MutableDataPartPtr & part, MergeTreeTransaction * txn, + SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); /// The same as renameTempPartAndAdd but the block range of the part can contain existing parts. /// 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); + MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr); /// Low-level version of previous one, doesn't lock mutex bool renameTempPartAndReplace( - MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock, + MutableDataPartPtr & part, MergeTreeTransaction * txn, + SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock, DataPartsVector * out_covered_parts = nullptr); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4269aa89ad1..7209053400f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1337,7 +1337,7 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart MergeTreeData::Transaction * out_transaction) { /// 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, nullptr, nullptr, out_transaction); /// Let's check that all original parts have been deleted and only them. if (replaced_parts.size() != parts.size()) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f3759107912..f62601b9924 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -158,7 +158,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const PartitionIdToMaxBlock * max_block_numbers_to_read) const { return readFromParts( - data.getDataPartsVector(), column_names_to_return, metadata_snapshot, + data.getDataPartsVector(context), column_names_to_return, metadata_snapshot, query_info, context, max_block_size, num_streams, max_block_numbers_to_read); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 529e3d2ab49..ed1d5c09b35 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -380,7 +380,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( bool renamed = false; try { - renamed = storage.renameTempPartAndAdd(part, nullptr, &transaction); + renamed = storage.renameTempPartAndAdd(part, nullptr, nullptr, &transaction); } catch (const Exception & e) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 10790057ac9..fbfc5b54897 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -226,7 +226,7 @@ BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Sto const auto & settings = context.getSettingsRef(); return std::make_shared( - *this, metadata_snapshot, settings.max_partitions_per_insert_block, context.getSettingsRef().optimize_on_insert); + *this, metadata_snapshot, settings.max_partitions_per_insert_block, context.getSettingsRef().optimize_on_insert, context); } void StorageMergeTree::checkTableCanBeDropped() const @@ -933,7 +933,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_list_entry), time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, table_lock_holder); - renameTempPartAndReplace(new_part); + renameTempPartAndReplace(new_part, nullptr); updateMutationEntriesErrors(future_part, true, ""); write_part_log({}); @@ -1231,7 +1231,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( { LOG_INFO(log, "Attaching part {} from {}", loaded_parts[i]->name, renamed_parts.old_and_new_names[i].second); String old_name = renamed_parts.old_and_new_names[i].first; - renameTempPartAndAdd(loaded_parts[i], &increment); + renameTempPartAndAdd(loaded_parts[i], context.getCurrentTransaction().get(), &increment); renamed_parts.old_and_new_names[i].first.clear(); results.push_back(PartitionCommandResultInfo{ @@ -1303,7 +1303,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con /// Populate transaction for (MutableDataPartPtr & part : dst_parts) - renameTempPartAndReplace(part, &increment, &transaction, data_parts_lock); + renameTempPartAndReplace(part, context.getCurrentTransaction().get(), &increment, &transaction, data_parts_lock); transaction.commit(&data_parts_lock); @@ -1379,7 +1379,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const DataPartsLock lock(mutex); for (MutableDataPartPtr & part : dst_parts) - dest_table_storage->renameTempPartAndReplace(part, &dest_table_storage->increment, &transaction, lock); + dest_table_storage->renameTempPartAndReplace(part, context.getCurrentTransaction().get(), &dest_table_storage->increment, &transaction, lock); removePartsFromWorkingSet(src_parts, true, lock); transaction.commit(&lock); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f9d63132a1b..ff97ffc1411 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1442,7 +1442,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) Transaction transaction(*this); - renameTempPartAndReplace(part, nullptr, &transaction); + renameTempPartAndReplace(part, nullptr, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, @@ -1810,7 +1810,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM new_part = merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, *merge_entry, entry.create_time, global_context, reserved_space, table_lock); - renameTempPartAndReplace(new_part, nullptr, &transaction); + renameTempPartAndReplace(new_part, nullptr, nullptr, &transaction); try { @@ -2433,7 +2433,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) Coordination::Requests ops; for (PartDescriptionPtr & part_desc : final_parts) { - renameTempPartAndReplace(part_desc->res_part, nullptr, &transaction); + renameTempPartAndReplace(part_desc->res_part, nullptr, nullptr, &transaction); getCommitPartOps(ops, part_desc->res_part); if (ops.size() > zkutil::MULTI_BATCH_SIZE) @@ -3802,7 +3802,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora if (!to_detached) { Transaction transaction(*this); - renameTempPartAndReplace(part, nullptr, &transaction); + renameTempPartAndReplace(part, nullptr, nullptr, &transaction); replaced_parts = checkPartChecksumsAndCommit(transaction, part); @@ -6131,7 +6131,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto data_parts_lock = lockParts(); for (MutableDataPartPtr & part : dst_parts) - renameTempPartAndReplace(part, nullptr, &transaction, data_parts_lock); + renameTempPartAndReplace(part, context.getCurrentTransaction().get(), nullptr, &transaction, data_parts_lock); } op_results = zookeeper->multi(ops); @@ -6322,7 +6322,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta DataPartsLock lock(mutex); for (MutableDataPartPtr & part : dst_parts) - dest_table_storage->renameTempPartAndReplace(part, nullptr, &transaction, lock); + dest_table_storage->renameTempPartAndReplace(part, query_context.getCurrentTransaction().get(), nullptr, &transaction, lock); op_results = zookeeper->multi(ops); diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index f100ecdfa9b..030226d86dc 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -272,11 +272,11 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(get_tid_as_field(part->versions.mintid)); if (columns_mask[src_index++]) - columns[res_index++]->insert(get_tid_as_field(part->versions.maxtid)); + columns[res_index++]->insert(get_tid_as_field(part->versions.getMaxTID())); if (columns_mask[src_index++]) - columns[res_index++]->insert(part->versions.mincsn); + columns[res_index++]->insert(part->versions.mincsn.load(std::memory_order_relaxed)); if (columns_mask[src_index++]) - columns[res_index++]->insert(part->versions.maxcsn); + columns[res_index++]->insert(part->versions.maxcsn.load(std::memory_order_relaxed)); } } diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.reference b/tests/queries/0_stateless/01174_select_insert_isolation.reference new file mode 100644 index 00000000000..51d57dfa0d6 --- /dev/null +++ b/tests/queries/0_stateless/01174_select_insert_isolation.reference @@ -0,0 +1 @@ +200 0 100 100 0 diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh new file mode 100755 index 00000000000..23fa9b4da32 --- /dev/null +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -0,0 +1,60 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mt"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; +$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES mt"; #FIXME + +function thread_insert_commit() +{ + for i in {1..50}; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO mt VALUES ($i, $1); + INSERT INTO mt VALUES (-$i, $1); + COMMIT;"; + done +} + +function thread_insert_rollback() +{ + for _ in {1..50}; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO mt VALUES (42, $1); + ROLLBACK;"; + done +} + +function thread_select() +{ + trap "exit 0" INT + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + SELECT arraySort(groupArray(n)), arraySort(groupArray(m)) FROM mt; + SELECT throwIf((SELECT sum(n) FROM mt) != 0) FORMAT Null; + SELECT throwIf((SELECT count() FROM mt) % 2 != 0) FORMAT Null; + SELECT arraySort(groupArray(n)), arraySort(groupArray(m)) FROM mt; + COMMIT;" | uniq | wc -l | grep -v "^1$" ||:; # Must be 1 if the first and the last queries got the same result + done +} + +thread_insert_commit 1 & PID_1=$! +thread_insert_commit 2 & PID_2=$! +thread_insert_rollback 3 & PID_3=$! +thread_select & PID_4=$! +wait $PID_1 && wait $PID_2 && wait $PID_3 +kill -INT $PID_4 +wait + +$CLICKHOUSE_CLIENT --multiquery --query " +BEGIN TRANSACTION; +SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; + +$CLICKHOUSE_CLIENT --query "DROP TABLE mt"; From f270fa1843ef20fdc2bc16ab068d5b0a838067bf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Apr 2021 15:53:51 +0300 Subject: [PATCH 003/372] rollback on exception --- src/Interpreters/Context.cpp | 6 ++ src/Interpreters/Context.h | 6 +- .../InterpreterTransactionControlQuery.cpp | 2 +- src/Interpreters/MergeTreeTransaction.cpp | 32 ++++++-- src/Interpreters/MergeTreeTransaction.h | 14 ++-- .../MergeTreeTransactionHolder.cpp | 70 ++++++++++++++++++ src/Interpreters/MergeTreeTransactionHolder.h | 32 ++++++++ src/Interpreters/TransactionLog.cpp | 14 ++-- src/Interpreters/TransactionLog.h | 2 +- src/Interpreters/executeQuery.cpp | 8 +- .../01172_transaction_counters.reference | 10 +++ .../01172_transaction_counters.sql | 22 ++++++ ...1173_transaction_control_queries.reference | 9 +++ .../01173_transaction_control_queries.sql | 74 +++++++++++++++++++ 14 files changed, 276 insertions(+), 25 deletions(-) create mode 100644 src/Interpreters/MergeTreeTransactionHolder.cpp create mode 100644 src/Interpreters/MergeTreeTransactionHolder.h create mode 100644 tests/queries/0_stateless/01172_transaction_counters.reference create mode 100644 tests/queries/0_stateless/01172_transaction_counters.sql create mode 100644 tests/queries/0_stateless/01173_transaction_control_queries.reference create mode 100644 tests/queries/0_stateless/01173_transaction_control_queries.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5c6afb38ff4..933d613f676 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2581,6 +2581,12 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const return metadata_transaction; } +void Context::initCurrentTransaction(MergeTreeTransactionPtr txn) +{ + merge_tree_transaction_holder = MergeTreeTransactionHolder(txn, false); + setCurrentTransaction(std::move(txn)); +} + void Context::setCurrentTransaction(MergeTreeTransactionPtr txn) { assert(!merge_tree_transaction || !txn); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e035031f993..3e621b61225 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -23,6 +23,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -122,8 +123,6 @@ struct BackgroundTaskSchedulingSettings; class ZooKeeperMetadataTransaction; using ZooKeeperMetadataTransactionPtr = std::shared_ptr; -class MergeTreeTransaction; -using MergeTreeTransactionPtr = std::shared_ptr; #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; @@ -293,6 +292,8 @@ private: /// 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(); @@ -761,6 +762,7 @@ public: /// Returns context of current distributed DDL query or nullptr. ZooKeeperMetadataTransactionPtr getZooKeeperMetadataTransaction() const; + void initCurrentTransaction(MergeTreeTransactionPtr txn); void setCurrentTransaction(MergeTreeTransactionPtr txn); MergeTreeTransactionPtr getCurrentTransaction() const; diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index 369f5edd1da..8038d53e118 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -37,7 +37,7 @@ BlockIO InterpreterTransactionControlQuery::executeBegin(Context & context) throw Exception(ErrorCodes::INVALID_TRANSACTION, "Nested transactions are not supported"); auto txn = TransactionLog::instance().beginTransaction(); - context.setCurrentTransaction(txn); + context.initCurrentTransaction(txn); query_context.setCurrentTransaction(txn); return {}; } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 5b267e45f83..2f8445530da 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -8,10 +9,19 @@ namespace DB MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id) : tid({snapshot_, local_tid_, host_id}) , snapshot(snapshot_) - , state(RUNNING) + , csn(Tx::UnknownCSN) { } +MergeTreeTransaction::State MergeTreeTransaction::getState() const +{ + if (csn == Tx::UnknownCSN) + return RUNNING; + if (csn == Tx::RolledBackCSN) + return ROLLED_BACK; + return COMMITTED; +} + void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; @@ -69,25 +79,35 @@ bool MergeTreeTransaction::isReadOnly() const void MergeTreeTransaction::beforeCommit() { - assert(state == RUNNING); + assert(csn == Tx::UnknownCSN); } -void MergeTreeTransaction::afterCommit() +void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept { - assert(state == COMMITTED); + assert(csn == Tx::UnknownCSN); + csn = assigned_csn; for (const auto & part : creating_parts) part->versions.mincsn.store(csn); for (const auto & part : removing_parts) part->versions.maxcsn.store(csn); } -void MergeTreeTransaction::rollback() +void MergeTreeTransaction::rollback() noexcept { - assert(state == RUNNING); + assert(csn == Tx::UnknownCSN); + csn = Tx::RolledBackCSN; for (const auto & part : creating_parts) part->versions.mincsn.store(Tx::RolledBackCSN); for (const auto & part : removing_parts) part->versions.unlockMaxTID(tid); } +void MergeTreeTransaction::onException() +{ + if (csn) + return; + + TransactionLog::instance().rollbackTransaction(shared_from_this()); +} + } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index c085fa96b0f..c9fad85f612 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -8,7 +9,7 @@ class IMergeTreeDataPart; using DataPartPtr = std::shared_ptr; using DataPartsVector = std::vector; -class MergeTreeTransaction +class MergeTreeTransaction : public std::enable_shared_from_this, private boost::noncopyable { friend class TransactionLog; public: @@ -20,7 +21,7 @@ public: }; Snapshot getSnapshot() const { return snapshot; } - State getState() const { return state; } + State getState() const; const TransactionID tid; @@ -36,18 +37,19 @@ public: bool isReadOnly() const; + void onException(); + private: void beforeCommit(); - void afterCommit(); - void rollback(); + void afterCommit(CSN assigned_csn) noexcept; + void rollback() noexcept; Snapshot snapshot; - State state; DataPartsVector creating_parts; DataPartsVector removing_parts; - CSN csn = Tx::UnknownCSN; + CSN csn; }; using MergeTreeTransactionPtr = std::shared_ptr; diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp new file mode 100644 index 00000000000..1de0d9c75c4 --- /dev/null +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -0,0 +1,70 @@ +#include +#include +#include + +namespace DB +{ + +MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_ = false) + : txn(txn_) + , autocommit(autocommit_) +{ + assert(!txn || txn->getState() == MergeTreeTransaction::RUNNING); +} + +MergeTreeTransactionHolder::MergeTreeTransactionHolder(MergeTreeTransactionHolder && rhs) noexcept + : txn(std::move(rhs.txn)) + , autocommit(rhs.autocommit) +{ + rhs.txn = {}; +} + +MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(MergeTreeTransactionHolder && rhs) noexcept +{ + onDestroy(); + txn = std::move(rhs.txn); + autocommit = rhs.autocommit; + rhs.txn = {}; + return *this; +} + +MergeTreeTransactionHolder::~MergeTreeTransactionHolder() +{ + onDestroy(); +} + +void MergeTreeTransactionHolder::onDestroy() noexcept +{ + if (!txn) + return; + if (txn->getState() != MergeTreeTransaction::RUNNING) + return; + + if (autocommit) + { + try + { + TransactionLog::instance().commitTransaction(txn); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } else + { + TransactionLog::instance().rollbackTransaction(txn); + } +} + +MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionHolder &) +{ + txn = nullptr; +} + +MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder &) +{ + txn = nullptr; + return *this; +} + +} diff --git a/src/Interpreters/MergeTreeTransactionHolder.h b/src/Interpreters/MergeTreeTransactionHolder.h new file mode 100644 index 00000000000..6172beb2a69 --- /dev/null +++ b/src/Interpreters/MergeTreeTransactionHolder.h @@ -0,0 +1,32 @@ +#pragma once +#include + +namespace DB +{ + +class MergeTreeTransaction; +using MergeTreeTransactionPtr = std::shared_ptr; + +class MergeTreeTransactionHolder +{ +public: + MergeTreeTransactionHolder() = default; + MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_); + MergeTreeTransactionHolder(MergeTreeTransactionHolder && rhs) noexcept; + MergeTreeTransactionHolder & operator=(MergeTreeTransactionHolder && rhs) noexcept; + ~MergeTreeTransactionHolder(); + + /// NOTE: We cannot make it noncopyable, because we use it as a filed 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); + +private: + void onDestroy() noexcept; + + MergeTreeTransactionPtr txn; + bool autocommit = false; +}; + +} diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 4b4f893f791..dac5f48abd4 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -62,27 +62,25 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) latest_snapshot.store(new_csn, std::memory_order_relaxed); } - txn->csn = new_csn; - txn->state = MergeTreeTransaction::COMMITTED; - txn->afterCommit(); + txn->afterCommit(new_csn); + { 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); } - return txn->csn; + return new_csn; } -void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) +void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { - txn->csn = Tx::RolledBackCSN; - txn->state = MergeTreeTransaction::ROLLED_BACK; + txn->rollback(); { 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); + abort(); } } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 8766ab24251..616cf62f242 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -21,7 +21,7 @@ public: CSN commitTransaction(const MergeTreeTransactionPtr & txn); - void rollbackTransaction(const MergeTreeTransactionPtr & txn); + void rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept; CSN getCSN(const TransactionID & tid) const; CSN getCSN(const TIDHash & tid) const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a00f8301622..819c17af3ae 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -50,7 +50,7 @@ #include #include #include -#include +#include #include #include @@ -825,6 +825,9 @@ static std::tuple 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(Quota::ERRORS, 1, /* check_exceeded = */ false); @@ -888,6 +891,9 @@ static std::tuple executeQueryImpl( } catch (...) { + if (auto txn = context.getCurrentTransaction()) + txn->onException(); + if (!internal) { if (query_for_logging.empty()) diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference new file mode 100644 index 00000000000..375f3540af9 --- /dev/null +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -0,0 +1,10 @@ +(0,0,'00000000-0000-0000-0000-000000000000') +all_1_1_0 0 +all_2_2_0 1 +all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 +all_2_2_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +all_1_1_0 0 +all_3_3_0 1 +all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 +all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 +all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql new file mode 100644 index 00000000000..db7aa639f5d --- /dev/null +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -0,0 +1,22 @@ +drop table if exists txn_counters; + +create table txn_counters (n Int64, mintid DEFAULT transactionID()) engine=MergeTree order by n; + +system stop merges txn_counters; --FIXME + +insert into txn_counters(n) values (1); +select transactionID(); + +begin transaction; +insert into txn_counters(n) values (2); +select system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +rollback; + +begin transaction; +insert into txn_counters(n) values (3); +select system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +commit; + +drop table txn_counters; diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.reference b/tests/queries/0_stateless/01173_transaction_control_queries.reference new file mode 100644 index 00000000000..a9cd56425f1 --- /dev/null +++ b/tests/queries/0_stateless/01173_transaction_control_queries.reference @@ -0,0 +1,9 @@ +commit [1,10] +rollback [1,2,10,20] +no nested [1,10] +on exception before start [1,3,10,30] +on exception while processing [1,4,10,40] +1 +on session close [1,6,10,60] +commit [1,7,10,70] +readonly [1,7,10,70] diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql new file mode 100644 index 00000000000..780259dc158 --- /dev/null +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -0,0 +1,74 @@ +drop table if exists mt1; +drop table if exists mt2; + +create table mt1 (n Int64) engine=MergeTree order by n; +create table mt2 (n Int64) engine=MergeTree order by n; +system stop merges mt1; --FIXME +system stop merges mt2; --FIXME + +commit; -- { serverError 581 } +rollback; -- { serverError 581 } + +begin transaction; +insert into mt1 values (1); +insert into mt2 values (10); +select 'commit', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +commit; + +begin transaction; +insert into mt1 values (2); +insert into mt2 values (20); +select 'rollback', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +rollback; + +begin transaction; +select 'no nested', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +begin transaction; -- { serverError 581 } +rollback; + +begin transaction; +insert into mt1 values (3); +insert into mt2 values (30); +select 'on exception before start', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +-- rollback on exception before start +select functionThatDoesNotExist(); -- { serverError 46 } +-- cannot commit after exception +commit; -- { serverError 581 } +begin transaction; -- { serverError 581 } +rollback; + +begin transaction; +insert into mt1 values (4); +insert into mt2 values (40); +select 'on exception while processing', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +-- rollback on exception while processing +select throwIf(100 < number) from numbers(1000); -- { serverError 395 } +-- cannot commit after exception +commit; -- { serverError 581 } +-- FIXME Transactions: do not allow queries after exception +insert into mt1 values (5); +insert into mt2 values (50); +select 1; +rollback; + +begin transaction; +insert into mt1 values (6); +insert into mt2 values (60); +select 'on session close', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +-- trigger reconnection by error on client, check rollback on session close +insert into mt1 values ([1]); -- { clientError 43 } +commit; -- { serverError 581 } +rollback; -- { serverError 581 } + +begin transaction; +insert into mt1 values (7); +insert into mt2 values (70); +select 'commit', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +commit; + +begin transaction; +select 'readonly', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); +commit; + +drop table mt1; +drop table mt2; From 1fd6142c1a1242cb2ae2cd47dcc431979d3fc55e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 17 May 2021 14:14:09 +0300 Subject: [PATCH 004/372] try enable merges --- src/Interpreters/MergeTreeTransaction.cpp | 2 ++ .../MergeTreeTransactionHolder.cpp | 10 +++--- src/Interpreters/MergeTreeTransactionHolder.h | 4 ++- src/Interpreters/TransactionLog.h | 1 + src/Interpreters/executeQuery.cpp | 9 +++++ src/Storages/MergeTree/MergeTreeData.cpp | 12 +++---- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 16 +++++++-- .../MergeTree/MergeTreeDataMergerMutator.h | 2 ++ src/Storages/StorageMergeTree.cpp | 35 ++++++++++++++----- src/Storages/StorageMergeTree.h | 18 ++++++++-- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++-- ...1173_transaction_control_queries.reference | 1 - .../01173_transaction_control_queries.sql | 11 +++--- .../01174_select_insert_isolation.sh | 12 ++++--- 15 files changed, 101 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 2f8445530da..93f7524123a 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -64,11 +64,13 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const DataPartPtr & new_pa void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part) { + assert(csn == Tx::UnknownCSN); creating_parts.push_back(new_part); } void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove) { + assert(csn == Tx::UnknownCSN); removing_parts.push_back(part_to_remove); } diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index 1de0d9c75c4..088f6ef8f05 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -23,8 +23,8 @@ MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(MergeTreeTran { onDestroy(); txn = std::move(rhs.txn); - autocommit = rhs.autocommit; rhs.txn = {}; + autocommit = rhs.autocommit; return *this; } @@ -40,20 +40,20 @@ void MergeTreeTransactionHolder::onDestroy() noexcept if (txn->getState() != MergeTreeTransaction::RUNNING) return; - if (autocommit) + if (autocommit && std::uncaught_exceptions() == 0) { try { TransactionLog::instance().commitTransaction(txn); + return; } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - } else - { - TransactionLog::instance().rollbackTransaction(txn); } + + TransactionLog::instance().rollbackTransaction(txn); } MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionHolder &) diff --git a/src/Interpreters/MergeTreeTransactionHolder.h b/src/Interpreters/MergeTreeTransactionHolder.h index 6172beb2a69..ec9cf1e1636 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.h +++ b/src/Interpreters/MergeTreeTransactionHolder.h @@ -16,12 +16,14 @@ public: MergeTreeTransactionHolder & operator=(MergeTreeTransactionHolder && rhs) noexcept; ~MergeTreeTransactionHolder(); - /// NOTE: We cannot make it noncopyable, because we use it as a filed of Context. + /// 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; diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 616cf62f242..11e896b0a2d 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include #include #include diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ec725566a64..77ebe1322ec 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -78,6 +79,7 @@ namespace ErrorCodes { extern const int INTO_OUTFILE_NOT_ALLOWED; extern const int QUERY_WAS_CANCELLED; + extern const int INVALID_TRANSACTION; } @@ -396,6 +398,13 @@ static std::tuple executeQueryImpl( ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); #endif + if (auto txn = context->getCurrentTransaction()) + { + assert(txn->getState() != MergeTreeTransaction::COMMITTED); + if (txn->getState() == MergeTreeTransaction::ROLLED_BACK && !ast->as()) + 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()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1119715bbd1..2b1d5bf5e83 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3218,14 +3218,14 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc DataPartsVector MergeTreeData::getDataPartsVector(ContextPtr local_context) const { - if (auto txn = local_context->getCurrentTransaction()) - return getVisibleDataPartsVector(*txn); - else - return getDataPartsVector(); + return getVisibleDataPartsVector(local_context->getCurrentTransaction()); } -MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const MergeTreeTransaction & txn) const +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const { + if (!txn) + return getDataPartsVector(); + DataPartsVector maybe_visible_parts = getDataPartsVector({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); if (maybe_visible_parts.empty()) return maybe_visible_parts; @@ -3234,7 +3234,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me auto it_last = maybe_visible_parts.end() - 1; while (it <= it_last) { - if ((*it)->versions.isVisible(txn)) + if ((*it)->versions.isVisible(*txn)) { ++it; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index eab330babb9..21fb24379aa 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -423,7 +423,7 @@ public: DataPartsVector getDataPartsVector() const; DataPartsVector getDataPartsVector(ContextPtr local_context) const; - DataPartsVector getVisibleDataPartsVector(const MergeTreeTransaction & txn) const; + DataPartsVector getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const; /// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr. DataPartPtr getActiveContainingPart(const String & part_name) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index cd186c0d38d..7a4de0811a5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -213,9 +213,13 @@ 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 = data.getVisibleDataPartsVector(txn); + //FIXME get rid of sorting + std::sort(data_parts.begin(), data_parts.end(), MergeTreeData::LessDataPart()); const auto data_settings = data.getSettings(); auto metadata_snapshot = data.getInMemoryMetadataPtr(); @@ -1461,10 +1465,18 @@ MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm( MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, const MergeTreeData::DataPartsVector & parts, + const MergeTreeTransactionPtr & txn, MergeTreeData::Transaction * out_transaction) { /// Rename new part, add to the set and remove original parts. - auto replaced_parts = data.renameTempPartAndReplace(new_data_part, nullptr, nullptr, out_transaction); + auto replaced_parts = data.renameTempPartAndReplace(new_data_part, txn.get(), nullptr, out_transaction); + //String parts_str; + //for (const auto & p : parts) + // parts_str += "\t" + p->name; + //String parts_str2; + //for (const auto & p : replaced_parts) + // parts_str2 += "\t" + p->name; + //LOG_ERROR(log, "WTF {}: source {}, replaced {}", new_data_part->name, parts_str, parts_str2); /// Let's check that all original parts have been deleted and only them. if (replaced_parts.size() != parts.size()) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index b082d063dcf..c76179ce435 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -93,6 +93,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. @@ -148,6 +149,7 @@ public: MergeTreeData::DataPartPtr renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, const MergeTreeData::DataPartsVector & parts, + const MergeTreeTransactionPtr & txn, MergeTreeData::Transaction * out_transaction = nullptr); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a6342818ac1..ef85abad126 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -676,7 +677,14 @@ void StorageMergeTree::loadMutations() } std::shared_ptr StorageMergeTree::selectPartsToMerge( - const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, bool optimize_skip_merged_partitions, SelectPartsDecision * select_decision_out) + const StorageMetadataPtr & metadata_snapshot, + bool aggressive, const String & partition_id, + bool final, + String * out_disable_reason, + TableLockHolder & /* table_lock_holder */, + const MergeTreeTransactionPtr & txn, + bool optimize_skip_merged_partitions, + SelectPartsDecision * select_decision_out) { std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -718,6 +726,7 @@ std::shared_ptr StorageMergeTree::se max_source_parts_size, can_merge, merge_with_ttl_allowed, + txn, out_disable_reason); } else if (out_disable_reason) @@ -787,6 +796,7 @@ bool StorageMergeTree::merge( bool final, bool deduplicate, const Names & deduplicate_by_columns, + const MergeTreeTransactionPtr & txn, String * out_disable_reason, bool optimize_skip_merged_partitions) { @@ -795,7 +805,7 @@ bool StorageMergeTree::merge( SelectPartsDecision select_decision; - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, optimize_skip_merged_partitions, &select_decision); + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, txn, optimize_skip_merged_partitions, &select_decision); /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) @@ -804,7 +814,7 @@ bool StorageMergeTree::merge( if (!merge_mutate_entry) return false; - return mergeSelectedParts(metadata_snapshot, deduplicate, deduplicate_by_columns, *merge_mutate_entry, table_lock_holder); + return mergeSelectedParts(metadata_snapshot, deduplicate, deduplicate_by_columns, *merge_mutate_entry, table_lock_holder, txn); } bool StorageMergeTree::mergeSelectedParts( @@ -812,7 +822,8 @@ bool StorageMergeTree::mergeSelectedParts( bool deduplicate, const Names & deduplicate_by_columns, MergeMutateSelectedEntry & merge_mutate_entry, - TableLockHolder & table_lock_holder) + TableLockHolder & table_lock_holder, + const MergeTreeTransactionPtr & txn) { auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; @@ -847,7 +858,7 @@ bool StorageMergeTree::mergeSelectedParts( deduplicate_by_columns, merging_params); - merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); + merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, txn, nullptr); write_part_log({}); } catch (...) @@ -1000,20 +1011,24 @@ std::optional StorageMergeTree::getDataProcessingJob() //-V657 if (merger_mutator.merges_blocker.isCancelled()) return {}; + /// FIXME Transactions: do not begin transaction if we don't need it + auto txn = TransactionLog::instance().beginTransaction(); + MergeTreeTransactionHolder autocommit{txn, true}; + auto metadata_snapshot = getInMemoryMetadataPtr(); std::shared_ptr merge_entry, mutate_entry; auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock); + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, txn); if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock, holder = std::move(autocommit)] () mutable { if (merge_entry) - return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock); + return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock, holder.getTransaction()); else if (mutate_entry) return mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock); @@ -1110,6 +1125,8 @@ bool StorageMergeTree::optimize( LOG_DEBUG(log, "DEDUPLICATE BY ('{}')", fmt::join(deduplicate_by_columns, "', '")); } + auto txn = local_context->getCurrentTransaction(); + String disable_reason; if (!partition && final) { @@ -1127,6 +1144,7 @@ bool StorageMergeTree::optimize( true, deduplicate, deduplicate_by_columns, + txn, &disable_reason, local_context->getSettingsRef().optimize_skip_merged_partitions)) { @@ -1153,6 +1171,7 @@ bool StorageMergeTree::optimize( final, deduplicate, deduplicate_by_columns, + txn, &disable_reason, local_context->getSettingsRef().optimize_skip_merged_partitions)) { diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 53199e1595a..8881008bc89 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -142,7 +142,14 @@ private: * If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query). * Returns true if merge is finished successfully. */ - bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, const Names & deduplicate_by_columns, String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false); + bool merge( + bool aggressive, + const String & partition_id, + bool final, bool deduplicate, + const Names & deduplicate_by_columns, + const MergeTreeTransactionPtr & txn, + String * out_disable_reason = nullptr, + bool optimize_skip_merged_partitions = false); /// Make part state outdated and queue it to remove without timeout /// If force, then stop merges and block them until part state became outdated. Throw exception if part doesn't exists @@ -196,10 +203,17 @@ private: bool final, String * disable_reason, TableLockHolder & table_lock_holder, + const MergeTreeTransactionPtr & txn, bool optimize_skip_merged_partitions = false, SelectPartsDecision * select_decision_out = nullptr); - bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, const Names & deduplicate_by_columns, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); + bool mergeSelectedParts( + const StorageMetadataPtr & metadata_snapshot, + bool deduplicate, + const Names & deduplicate_by_columns, + MergeMutateSelectedEntry & entry, + TableLockHolder & table_lock_holder, + const MergeTreeTransactionPtr & txn); std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5059a3bf171..46c7ffd2274 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1690,7 +1690,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) entry.deduplicate_by_columns, merging_params); - merger_mutator.renameMergedTemporaryPart(part, parts, &transaction); + merger_mutator.renameMergedTemporaryPart(part, parts, nullptr, &transaction); try { @@ -3078,7 +3078,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() future_merged_part.uuid = UUIDHelpers::generateV4(); if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr) == SelectPartsDecision::SELECTED) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr, nullptr) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts( zookeeper, @@ -4446,7 +4446,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { select_decision = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, false, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, false, nullptr, &disable_reason); } else { diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.reference b/tests/queries/0_stateless/01173_transaction_control_queries.reference index a9cd56425f1..6e27f29012b 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.reference +++ b/tests/queries/0_stateless/01173_transaction_control_queries.reference @@ -3,7 +3,6 @@ rollback [1,2,10,20] no nested [1,10] on exception before start [1,3,10,30] on exception while processing [1,4,10,40] -1 on session close [1,6,10,60] commit [1,7,10,70] readonly [1,7,10,70] diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index 486b738e724..02397b85cb8 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -3,8 +3,8 @@ drop table if exists mt2; create table mt1 (n Int64) engine=MergeTree order by n; create table mt2 (n Int64) engine=MergeTree order by n; -system stop merges mt1; --FIXME -system stop merges mt2; --FIXME +--system stop merges mt1; --FIXME +--system stop merges mt2; --FIXME commit; -- { serverError 585 } rollback; -- { serverError 585 } @@ -45,10 +45,9 @@ select 'on exception while processing', arraySort(groupArray(n)) from (select n select throwIf(100 < number) from numbers(1000); -- { serverError 395 } -- cannot commit after exception commit; -- { serverError 585 } --- FIXME Transactions: do not allow queries after exception -insert into mt1 values (5); -insert into mt2 values (50); -select 1; +insert into mt1 values (5); -- { serverError 585 } +insert into mt2 values (50); -- { serverError 585 } +select 1; -- { serverError 585 } rollback; begin transaction; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 23fa9b4da32..5697a81bedd 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -8,7 +8,7 @@ set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mt"; $CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; -$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES mt"; #FIXME +#$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES mt"; #FIXME function thread_insert_commit() { @@ -35,13 +35,15 @@ function thread_select() { trap "exit 0" INT while true; do + # Result of `uniq | wc -l` must be 1 if the first and the last queries got the same result $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - SELECT arraySort(groupArray(n)), arraySort(groupArray(m)) FROM mt; + SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt; SELECT throwIf((SELECT sum(n) FROM mt) != 0) FORMAT Null; SELECT throwIf((SELECT count() FROM mt) % 2 != 0) FORMAT Null; - SELECT arraySort(groupArray(n)), arraySort(groupArray(m)) FROM mt; - COMMIT;" | uniq | wc -l | grep -v "^1$" ||:; # Must be 1 if the first and the last queries got the same result + SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt; + COMMIT;" | tee -a ./wtf.log | uniq | wc -l | grep -v "^1$" && $CLICKHOUSE_CLIENT -q "SELECT * FROM system.parts + WHERE database='$CLICKHOUSE_DATABASE' AND table='mt'" ||:; done } @@ -57,4 +59,4 @@ $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; -$CLICKHOUSE_CLIENT --query "DROP TABLE mt"; +#$CLICKHOUSE_CLIENT --query "DROP TABLE mt"; From 529d1aeb197b72e46a01aeb1c6d48a48983648e4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 18 May 2021 20:07:29 +0300 Subject: [PATCH 005/372] fix merges of uncommitted parts --- base/daemon/BaseDaemon.cpp | 2 +- src/Common/TransactionMetadata.cpp | 7 ++-- src/Interpreters/MergeTreeTransaction.cpp | 21 +++++++++++ src/Interpreters/MergeTreeTransaction.h | 2 ++ src/Interpreters/TransactionLog.cpp | 8 +++++ src/Interpreters/TransactionLog.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 6 +++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 13 ++++--- .../MergeTree/MergeTreeDataMergerMutator.h | 5 ++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 1 + .../MergeTree/ReplicatedMergeTreeQueue.h | 1 + src/Storages/StorageMergeTree.cpp | 35 ++++++++++++++----- .../01172_transaction_counters.sql | 2 -- .../01173_transaction_control_queries.sql | 2 -- .../01174_select_insert_isolation.sh | 3 +- 15 files changed, 83 insertions(+), 27 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 83384038b7c..48f8bd30c4b 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -787,7 +787,7 @@ 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({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP}, signalHandler, &handled_signals); addSignalHandler({SIGHUP, SIGUSR1}, closeLogsSignalHandler, &handled_signals); addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals); diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp index a4dcebbf3ef..7116ede8e9a 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Common/TransactionMetadata.cpp @@ -123,9 +123,12 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) return false; /// Otherwise, part is definitely visible if: - /// - creation was committed after we took the snapshot and nobody tried to remove the part + /// - 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 (!max_lock && min && min <= snapshot_version) + if (min && min <= snapshot_version && !max_lock) + return true; + if (min && min <= snapshot_version && max && snapshot_version < max) return true; if (mintid == txn.tid) return true; diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 93f7524123a..726a92a6d68 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -112,4 +112,25 @@ void MergeTreeTransaction::onException() TransactionLog::instance().rollbackTransaction(shared_from_this()); } +String MergeTreeTransaction::dumpDescription() const +{ + String res = "\ncreating parts:\n"; + for (const auto & part : creating_parts) + { + res += part->name; + res += "\n"; + } + + res += "removing parts:\n"; + for (const auto & part : removing_parts) + { + res += part->name; + res += fmt::format(" (created by {}, {})\n", part->versions.getMinTID(), part->versions.mincsn); + assert(!part->versions.mincsn || part->versions.mincsn <= snapshot); + assert(!part->versions.maxcsn); + } + + return res; +} + } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index c9fad85f612..832ee92b230 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -39,6 +39,8 @@ public: void onException(); + String dumpDescription() const; + private: void beforeCommit(); void afterCommit(CSN assigned_csn) noexcept; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index dac5f48abd4..7e7357c68cc 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -17,6 +18,7 @@ TransactionLog & TransactionLog::instance() } TransactionLog::TransactionLog() + : log(&Poco::Logger::get("TransactionLog")) { latest_snapshot = 1; csn_counter = 1; @@ -39,6 +41,7 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); } + LOG_TRACE(log, "Beginning transaction {}", txn->tid); return txn; } @@ -50,10 +53,12 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// TODO Transactions: reset local_tid_counter if (txn->isReadOnly()) { + LOG_TRACE(log, "Closing readonly transaction {}", txn->tid); new_csn = txn->snapshot; } else { + LOG_TRACE(log, "Committing transaction {}{}", txn->tid, txn->dumpDescription()); std::lock_guard lock{commit_mutex}; new_csn = 1 + csn_counter.fetch_add(1); bool inserted = tid_to_csn.try_emplace(txn->tid.getHash(), new_csn).second; /// Commit point @@ -62,6 +67,8 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) latest_snapshot.store(new_csn, std::memory_order_relaxed); } + LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn); + txn->afterCommit(new_csn); { @@ -75,6 +82,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { + LOG_TRACE(log, "Rolling back transaction {}", txn->tid); txn->rollback(); { std::lock_guard lock{running_list_mutex}; diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 11e896b0a2d..d4894deac5c 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -30,6 +30,8 @@ public: MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); private: + Poco::Logger * log; + std::atomic latest_snapshot; std::atomic csn_counter; std::atomic local_tid_counter; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 09f4811fe71..29396bdf213 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1084,7 +1084,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); modifyPartState(it, DataPartState::Outdated); - (*it)->versions.maxtid = Tx::PrehistoricTID; + (*it)->versions.lockMaxTID(Tx::PrehistoricTID); (*it)->versions.maxcsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); removePartContributionToDataVolume(*it); }; @@ -3225,10 +3225,13 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me auto it = maybe_visible_parts.begin(); auto it_last = maybe_visible_parts.end() - 1; + String visible_parts_str; while (it <= it_last) { if ((*it)->versions.isVisible(*txn)) { + visible_parts_str += (*it)->name; + visible_parts_str += " "; ++it; } else @@ -3239,6 +3242,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me } size_t new_size = it_last - maybe_visible_parts.begin() + 1; + LOG_TRACE(log, "Got {} parts visible for {}: {}", new_size, txn->tid, visible_parts_str); maybe_visible_parts.resize(new_size); return maybe_visible_parts; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2a3f8608dee..289ba50e434 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -216,10 +216,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const MergeTreeTransactionPtr & txn, String * out_disable_reason) { - - MergeTreeData::DataPartsVector data_parts = data.getVisibleDataPartsVector(txn); - //FIXME get rid of sorting - std::sort(data_parts.begin(), data_parts.end(), MergeTreeData::LessDataPart()); + /// NOTE It will contain uncommitted parts and future parts. + /// But It's ok since merge predicate allows to include in range visible parts only. + MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); auto metadata_snapshot = data.getInMemoryMetadataPtr(); @@ -265,7 +264,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; } @@ -273,7 +272,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)) { /// Starting new interval in the same partition assert(!parts_ranges.back().empty()); @@ -415,7 +414,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, nullptr, out_disable_reason)) { return SelectPartsDecision::CANNOT_SELECT; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index c76179ce435..7340a86f4a7 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -61,7 +61,10 @@ struct FutureMergedMutatedPart class MergeTreeDataMergerMutator { public: - using AllowedMergingPredicate = std::function; + using AllowedMergingPredicate = std::function; MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index b81b5712594..1dd4962453e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1811,6 +1811,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( bool ReplicatedMergeTreeMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, + const MergeTreeTransaction *, String * out_reason) const { if (left) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 8b1028623b2..111168d198e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -451,6 +451,7 @@ public: /// Depending on the existence of left part checks a merge predicate for two parts or for single part. bool operator()(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, + const MergeTreeTransaction * txn, String * out_reason = nullptr) const; /// Can we assign a merge with these two parts? diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4e56076a28a..f21d039d937 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -698,8 +698,18 @@ std::shared_ptr StorageMergeTree::se CurrentlyMergingPartsTaggerPtr merging_tagger; MergeList::EntryPtr merge_entry; - auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, String *) -> bool + auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, String *) -> bool { + if (tx) + { + /// Cannot merge parts if some of them is not visible in current snapshot + /// TODO We can use simplified visibility rules (without CSN lookup) here + if (left && !left->versions.isVisible(*tx)) + return false; + if (right && !right->versions.isVisible(*tx)) + return false; + } + /// This predicate is checked for the first part of each range. /// (left = nullptr, right = "first part of partition") if (!left) @@ -1022,28 +1032,35 @@ std::optional StorageMergeTree::getDataProcessingJob() //-V657 if (merger_mutator.merges_blocker.isCancelled()) return {}; - /// FIXME Transactions: do not begin transaction if we don't need it - auto txn = TransactionLog::instance().beginTransaction(); - MergeTreeTransactionHolder autocommit{txn, true}; - auto metadata_snapshot = getInMemoryMetadataPtr(); std::shared_ptr merge_entry, mutate_entry; auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, txn); + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, nullptr); if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock, holder = std::move(autocommit)] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable { + merge_entry = {}; + mutate_entry = {}; + /// FIXME Transactions: do not begin transaction if we don't need it + auto txn = TransactionLog::instance().beginTransaction(); + MergeTreeTransactionHolder autocommit{txn, true}; + + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, txn); + if (!merge_entry) + mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); + if (merge_entry) - return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock, holder.getTransaction()); + return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock, txn); else if (mutate_entry) return mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock); - __builtin_unreachable(); + return true; + //__builtin_unreachable(); }, PoolType::MERGE_MUTATE}; } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index db7aa639f5d..2637041bcf7 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -2,8 +2,6 @@ drop table if exists txn_counters; create table txn_counters (n Int64, mintid DEFAULT transactionID()) engine=MergeTree order by n; -system stop merges txn_counters; --FIXME - insert into txn_counters(n) values (1); select transactionID(); diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index 02397b85cb8..b85d34f667b 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -3,8 +3,6 @@ drop table if exists mt2; create table mt1 (n Int64) engine=MergeTree order by n; create table mt2 (n Int64) engine=MergeTree order by n; ---system stop merges mt1; --FIXME ---system stop merges mt2; --FIXME commit; -- { serverError 585 } rollback; -- { serverError 585 } diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 5697a81bedd..e692b36e1af 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -8,7 +8,6 @@ set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mt"; $CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; -#$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES mt"; #FIXME function thread_insert_commit() { @@ -59,4 +58,4 @@ $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; -#$CLICKHOUSE_CLIENT --query "DROP TABLE mt"; +$CLICKHOUSE_CLIENT --query "DROP TABLE mt"; From 19337dc227338258b3c8e71a08d78cc0f3963645 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 18 May 2021 22:34:49 +0300 Subject: [PATCH 006/372] add config --- src/Interpreters/Context.cpp | 2 +- src/Storages/System/StorageSystemParts.cpp | 10 +++++----- tests/config/install.sh | 1 + tests/config/transactions.xml | 3 +++ 4 files changed, 10 insertions(+), 6 deletions(-) create mode 100644 tests/config/transactions.xml diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 88cba971f39..ec4c4625f95 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2656,7 +2656,7 @@ void Context::setCurrentTransaction(MergeTreeTransactionPtr txn) { assert(!merge_tree_transaction || !txn); assert(this == session_context.lock().get() || this == query_context.lock().get()); - int enable_mvcc_test_helper = getConfigRef().getInt("_enable_mvcc_test_helper_dev", 0); + int enable_mvcc_test_helper = getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0); if (enable_mvcc_test_helper != 42) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); merge_tree_transaction = std::move(txn); diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 030226d86dc..d3d1f4c85ee 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -259,11 +259,6 @@ void StorageSystemParts::processNextStorage( add_ttl_info_map(part->ttl_infos.group_by_ttl); add_ttl_info_map(part->ttl_infos.rows_where_ttl); - /// _state column should be the latest. - /// Do not use part->getState*, it can be changed from different thread - if (has_state_column) - columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state)); - auto get_tid_as_field = [](const TransactionID & tid) -> Field { return Tuple{tid.start_csn, tid.local_tid, tid.host_id}; @@ -277,6 +272,11 @@ void StorageSystemParts::processNextStorage( columns[res_index++]->insert(part->versions.mincsn.load(std::memory_order_relaxed)); if (columns_mask[src_index++]) columns[res_index++]->insert(part->versions.maxcsn.load(std::memory_order_relaxed)); + + /// _state column should be the latest. + /// Do not use part->getState*, it can be changed from different thread + if (has_state_column) + columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state)); } } diff --git a/tests/config/install.sh b/tests/config/install.sh index 7e01860e241..43e04a8109e 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -46,6 +46,7 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/transactions.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/ diff --git a/tests/config/transactions.xml b/tests/config/transactions.xml new file mode 100644 index 00000000000..731a312ed58 --- /dev/null +++ b/tests/config/transactions.xml @@ -0,0 +1,3 @@ + + <_enable_experimental_mvcc_prototype_test_helper_dev>42 + From 02d966dcc95850ecb1a1ec335a8b94008ef73aad Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 May 2021 21:28:36 +0300 Subject: [PATCH 007/372] fix --- .../FunctionsTransactionCounters.cpp | 2 +- tests/config/{ => config.d}/transactions.xml | 0 tests/config/install.sh | 2 +- .../01173_transaction_control_queries.sql | 22 +++++++++---------- 4 files changed, 13 insertions(+), 13 deletions(-) rename tests/config/{ => config.d}/transactions.xml (100%) diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp index b09a5e3e064..81ea7bdeaea 100644 --- a/src/Functions/FunctionsTransactionCounters.cpp +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -14,7 +14,7 @@ class FunctionTransactionID : public IFunction public: static constexpr auto name = "transactionID"; - static FunctionPtr create(ContextPtr context) + static FunctionPtr create(ContextConstPtr context) { return std::make_shared(context->getCurrentTransaction()); } diff --git a/tests/config/transactions.xml b/tests/config/config.d/transactions.xml similarity index 100% rename from tests/config/transactions.xml rename to tests/config/config.d/transactions.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index 43e04a8109e..7fe5f35c3a4 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -34,6 +34,7 @@ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ @@ -46,7 +47,6 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/ -ln -sf $SRC_PATH/transactions.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/ diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index b85d34f667b..0780efb7a7a 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -4,8 +4,8 @@ drop table if exists mt2; create table mt1 (n Int64) engine=MergeTree order by n; create table mt2 (n Int64) engine=MergeTree order by n; -commit; -- { serverError 585 } -rollback; -- { serverError 585 } +commit; -- { serverError 586 } +rollback; -- { serverError 586 } begin transaction; insert into mt1 values (1); @@ -21,7 +21,7 @@ rollback; begin transaction; select 'no nested', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); -begin transaction; -- { serverError 585 } +begin transaction; -- { serverError 586 } rollback; begin transaction; @@ -31,8 +31,8 @@ select 'on exception before start', arraySort(groupArray(n)) from (select n from -- rollback on exception before start select functionThatDoesNotExist(); -- { serverError 46 } -- cannot commit after exception -commit; -- { serverError 585 } -begin transaction; -- { serverError 585 } +commit; -- { serverError 586 } +begin transaction; -- { serverError 586 } rollback; begin transaction; @@ -42,10 +42,10 @@ select 'on exception while processing', arraySort(groupArray(n)) from (select n -- rollback on exception while processing select throwIf(100 < number) from numbers(1000); -- { serverError 395 } -- cannot commit after exception -commit; -- { serverError 585 } -insert into mt1 values (5); -- { serverError 585 } -insert into mt2 values (50); -- { serverError 585 } -select 1; -- { serverError 585 } +commit; -- { serverError 586 } +insert into mt1 values (5); -- { serverError 586 } +insert into mt2 values (50); -- { serverError 586 } +select 1; -- { serverError 586 } rollback; begin transaction; @@ -54,8 +54,8 @@ insert into mt2 values (60); select 'on session close', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); -- trigger reconnection by error on client, check rollback on session close insert into mt1 values ([1]); -- { clientError 43 } -commit; -- { serverError 585 } -rollback; -- { serverError 585 } +commit; -- { serverError 586 } +rollback; -- { serverError 586 } begin transaction; insert into mt1 values (7); From 9a9e95172f7666dd1d7870d40ad9f71a3e1ca341 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Jun 2021 23:03:44 +0300 Subject: [PATCH 008/372] add test with mv --- docker/test/fasttest/run.sh | 1 + src/Common/TransactionMetadata.cpp | 3 +- .../InterpreterTransactionControlQuery.cpp | 2 + src/Interpreters/MergeTreeTransaction.cpp | 4 +- src/Interpreters/MergeTreeTransaction.h | 2 +- .../MergeTreeTransactionHolder.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +- .../MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/System/StorageSystemParts.cpp | 12 +- src/Storages/System/StorageSystemParts.h | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- .../System/StorageSystemPartsColumns.cpp | 2 +- .../System/StorageSystemPartsColumns.h | 2 +- .../System/StorageSystemProjectionParts.cpp | 2 +- .../System/StorageSystemProjectionParts.h | 2 +- .../StorageSystemProjectionPartsColumns.cpp | 2 +- .../StorageSystemProjectionPartsColumns.h | 2 +- ..._mv_select_insert_isolation_long.reference | 2 + .../01171_mv_select_insert_isolation_long.sh | 131 ++++++++++++++++++ .../01174_select_insert_isolation.sh | 4 +- 24 files changed, 173 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference create mode 100755 tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index fc73a0df0ee..68eab56cb85 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -247,6 +247,7 @@ function configure cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA" "$FASTTEST_SOURCE/tests/config/install.sh" "$FASTTEST_DATA" "$FASTTEST_DATA/client-config" cp -a "$FASTTEST_SOURCE/programs/server/config.d/log_to_console.xml" "$FASTTEST_DATA/config.d" + cp -a "$FASTTEST_SOURCE/tests/config/config.d/transactions.xml" "$FASTTEST_DATA/config.d" # doesn't support SSL rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" } diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp index 7116ede8e9a..c441aa86a5b 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Common/TransactionMetadata.cpp @@ -91,7 +91,8 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid) void VersionMetadata::setMinTID(const TransactionID & tid) { /// TODO Transactions: initialize it in constructor on part creation and remove this method - assert(!mintid); + /// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times + assert(!mintid || mintid == tid); const_cast(mintid) = tid; } diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index b7621b3c81f..81a8d8cc2b3 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -29,6 +29,8 @@ BlockIO InterpreterTransactionControlQuery::execute() case ASTTransactionControl::ROLLBACK: return executeRollback(session_context); } + assert(false); + __builtin_unreachable(); } BlockIO InterpreterTransactionControlQuery::executeBegin(ContextPtr session_context) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 726a92a6d68..b6a2e87c921 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -54,7 +54,7 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const DataPartPtr & new_pa new_part->storage.getStorageID().getNameForLogs(), new_part->name); error_context += ", part_name: {}"; - for (auto covered : covered_parts) + for (const auto & covered : covered_parts) { covered->versions.lockMaxTID(tid, fmt::format(error_context, covered->name)); if (txn) @@ -79,7 +79,7 @@ bool MergeTreeTransaction::isReadOnly() const return creating_parts.empty() && removing_parts.empty(); } -void MergeTreeTransaction::beforeCommit() +void MergeTreeTransaction::beforeCommit() const { assert(csn == Tx::UnknownCSN); } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 832ee92b230..927dff6cf12 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -42,7 +42,7 @@ public: String dumpDescription() const; private: - void beforeCommit(); + void beforeCommit() const; void afterCommit(CSN assigned_csn) noexcept; void rollback() noexcept; diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index 088f6ef8f05..b6492286c3f 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -63,7 +63,7 @@ MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactio MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder &) { - txn = nullptr; + assert(txn == nullptr); return *this; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 22ad305c0ed..60965a111d4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1061,14 +1061,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part->renameToDetached(""); - for (auto it = data_parts_by_state_and_info.begin(); it != data_parts_by_state_and_info.end(); ++it) + for (const auto & part : data_parts_by_state_and_info) { /// 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. /// TODO Transactions: distinguish "prehistoric" parts from uncommitted parts in case of hard restart - (*it)->versions.setMinTID(Tx::PrehistoricTID); - (*it)->versions.mincsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); + part->versions.setMinTID(Tx::PrehistoricTID); + part->versions.mincsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); } /// Delete from the set of current parts those parts that are covered by another part (those parts that diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3fe7dd1c309..f93b118680e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -387,6 +387,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) { @@ -417,7 +418,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, nullptr, out_disable_reason)) + if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, txn.get(), out_disable_reason)) { return SelectPartsDecision::CANNOT_SELECT; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 7340a86f4a7..012258f883f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -111,6 +111,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); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f21d039d937..7a091dfbdb0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -748,7 +748,7 @@ std::shared_ptr StorageMergeTree::se { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason, optimize_skip_merged_partitions); + future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, txn, out_disable_reason, optimize_skip_merged_partitions); auto timeout_ms = getSettings()->lock_acquire_timeout_for_background_operations.totalMilliseconds(); auto timeout = std::chrono::milliseconds(timeout_ms); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 890a891c2be..bd9ff4d8670 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4542,7 +4542,7 @@ bool StorageReplicatedMergeTree::optimize( future_merged_part.uuid = UUIDHelpers::generateV4(); SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, query_context->getSettingsRef().optimize_skip_merged_partitions); + future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, nullptr, query_context->getSettingsRef().optimize_skip_merged_partitions); if (select_decision != SelectPartsDecision::SELECTED) break; @@ -4593,7 +4593,7 @@ bool StorageReplicatedMergeTree::optimize( UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason, query_context->getSettingsRef().optimize_skip_merged_partitions); + future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, nullptr, &disable_reason, query_context->getSettingsRef().optimize_skip_merged_partitions); } /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index d3d1f4c85ee..0ca9fa8f487 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -78,6 +78,7 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, {"rows_where_ttl_info.max", std::make_shared(std::make_shared())}, + {"visible", std::make_shared()}, {"mintid", TransactionID::getDataType()}, {"maxtid", TransactionID::getDataType()}, {"mincsn", std::make_shared()}, @@ -88,7 +89,7 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) } void StorageSystemParts::processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) + ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { using State = IMergeTreeDataPart::State; MergeTreeData::DataPartStateVector all_parts_state; @@ -259,6 +260,15 @@ void StorageSystemParts::processNextStorage( add_ttl_info_map(part->ttl_infos.group_by_ttl); add_ttl_info_map(part->ttl_infos.rows_where_ttl); + if (columns_mask[src_index++]) + { + auto txn = context->getCurrentTransaction(); + if (txn) + columns[res_index++]->insert(part->versions.isVisible(*txn)); + else + columns[res_index++]->insert(part_state == State::Committed); + } + auto get_tid_as_field = [](const TransactionID & tid) -> Field { return Tuple{tid.start_csn, tid.local_tid, tid.host_id}; diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h index d67e62049cd..5e74666a998 100644 --- a/src/Storages/System/StorageSystemParts.h +++ b/src/Storages/System/StorageSystemParts.h @@ -21,7 +21,7 @@ public: protected: explicit StorageSystemParts(const StorageID & table_id_); void processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; + ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 7243e5aa3ba..5419e590d48 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -268,7 +268,7 @@ Pipe StorageSystemPartsBase::read( while (StoragesInfo info = stream.next()) { - processNextStorage(res_columns, columns_mask, info, has_state_column); + processNextStorage(context, res_columns, columns_mask, info, has_state_column); } if (has_state_column) diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 45057616dad..a4388458a47 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -76,7 +76,7 @@ protected: StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_); virtual void - processNextStorage(MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0; + processNextStorage(ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0; }; } diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 33ec5c457f6..9821f747859 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -65,7 +65,7 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ } void StorageSystemPartsColumns::processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) + ContextPtr, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { /// Prepare information about columns in storage. struct ColumnInfo diff --git a/src/Storages/System/StorageSystemPartsColumns.h b/src/Storages/System/StorageSystemPartsColumns.h index ec12a608cd1..82bc3ec9168 100644 --- a/src/Storages/System/StorageSystemPartsColumns.h +++ b/src/Storages/System/StorageSystemPartsColumns.h @@ -23,7 +23,7 @@ public: protected: StorageSystemPartsColumns(const StorageID & table_id_); void processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; + ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; } diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 7ae8a91ad60..319d6a1bec1 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -90,7 +90,7 @@ StorageSystemProjectionParts::StorageSystemProjectionParts(const StorageID & tab } void StorageSystemProjectionParts::processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) + ContextPtr, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { using State = IMergeTreeDataPart::State; MergeTreeData::DataPartStateVector all_parts_state; diff --git a/src/Storages/System/StorageSystemProjectionParts.h b/src/Storages/System/StorageSystemProjectionParts.h index 11a7b034b6e..2bcf9d9a4df 100644 --- a/src/Storages/System/StorageSystemProjectionParts.h +++ b/src/Storages/System/StorageSystemProjectionParts.h @@ -21,6 +21,6 @@ public: protected: explicit StorageSystemProjectionParts(const StorageID & table_id_); void processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; + ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; } diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index bdbe9a46846..8dde0348acf 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -73,7 +73,7 @@ StorageSystemProjectionPartsColumns::StorageSystemProjectionPartsColumns(const S } void StorageSystemProjectionPartsColumns::processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) + ContextPtr, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { /// Prepare information about columns in storage. struct ColumnInfo diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.h b/src/Storages/System/StorageSystemProjectionPartsColumns.h index 16a32823db8..7ddb39b4dd7 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.h +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.h @@ -23,6 +23,6 @@ public: protected: StorageSystemProjectionPartsColumns(const StorageID & table_id_); void processNextStorage( - MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; + ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; } diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference new file mode 100644 index 00000000000..8eefde3e0b4 --- /dev/null +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference @@ -0,0 +1,2 @@ +275 0 138 136 0 +275 0 diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh new file mode 100755 index 00000000000..d4c5e3c8d1c --- /dev/null +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -0,0 +1,131 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm"; +$CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int16) AS SELECT n*m AS nm FROM src"; + +$CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt8, nm Int16) ENGINE=MergeTree ORDER BY (x, nm)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO src VALUES (0, 0)" + +# some transactions will fail due to constraint +function thread_insert_commit() +{ + for i in {1..100}; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES ($i, $1); + SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $1))) FORMAT Null; + INSERT INTO src VALUES (-$i, $1); + COMMIT;" 2>&1| grep -Fv "is violated at row" | grep -Fv "Transaction is not in RUNNING state" | grep -F "Received from " ||: + done +} + +function thread_insert_rollback() +{ + for _ in {1..100}; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES (42, $1); + SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$1) != 1) FORMAT Null; + ROLLBACK;" + done +} + +# make merges more aggressive +function thread_optimize() +{ + trap "exit 0" INT + while true; do + optimize_query="OPTIMIZE TABLE src" + if (( RANDOM % 2 )); then + optimize_query="OPTIMIZE TABLE dst" + fi + if (( RANDOM % 2 )); then + optimize_query="$optimize_query FINAL" + fi + action="COMMIT" + if (( RANDOM % 2 )); then + action="ROLLBACK" + fi + + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + $optimize_query; + $action; + " + sleep 0.$RANDOM; + done +} + +function thread_select() +{ + trap "exit 0" INT + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + SELECT throwIf((SELECT (sum(n), count() % 2) FROM src) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM mv) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; + SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; + COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT 'src', arraySort(groupArray(n*m)) FROM src UNION ALL SELECT 'mv', arraySort(groupArray(nm)) FROM mv" + done +} + +function thread_select_insert() +{ + trap "exit 0" INT + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null; + INSERT INTO tmp SELECT 1, n*m FROM src; + INSERT INTO tmp SELECT 2, nm FROM mv; + INSERT INTO tmp SELECT 3, nm FROM dst; + INSERT INTO tmp SELECT 4, (*,).1 FROM (SELECT n*m FROM src UNION ALL SELECT nm FROM mv UNION ALL SELECT nm FROM dst); + SELECT throwIf((SELECT countDistinct(x) FROM tmp) != 4) FORMAT Null; + + -- now check that all results are the same + SELECT throwIf(1 != (SELECT countDistinct(arr) FROM (SELECT x, arraySort(groupArray(nm)) AS arr FROM tmp WHERE x!=4 GROUP BY x))) FORMAT Null; + SELECT throwIf((SELECT count(), sum(nm) FROM tmp WHERE x=4) != (SELECT count(), sum(nm) FROM tmp WHERE x!=4)) FORMAT Null; + ROLLBACK;" || $CLICKHOUSE_CLIENT -q "SELECT x, arraySort(groupArray(nm)) AS arr FROM tmp GROUP BY x" + done +} + +thread_insert_commit 1 & PID_1=$! +thread_insert_commit 2 & PID_2=$! +thread_insert_rollback 3 & PID_3=$! + +thread_optimize & PID_4=$! +thread_select & PID_5=$! +thread_select_insert & PID_6=$! +sleep 0.$RANDOM; +thread_select & PID_7=$! +thread_select_insert & PID_8=$! + +wait $PID_1 && wait $PID_2 && wait $PID_3 +kill -INT $PID_4 +kill -INT $PID_5 +kill -INT $PID_6 +kill -INT $PID_7 +kill -INT $PID_8 +wait + +$CLICKHOUSE_CLIENT --multiquery --query " +BEGIN TRANSACTION; +SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM src; +SELECT count(), sum(nm) FROM mv"; + +$CLICKHOUSE_CLIENT --query "DROP TABLE src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE dst"; +$CLICKHOUSE_CLIENT --query "DROP TABLE mv"; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index e692b36e1af..1f04e920628 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -41,8 +41,8 @@ function thread_select() SELECT throwIf((SELECT sum(n) FROM mt) != 0) FORMAT Null; SELECT throwIf((SELECT count() FROM mt) % 2 != 0) FORMAT Null; SELECT arraySort(groupArray(n)), arraySort(groupArray(m)), arraySort(groupArray(_part)) FROM mt; - COMMIT;" | tee -a ./wtf.log | uniq | wc -l | grep -v "^1$" && $CLICKHOUSE_CLIENT -q "SELECT * FROM system.parts - WHERE database='$CLICKHOUSE_DATABASE' AND table='mt'" ||:; + COMMIT;" | uniq | wc -l | grep -v "^1$" && $CLICKHOUSE_CLIENT -q "SELECT * FROM system.parts + WHERE database='$CLICKHOUSE_DATABASE' AND table='mt'" ||:; done } From 881889ef22a7b25497f03965e3be16210d872c0e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Jun 2021 12:26:47 +0300 Subject: [PATCH 009/372] parts cleanup, fixes --- docker/test/fasttest/run.sh | 1 - src/Common/TransactionMetadata.cpp | 23 +++++++++--- src/Common/TransactionMetadata.h | 3 ++ src/Interpreters/MergeTreeTransaction.cpp | 31 +++++++++++----- src/Interpreters/MergeTreeTransaction.h | 18 +++++++--- src/Interpreters/TransactionLog.cpp | 20 ++++++++--- src/Interpreters/TransactionLog.h | 3 ++ src/Storages/MergeTree/MergeTreeData.cpp | 35 ++++++++++++++++--- src/Storages/MergeTree/MergeTreeData.h | 6 +++- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 14 ++++---- ..._mv_select_insert_isolation_long.reference | 2 ++ .../01171_mv_select_insert_isolation_long.sh | 7 ++-- .../01174_select_insert_isolation.reference | 1 + .../01174_select_insert_isolation.sh | 2 ++ 16 files changed, 130 insertions(+), 42 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 68eab56cb85..fc73a0df0ee 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -247,7 +247,6 @@ function configure cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA" "$FASTTEST_SOURCE/tests/config/install.sh" "$FASTTEST_DATA" "$FASTTEST_DATA/client-config" cp -a "$FASTTEST_SOURCE/programs/server/config.d/log_to_console.xml" "$FASTTEST_DATA/config.d" - cp -a "$FASTTEST_SOURCE/tests/config/config.d/transactions.xml" "$FASTTEST_DATA/config.d" # doesn't support SSL rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" } diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp index c441aa86a5b..bebebe5d550 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Common/TransactionMetadata.cpp @@ -32,7 +32,7 @@ TIDHash TransactionID::getHash() const return hash.get64(); } -/// It can be used fro introspection purposes only +/// It can be used for introspection purposes only TransactionID VersionMetadata::getMaxTID() const { TIDHash max_lock = maxtid_lock.load(); @@ -40,6 +40,8 @@ TransactionID VersionMetadata::getMaxTID() const { if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) return txn->tid; + if (max_lock == Tx::PrehistoricTID.getHash()) + return Tx::PrehistoricTID; } if (maxcsn.load(std::memory_order_relaxed)) @@ -53,6 +55,7 @@ TransactionID VersionMetadata::getMaxTID() const void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context) { + assert(tid); TIDHash max_lock_value = tid.getHash(); TIDHash expected_max_lock_value = 0; bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); @@ -69,6 +72,7 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error void VersionMetadata::unlockMaxTID(const TransactionID & tid) { + assert(tid); TIDHash max_lock_value = tid.getHash(); TIDHash locked_by = maxtid_lock.load(); @@ -88,6 +92,11 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid) throw_cannot_unlock(); } +bool VersionMetadata::isMaxTIDLocked() const +{ + return maxtid_lock.load() != 0; +} + void VersionMetadata::setMinTID(const TransactionID & tid) { /// TODO Transactions: initialize it in constructor on part creation and remove this method @@ -98,7 +107,11 @@ void VersionMetadata::setMinTID(const TransactionID & tid) bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) { - Snapshot snapshot_version = txn.getSnapshot(); + return isVisible(txn.getSnapshot(), txn.tid); +} + +bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) +{ assert(mintid); CSN min = mincsn.load(std::memory_order_relaxed); TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); @@ -120,7 +133,7 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) return false; if (max && max <= snapshot_version) return false; - if (max_lock && max_lock == txn.tid.getHash()) + if (current_tid && max_lock && max_lock == current_tid.getHash()) return false; /// Otherwise, part is definitely visible if: @@ -131,7 +144,7 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) return true; if (min && min <= snapshot_version && max && snapshot_version < max) return true; - if (mintid == txn.tid) + if (current_tid && mintid == current_tid) return true; /// End of fast path. @@ -140,7 +153,7 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) /// 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 ot not. assert(!had_mincsn || (had_maxtid && !had_maxcsn)); - assert(mintid != txn.tid && max_lock != txn.tid.getHash()); + assert(!current_tid || (mintid != current_tid && max_lock != current_tid.getHash())); /// Before doing CSN lookup, let's check some extra conditions. /// If snapshot_version <= some_tid.start_csn, then changes of transaction with some_tid diff --git a/src/Common/TransactionMetadata.h b/src/Common/TransactionMetadata.h index 6abf7c2b4b3..b716b0be861 100644 --- a/src/Common/TransactionMetadata.h +++ b/src/Common/TransactionMetadata.h @@ -71,6 +71,7 @@ struct VersionMetadata std::atomic maxcsn = Tx::UnknownCSN; bool isVisible(const MergeTreeTransaction & txn); + bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); TransactionID getMinTID() const { return mintid; } TransactionID getMaxTID() const; @@ -78,6 +79,8 @@ struct VersionMetadata void lockMaxTID(const TransactionID & tid, const String & error_context = {}); void unlockMaxTID(const TransactionID & tid); + bool isMaxTIDLocked() const; + /// It can be called only from MergeTreeTransaction or on server startup void setMinTID(const TransactionID & tid); }; diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index b6a2e87c921..588c64fdaa4 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -22,16 +22,16 @@ MergeTreeTransaction::State MergeTreeTransaction::getState() const return COMMITTED; } -void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part, MergeTreeTransaction * txn) +void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; new_part->versions.setMinTID(tid); if (txn) - txn->addNewPart(new_part); + txn->addNewPart(storage, new_part); } -void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) +void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; String error_context = fmt::format("Table: {}, part name: {}", @@ -39,16 +39,16 @@ void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove, Mer part_to_remove->name); part_to_remove->versions.lockMaxTID(tid, error_context); if (txn) - txn->removeOldPart(part_to_remove); + txn->removeOldPart(storage, part_to_remove); } -void MergeTreeTransaction::addNewPartAndRemoveCovered(const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) +void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; new_part->versions.setMinTID(tid); if (txn) - txn->addNewPart(new_part); + txn->addNewPart(storage, new_part); String error_context = fmt::format("Table: {}, covering part name: {}", new_part->storage.getStorageID().getNameForLogs(), @@ -58,19 +58,21 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const DataPartPtr & new_pa { covered->versions.lockMaxTID(tid, fmt::format(error_context, covered->name)); if (txn) - txn->removeOldPart(covered); + txn->removeOldPart(storage, covered); } } -void MergeTreeTransaction::addNewPart(const DataPartPtr & new_part) +void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part) { assert(csn == Tx::UnknownCSN); + storages.insert(storage); creating_parts.push_back(new_part); } -void MergeTreeTransaction::removeOldPart(const DataPartPtr & part_to_remove) +void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) { assert(csn == Tx::UnknownCSN); + storages.insert(storage); removing_parts.push_back(part_to_remove); } @@ -100,8 +102,19 @@ void MergeTreeTransaction::rollback() noexcept csn = Tx::RolledBackCSN; for (const auto & part : creating_parts) part->versions.mincsn.store(Tx::RolledBackCSN); + for (const auto & part : removing_parts) part->versions.unlockMaxTID(tid); + + /// FIXME const_cast + for (const auto & part : creating_parts) + const_cast(part->storage).removePartsFromWorkingSet({part}, true); + + for (const auto & part : removing_parts) + if (part->versions.getMinTID() != tid) + const_cast(part->storage).restoreAndActivatePart(part); + + /// FIXME seems like session holds shared_ptr to Transaction and transaction holds shared_ptr to parts preventing cleanup } void MergeTreeTransaction::onException() diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 927dff6cf12..1ead2fdce87 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -1,6 +1,10 @@ #pragma once #include #include +#include + +#include +#include namespace DB { @@ -28,12 +32,12 @@ public: MergeTreeTransaction() = delete; MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id); - void addNewPart(const DataPartPtr & new_part); - void removeOldPart(const DataPartPtr & part_to_remove); + void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part); + void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove); - static void addNewPart(const DataPartPtr & new_part, MergeTreeTransaction * txn); - static void removeOldPart(const DataPartPtr & part_to_remove, MergeTreeTransaction * txn); - static void addNewPartAndRemoveCovered(const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn); + 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; @@ -48,10 +52,14 @@ private: Snapshot snapshot; + std::unordered_set storages; DataPartsVector creating_parts; DataPartsVector removing_parts; CSN csn; + + /// FIXME it's ugly + std::list::iterator snapshot_in_use_it; }; using MergeTreeTransactionPtr = std::shared_ptr; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 7e7357c68cc..25333c66a5f 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -32,14 +32,16 @@ Snapshot TransactionLog::getLatestSnapshot() const MergeTreeTransactionPtr TransactionLog::beginTransaction() { - Snapshot snapshot = latest_snapshot.load(); - LocalTID ltid = 1 + local_tid_counter.fetch_add(1); - auto txn = std::make_shared(snapshot, ltid, UUIDHelpers::Nil); + MergeTreeTransactionPtr txn; { std::lock_guard lock{running_list_mutex}; - bool inserted = running_list.try_emplace(txn->tid.getHash(), txn).second; /// Commit point + Snapshot snapshot = latest_snapshot.load(); + LocalTID ltid = 1 + local_tid_counter.fetch_add(1); + txn = std::make_shared(snapshot, ltid, UUIDHelpers::Nil); + 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_TRACE(log, "Beginning transaction {}", txn->tid); return txn; @@ -76,6 +78,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) 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; } @@ -89,6 +92,7 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no bool removed = running_list.erase(txn->tid.getHash()); if (!removed) abort(); + snapshots_in_use.erase(txn->snapshot_in_use_it); } } @@ -120,4 +124,12 @@ CSN TransactionLog::getCSN(const TIDHash & tid) const return it->second; } +Snapshot TransactionLog::getOldestSnapshot() const +{ + std::lock_guard lock{running_list_mutex}; + if (snapshots_in_use.empty()) + return getLatestSnapshot(); + return snapshots_in_use.front(); +} + } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index d4894deac5c..c2edeafb523 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -29,6 +29,8 @@ public: MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); + Snapshot getOldestSnapshot() const; + private: Poco::Logger * log; @@ -42,6 +44,7 @@ private: mutable std::mutex running_list_mutex; std::unordered_map running_list; + std::list snapshots_in_use; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f83079abd97..a11fc627947 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -1208,6 +1209,10 @@ 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->versions.isVisible(TransactionLog::instance().getOldestSnapshot())) + 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). @@ -2157,7 +2162,6 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, MergeTreeTra DataPartsVector covered_parts; { auto lock = lockParts(); - if (!renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts)) if (!renameTempPartAndReplace(part, txn, increment, out_transaction, lock, &covered_parts, deduplication_log)) return false; } @@ -2225,10 +2229,6 @@ bool MergeTreeData::renameTempPartAndReplace( return false; } - /// FIXME Transactions: it's not the best place for checking and setting maxtid, - /// because it's too optimistic. We should lock maxtid of covered parts at the beginning of operation. - MergeTreeTransaction::addNewPartAndRemoveCovered(part, covered_parts, txn); - /// Deduplication log used only from non-replicated MergeTree. Replicated /// tables have their own mechanism. We try to deduplicate at such deep /// level, because only here we know real part name which is required for @@ -2255,6 +2255,9 @@ bool MergeTreeData::renameTempPartAndReplace( part->setState(DataPartState::PreCommitted); part->renameTo(part_name, true); + /// FIXME Transactions: it's not the best place for checking and setting maxtid, + /// because it's too optimistic. We should lock maxtid of covered parts at the beginning of operation. + MergeTreeTransaction::addNewPartAndRemoveCovered(shared_from_this(), part, covered_parts, txn); auto part_it = data_parts_indexes.insert(part).first; if (out_transaction) @@ -2349,6 +2352,7 @@ 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); + assert(part->getState() == IMergeTreeDataPart::State::PreCommitted); modifyPartState(part, IMergeTreeDataPart::State::Temporary); /// Erase immediately data_parts_indexes.erase(it_part); @@ -2417,6 +2421,15 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c return parts_to_remove; } +void MergeTreeData::restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock) +{ + auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); + assert(part->getState() != DataPartState::Committed); + addPartContributionToColumnSizes(part); + addPartContributionToDataVolume(part); + modifyPartState(part, DataPartState::Committed); +} + void MergeTreeData::forgetPartAndMoveToDetached(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered) { @@ -3762,6 +3775,18 @@ void MergeTreeData::Transaction::rollback() buf << "."; LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str()); + 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->versions.unlockMaxTID(Tx::PrehistoricTID); + } + } + data.removePartsFromWorkingSet( DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()), /* clear_without_timeout = */ true); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 19efa763806..c63c2b8cefd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -239,7 +239,7 @@ public: class Transaction : private boost::noncopyable { public: - Transaction(MergeTreeData & data_) : data(data_) {} + Transaction(MergeTreeData & data_, MergeTreeTransaction * txn_) : data(data_), txn(txn_) {} DataPartsVector commit(MergeTreeData::DataPartsLock * acquired_parts_lock = nullptr); @@ -268,6 +268,7 @@ public: friend class MergeTreeData; MergeTreeData & data; + MergeTreeTransaction * txn; DataParts precommitted_parts; void clear() { precommitted_parts.clear(); } @@ -503,6 +504,9 @@ public: DataPartsVector removePartsInRangeFromWorkingSet(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/_ 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. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 693c2f00610..cb337559946 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -382,7 +382,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// Information about the part. storage.getCommitPartOps(ops, part, block_id_path); - MergeTreeData::Transaction transaction(storage); /// If you can not add a part to ZK, we'll remove it back from the working set. + MergeTreeData::Transaction transaction(storage, nullptr); /// If you can not add a part to ZK, we'll remove it back from the working set. bool renamed = false; try { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5cb8797217c..56a29f97576 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1418,7 +1418,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con { /// Here we use the transaction just like RAII since rare errors in renameTempPartAndReplace() are possible /// and we should be able to rollback already added (Precomitted) parts - Transaction transaction(*this); + Transaction transaction(*this, local_context->getCurrentTransaction().get()); auto data_parts_lock = lockParts(); @@ -1491,7 +1491,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const try { { - Transaction transaction(*dest_table_storage); + Transaction transaction(*dest_table_storage, local_context->getCurrentTransaction().get()); auto src_data_parts_lock = lockParts(); auto dest_data_parts_lock = dest_table_storage->lockParts(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5d8d9ce914d..c243cd24c49 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1499,7 +1499,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { LOG_TRACE(log, "Found valid part to attach from local data, preparing the transaction"); - Transaction transaction(*this); + Transaction transaction(*this, nullptr); renameTempPartAndReplace(part, nullptr, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); @@ -1726,7 +1726,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) /// Add merge to list MergeList::EntryPtr merge_entry = getContext()->getMergeList().insert(table_id.database_name, table_id.table_name, future_merged_part); - Transaction transaction(*this); + Transaction transaction(*this, nullptr); MutableDataPartPtr part; Stopwatch stopwatch; @@ -1859,7 +1859,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); MutableDataPartPtr new_part; - Transaction transaction(*this); + Transaction transaction(*this, nullptr); FutureMergedMutatedPart future_mutated_part; future_mutated_part.name = entry.new_part_name; @@ -2530,7 +2530,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { /// Commit parts auto zookeeper = getZooKeeper(); - Transaction transaction(*this); + Transaction transaction(*this, nullptr); Coordination::Requests ops; for (PartDescriptionPtr & part_desc : final_parts) @@ -4044,7 +4044,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora if (!to_detached) { - Transaction transaction(*this); + Transaction transaction(*this, nullptr); renameTempPartAndReplace(part, nullptr, nullptr, &transaction); replaced_parts = checkPartChecksumsAndCommit(transaction, part); @@ -6465,7 +6465,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - Transaction transaction(*this); + Transaction transaction(*this, nullptr); { auto data_parts_lock = lockParts(); @@ -6655,7 +6655,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta entry.toString(), zkutil::CreateMode::PersistentSequential)); { - Transaction transaction(*dest_table_storage); + Transaction transaction(*dest_table_storage, nullptr); auto src_data_parts_lock = lockParts(); auto dest_data_parts_lock = dest_table_storage->lockParts(); diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference index 8eefde3e0b4..d8bb9e310e6 100644 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.reference @@ -1,2 +1,4 @@ 275 0 138 136 0 275 0 +275 0 138 136 0 +275 0 diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index d4c5e3c8d1c..1f0148aa093 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -54,7 +54,7 @@ function thread_optimize() optimize_query="$optimize_query FINAL" fi action="COMMIT" - if (( RANDOM % 2 )); then + if (( RANDOM % 4 )); then action="ROLLBACK" fi @@ -62,7 +62,7 @@ function thread_optimize() BEGIN TRANSACTION; $optimize_query; $action; - " + " 2>&1| grep -Fv "already exists, but it will be deleted soon" | grep -F "Received from " ||: sleep 0.$RANDOM; done } @@ -126,6 +126,9 @@ BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM src; SELECT count(), sum(nm) FROM mv"; +$CLICKHOUSE_CLIENT --query "SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM src" +$CLICKHOUSE_CLIENT --query "SELECT count(), sum(nm) FROM mv" + $CLICKHOUSE_CLIENT --query "DROP TABLE src"; $CLICKHOUSE_CLIENT --query "DROP TABLE dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE mv"; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.reference b/tests/queries/0_stateless/01174_select_insert_isolation.reference index 51d57dfa0d6..ba5f4de36ac 100644 --- a/tests/queries/0_stateless/01174_select_insert_isolation.reference +++ b/tests/queries/0_stateless/01174_select_insert_isolation.reference @@ -1 +1,2 @@ 200 0 100 100 0 +200 0 100 100 0 diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 1f04e920628..1be10e29171 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -58,4 +58,6 @@ $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;"; +$CLICKHOUSE_CLIENT --query "SELECT count(), sum(n), sum(m=1), sum(m=2), sum(m=3) FROM mt;" + $CLICKHOUSE_CLIENT --query "DROP TABLE mt"; From ceef0665e2084ad278ba2386b926bda89bc1986c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Jun 2021 13:01:49 +0300 Subject: [PATCH 010/372] fix --- src/Interpreters/Context.cpp | 2 ++ src/Interpreters/MergeTreeTransactionHolder.h | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7dd94bceabb..e5dadb02c66 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2657,6 +2657,8 @@ void Context::setCurrentTransaction(MergeTreeTransactionPtr txn) if (enable_mvcc_test_helper != 42) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); merge_tree_transaction = std::move(txn); + if (!merge_tree_transaction) + merge_tree_transaction_holder = {}; } MergeTreeTransactionPtr Context::getCurrentTransaction() const diff --git a/src/Interpreters/MergeTreeTransactionHolder.h b/src/Interpreters/MergeTreeTransactionHolder.h index ec9cf1e1636..11ab0627d00 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.h +++ b/src/Interpreters/MergeTreeTransactionHolder.h @@ -5,6 +5,7 @@ namespace DB { 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; class MergeTreeTransactionHolder From 9a88b9dacc5cefc89372d3ec4a74fcf9685d18e7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Jun 2021 21:17:18 +0300 Subject: [PATCH 011/372] set maxtid on drop part --- src/Interpreters/MergeTreeTransaction.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++++++------- src/Storages/MergeTree/MergeTreeData.h | 6 ++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 ------ src/Storages/StorageMergeTree.cpp | 23 +++++++++---------- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 +++++----- .../01172_transaction_counters.reference | 18 +++++++-------- .../01172_transaction_counters.sql | 8 +++---- 10 files changed, 49 insertions(+), 52 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 588c64fdaa4..b9fb94de8e3 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -108,7 +108,7 @@ void MergeTreeTransaction::rollback() noexcept /// FIXME const_cast for (const auto & part : creating_parts) - const_cast(part->storage).removePartsFromWorkingSet({part}, true); + const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); for (const auto & part : removing_parts) if (part->versions.getMinTID() != tid) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 25333c66a5f..64b32298aa4 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -43,7 +43,7 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); txn->snapshot_in_use_it = snapshots_in_use.insert(snapshots_in_use.end(), snapshot); } - LOG_TRACE(log, "Beginning transaction {}", txn->tid); + LOG_TRACE(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash()); return txn; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a11fc627947..974ae2d85ba 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2255,10 +2255,10 @@ bool MergeTreeData::renameTempPartAndReplace( part->setState(DataPartState::PreCommitted); 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 maxtid, /// because it's too optimistic. We should lock maxtid of covered parts at the beginning of operation. MergeTreeTransaction::addNewPartAndRemoveCovered(shared_from_this(), part, covered_parts, txn); - auto part_it = data_parts_indexes.insert(part).first; if (out_transaction) { @@ -2319,12 +2319,15 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( 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*/) { auto remove_time = clear_without_timeout ? 0 : time(nullptr); for (const DataPartPtr & part : remove) { + if (part->versions.mincsn != Tx::RolledBackCSN) + MergeTreeTransaction::removeOldPart(shared_from_this(), part, txn); + if (part->getState() == IMergeTreeDataPart::State::Committed) { removePartContributionToColumnSizes(part); @@ -2359,7 +2362,8 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con } } -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(); @@ -2371,11 +2375,12 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo part->assertState({DataPartState::PreCommitted, DataPartState::Committed, 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; @@ -2416,7 +2421,7 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c 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; } @@ -3787,7 +3792,7 @@ void MergeTreeData::Transaction::rollback() } } - data.removePartsFromWorkingSet( + data.removePartsFromWorkingSet(txn, DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()), /* clear_without_timeout = */ true); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c63c2b8cefd..fd257a1e343 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -495,13 +495,13 @@ public: /// Parts in add must already be in data_parts with PreCommitted, Committed, 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 diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c704bd7a857..1209598c2da 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1477,13 +1477,6 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart { /// Rename new part, add to the set and remove original parts. auto replaced_parts = data.renameTempPartAndReplace(new_data_part, txn.get(), nullptr, out_transaction); - //String parts_str; - //for (const auto & p : parts) - // parts_str += "\t" + p->name; - //String parts_str2; - //for (const auto & p : replaced_parts) - // parts_str2 += "\t" + p->name; - //LOG_ERROR(log, "WTF {}: source {}, replaced {}", new_data_part->name, parts_str, parts_str2); /// Let's check that all original parts have been deleted and only them. if (replaced_parts.size() != parts.size()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 56a29f97576..4fb510a9b97 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -243,7 +243,7 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { { /// Asks to complete merges and does not allow them to start. @@ -251,7 +251,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont auto merge_blocker = stopMergesAndWait(); auto parts_to_remove = getDataPartsVector(); - removePartsFromWorkingSet(parts_to_remove, true); + removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), parts_to_remove, true); LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); } @@ -1058,7 +1058,6 @@ std::optional StorageMergeTree::getDataProcessingJob() //-V657 return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock, txn); else if (mutate_entry) return mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock); - return true; //__builtin_unreachable(); }, PoolType::MERGE_MUTATE}; @@ -1242,7 +1241,7 @@ ActionLock StorageMergeTree::stopMergesAndWait() } -MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, bool force) +MergeTreeDataPartPtr StorageMergeTree::outdatePart(MergeTreeTransaction * txn, const String & part_name, bool force) { if (force) @@ -1252,7 +1251,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); if (!part) throw Exception("Part " + part_name + " not found, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); - removePartsFromWorkingSet({part}, true); + removePartsFromWorkingSet(txn, {part}, true); return part; } else @@ -1271,22 +1270,22 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo if (currently_merging_mutating_parts.count(part)) return nullptr; - removePartsFromWorkingSet({part}, true); + removePartsFromWorkingSet(txn, {part}, true); return part; } } void StorageMergeTree::dropPartNoWaitNoThrow(const String & part_name) { - if (auto part = outdatePart(part_name, /*force=*/ false)) + if (auto part = outdatePart(nullptr, part_name, /*force=*/ false)) dropPartsImpl({part}, /*detach=*/ false); /// Else nothing to do, part was removed in some different way } -void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPtr /*query_context*/) +void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPtr query_context) { - if (auto part = outdatePart(part_name, /*force=*/ true)) + if (auto part = outdatePart(query_context->getCurrentTransaction().get(), part_name, /*force=*/ true)) dropPartsImpl({part}, detach); } @@ -1302,7 +1301,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); /// TODO should we throw an exception if parts_to_remove is empty? - removePartsFromWorkingSet(parts_to_remove, true); + removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), parts_to_remove, true); } dropPartsImpl(std::move(parts_to_remove), detach); @@ -1430,7 +1429,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con /// If it is REPLACE (not ATTACH), remove all parts which max_block_number less then min_block_number of the first new block if (replace) - removePartsInRangeFromWorkingSet(drop_range, true, data_parts_lock); + removePartsInRangeFromWorkingSet(local_context->getCurrentTransaction().get(), drop_range, true, data_parts_lock); } PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); @@ -1502,7 +1501,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const for (MutableDataPartPtr & part : dst_parts) dest_table_storage->renameTempPartAndReplace(part, local_context->getCurrentTransaction().get(), &dest_table_storage->increment, &transaction, lock); - removePartsFromWorkingSet(src_parts, true, lock); + removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), src_parts, true, lock); transaction.commit(&lock); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d8585b967f0..24c0b46e5d6 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -154,7 +154,7 @@ private: /// Make part state outdated and queue it to remove without timeout /// If force, then stop merges and block them until part state became outdated. Throw exception if part doesn't exists /// If not force, then take merges selector and check that part is not participating in background operations. - MergeTreeDataPartPtr outdatePart(const String & part_name, bool force); + MergeTreeDataPartPtr outdatePart(MergeTreeTransaction * txn, const String & part_name, bool force); ActionLock stopMergesAndWait(); /// Allocate block number for new mutation, write mutation to disk diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c243cd24c49..6dbf7d270ea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2195,7 +2195,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) DataPartsVector parts_to_remove; { auto data_parts_lock = lockParts(); - parts_to_remove = removePartsInRangeFromWorkingSet(drop_range_info, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range_info, true, data_parts_lock); } if (entry.detach) @@ -2311,7 +2311,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (parts_to_add.empty() && replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -2554,7 +2554,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) transaction.commit(&data_parts_lock); if (replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -2777,7 +2777,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo } } - removePartsFromWorkingSet(parts_to_remove_from_working_set, true); + removePartsFromWorkingSet(nullptr, parts_to_remove_from_working_set, true); for (const String & name : active_parts) { @@ -6486,7 +6486,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( transaction.commit(&data_parts_lock); if (replace) - parts_to_remove = removePartsInRangeFromWorkingSet(drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); } PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); @@ -6672,7 +6672,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta else zkutil::KeeperMultiException::check(code, ops, op_results); - parts_to_remove = removePartsInRangeFromWorkingSet(drop_range, true, lock); + parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, lock); transaction.commit(&lock); } diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 375f3540af9..fe055805d93 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -1,10 +1,10 @@ (0,0,'00000000-0000-0000-0000-000000000000') -all_1_1_0 0 -all_2_2_0 1 -all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 -all_2_2_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 -all_1_1_0 0 -all_3_3_0 1 -all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 -all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 -all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +1 all_1_1_0 0 +1 all_2_2_0 1 +2 all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 +2 all_2_2_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +3 all_1_1_0 0 +3 all_3_3_0 1 +4 all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 +4 all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 +4 all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 2637041bcf7..f36b2c683ba 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -7,14 +7,14 @@ select transactionID(); begin transaction; insert into txn_counters(n) values (2); -select system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 1, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 2, name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; rollback; begin transaction; insert into txn_counters(n) values (3); -select system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 3, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 4, name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; commit; drop table txn_counters; From e5d8a97546765ad4cb194f55bc0b61fb3a4f0adc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Jun 2021 15:14:38 +0300 Subject: [PATCH 012/372] fix --- src/Common/TransactionMetadata.cpp | 35 +++++++++++++++++++ src/Common/TransactionMetadata.h | 4 ++- src/Interpreters/Context.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 13 +++++-- ...034_move_partition_from_table_zookeeper.sh | 4 +-- .../01171_mv_select_insert_isolation_long.sh | 6 ++-- .../01172_transaction_counters.sql | 3 ++ .../01174_select_insert_isolation.sh | 2 +- 8 files changed, 58 insertions(+), 10 deletions(-) diff --git a/src/Common/TransactionMetadata.cpp b/src/Common/TransactionMetadata.cpp index bebebe5d550..c3513380a49 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Common/TransactionMetadata.cpp @@ -186,4 +186,39 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return min <= snapshot_version && (!max || snapshot_version < max); } +bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) +{ + CSN min = mincsn.load(std::memory_order_relaxed); + if (min == Tx::RolledBackCSN) + return true; + + if (!min) + { + min = TransactionLog::instance().getCSN(mintid); + if (min) + mincsn.store(min, std::memory_order_relaxed); + else + return false; + } + + if (oldest_snapshot_version < min) + return false; + + TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); + if (!max_lock) + return false; + + CSN max = maxcsn.load(std::memory_order_relaxed); + if (!max) + { + max = TransactionLog::instance().getCSN(max_lock); + if (max) + maxcsn.store(max, std::memory_order_relaxed); + else + return false; + } + + return max <= oldest_snapshot_version; +} + } diff --git a/src/Common/TransactionMetadata.h b/src/Common/TransactionMetadata.h index b716b0be861..8aa1488f1a6 100644 --- a/src/Common/TransactionMetadata.h +++ b/src/Common/TransactionMetadata.h @@ -53,7 +53,7 @@ const CSN PrehistoricCSN = 1; const LocalTID PrehistoricLocalTID = 1; const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; -const TransactionID PrehistoricTID = {0, PrehistoricLocalTID, UUIDHelpers::Nil}; +const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil}; /// So far, that changes will never become visible const CSN RolledBackCSN = std::numeric_limits::max(); @@ -83,6 +83,8 @@ struct VersionMetadata /// It can be called only from MergeTreeTransaction or on server startup void setMinTID(const TransactionID & tid); + + bool canBeRemoved(Snapshot oldest_snapshot_version); }; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b72439b9cd3..0c9f21556b7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -844,6 +844,7 @@ struct NamedSession NamedSession(NamedSessionKey key_, ContextPtr context_, std::chrono::steady_clock::duration timeout_, NamedSessions & parent_) : key(key_), context(Context::createCopy(context_)), timeout(timeout_), parent(parent_) { + context->setSessionContext(context); } void release(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee3a81ee88e..6f9bcead9fe 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1210,7 +1210,7 @@ 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->versions.isVisible(TransactionLog::instance().getOldestSnapshot())) + if (!part->versions.canBeRemoved(TransactionLog::instance().getOldestSnapshot())) continue; auto part_remove_time = part->remove_time.load(std::memory_order_relaxed); @@ -2417,8 +2417,15 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( 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 + TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + if (!part->versions.isVisible(tid.start_csn, tid)) + continue; + + parts_to_remove.emplace_back(part); } removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock); diff --git a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh index ae3dd7851c8..af4969d77c8 100755 --- a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh @@ -75,8 +75,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE dst;" $CLICKHOUSE_CLIENT --query="SELECT 'MOVE incompatible schema different order by';" -$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CURR_DATABASE/src3', '1') PARTITION BY p ORDER BY (p, k, d);" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CURR_DATABASE/dst3', '1') PARTITION BY p ORDER BY (d, k, p);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src3', '1') PARTITION BY p ORDER BY (p, k, d);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst3', '1') PARTITION BY p ORDER BY (d, k, p);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 1f0148aa093..f506962a36a 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -9,11 +9,11 @@ set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm SETTINGS old_parts_lifetime=0"; $CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int16) AS SELECT n*m AS nm FROM src"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt8, nm Int16) ENGINE=MergeTree ORDER BY (x, nm)" +$CLICKHOUSE_CLIENT --query "CREATE TABLE tmp (x UInt8, nm Int16) ENGINE=MergeTree ORDER BY (x, nm) SETTINGS old_parts_lifetime=0" $CLICKHOUSE_CLIENT --query "INSERT INTO src VALUES (0, 0)" diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index f36b2c683ba..1d9311e4e86 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -5,6 +5,9 @@ create table txn_counters (n Int64, mintid DEFAULT transactionID()) engine=Merge insert into txn_counters(n) values (1); select transactionID(); +-- stop background cleanup +system stop merges txn_counters; + begin transaction; insert into txn_counters(n) values (2); select 1, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 1be10e29171..2bc7185e3fa 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mt"; -$CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int8, m Int8) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; function thread_insert_commit() { From 4114b8293709fd69565ff691391b0bbfa25acfb3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Sep 2021 23:15:49 +0300 Subject: [PATCH 013/372] hotfixes for merge task --- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.h | 9 +++++++++ src/Storages/StorageMergeTree.cpp | 9 ++++++++- 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 0830463a2fe..887725d1aba 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -108,7 +108,7 @@ void MergePlainMergeTreeTask::prepare() void MergePlainMergeTreeTask::finish() { new_part = merge_task->getFuture().get(); - storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, nullptr, nullptr); //FIXME + storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, txn, nullptr); //FIXME write_part_log({}); } diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index f199557684c..0e6b4a0405a 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -36,6 +36,12 @@ public: StorageID getStorageID() override; + 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: IExecutableTask::TaskResultCallback task_result_callback; MergeTaskPtr merge_task{nullptr}; + + MergeTreeTransactionHolder txn_holder; + MergeTreeTransactionPtr txn; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c604e4ecb94..abba8d37695 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -857,6 +857,8 @@ bool StorageMergeTree::merge( auto task = std::make_shared( *this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, [](bool){}); + task->setCurrentTransaction(MergeTreeTransactionHolder{}, MergeTreeTransactionPtr{txn}); + executeHere(task); return true; @@ -1033,13 +1035,17 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + /// FIXME Transactions: do not begin transaction if we don't need it + auto txn = TransactionLog::instance().beginTransaction(); + MergeTreeTransactionHolder autocommit{txn, true}; + bool has_mutations = false; { std::unique_lock lock(currently_processing_in_background_mutex); if (merger_mutator.merges_blocker.isCancelled()) return false; - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, lock, nullptr); + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock, lock, txn); if (!merge_entry) { mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); @@ -1050,6 +1056,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (merge_entry) { auto task = std::make_shared(*this, metadata_snapshot, false, Names{}, merge_entry, share_lock, common_assignee_trigger); + task->setCurrentTransaction(std::move(autocommit), std::move(txn)); assignee.scheduleMergeMutateTask(task); return true; } From 5656203bc65a5728ac74f784a0b8888124567e20 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 8 Nov 2021 21:56:09 +0300 Subject: [PATCH 014/372] minor fixes --- src/Common/TransactionID.cpp | 16 ++++++ ...{TransactionMetadata.h => TransactionID.h} | 33 +---------- src/Core/Settings.h | 2 +- .../FunctionsTransactionCounters.cpp | 2 +- src/Interpreters/Context.cpp | 1 + src/Interpreters/InterpreterFactory.cpp | 1 - .../InterpreterTransactionControlQuery.h | 4 +- src/Interpreters/MergeTreeTransaction.h | 2 +- .../MergeTreeTransactionHolder.cpp | 2 +- src/Interpreters/QueryLog.cpp | 2 +- src/Interpreters/QueryLog.h | 2 +- src/Interpreters/TransactionLog.cpp | 11 ++-- .../TransactionVersionMetadata.cpp} | 56 +++++++++---------- src/Interpreters/TransactionVersionMetadata.h | 36 ++++++++++++ src/Parsers/ASTTransactionControl.cpp | 5 -- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/System/StorageSystemParts.cpp | 6 +- .../01171_mv_select_insert_isolation_long.sh | 2 + .../01172_transaction_counters.reference | 1 + .../01172_transaction_counters.sql | 1 + .../01174_select_insert_isolation.sh | 2 + 22 files changed, 108 insertions(+), 83 deletions(-) create mode 100644 src/Common/TransactionID.cpp rename src/Common/{TransactionMetadata.h => TransactionID.h} (64%) rename src/{Common/TransactionMetadata.cpp => Interpreters/TransactionVersionMetadata.cpp} (84%) create mode 100644 src/Interpreters/TransactionVersionMetadata.h diff --git a/src/Common/TransactionID.cpp b/src/Common/TransactionID.cpp new file mode 100644 index 00000000000..4cf93636c11 --- /dev/null +++ b/src/Common/TransactionID.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +TIDHash TransactionID::getHash() const +{ + SipHash hash; + hash.update(start_csn); + hash.update(local_tid); + hash.update(host_id); + return hash.get64(); +} + +} diff --git a/src/Common/TransactionMetadata.h b/src/Common/TransactionID.h similarity index 64% rename from src/Common/TransactionMetadata.h rename to src/Common/TransactionID.h index 8aa1488f1a6..fa745efcf43 100644 --- a/src/Common/TransactionMetadata.h +++ b/src/Common/TransactionID.h @@ -21,9 +21,7 @@ struct TransactionID { CSN start_csn = 0; LocalTID local_tid = 0; - UUID host_id = UUIDHelpers::Nil; /// Depends on #17278, leave it Nil for now. - - static DataTypePtr getDataType(); + UUID host_id = UUIDHelpers::Nil; bool operator == (const TransactionID & rhs) const { @@ -49,8 +47,10 @@ namespace Tx const CSN UnknownCSN = 0; const CSN PrehistoricCSN = 1; +const CSN MaxReservedCSN = 16; const LocalTID PrehistoricLocalTID = 1; +const LocalTID MaxReservedLocalTID = 16; const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil}; @@ -60,33 +60,6 @@ const CSN RolledBackCSN = std::numeric_limits::max(); } -struct VersionMetadata -{ - const TransactionID mintid = Tx::EmptyTID; - TransactionID maxtid = Tx::EmptyTID; - - std::atomic maxtid_lock = 0; - - std::atomic mincsn = Tx::UnknownCSN; - std::atomic maxcsn = Tx::UnknownCSN; - - bool isVisible(const MergeTreeTransaction & txn); - bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); - - TransactionID getMinTID() const { return mintid; } - TransactionID getMaxTID() const; - - void lockMaxTID(const TransactionID & tid, const String & error_context = {}); - void unlockMaxTID(const TransactionID & tid); - - bool isMaxTIDLocked() const; - - /// It can be called only from MergeTreeTransaction or on server startup - void setMinTID(const TransactionID & tid); - - bool canBeRemoved(Snapshot oldest_snapshot_version); -}; - } template<> diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f8b574a21d8..e3d38127cc9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -150,7 +150,7 @@ class IColumn; M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 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(UInt64, merge_tree_clear_old_parts_interval_seconds, 30, "The period of executing the clear old parts operation in background.", 0) \ M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp index 7e821645b76..f2547734e52 100644 --- a/src/Functions/FunctionsTransactionCounters.cpp +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -29,7 +29,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes &) const override { - return TransactionID::getDataType(); + return getTransactionIDDataType(); } bool isDeterministic() const override { return false; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a2afd9a7e94..c6cb959f529 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -132,6 +132,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; } diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 4fc73f1a806..5dcee1eae05 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -63,7 +63,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/InterpreterTransactionControlQuery.h b/src/Interpreters/InterpreterTransactionControlQuery.h index b3d2cc06b9e..fc71939502e 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.h +++ b/src/Interpreters/InterpreterTransactionControlQuery.h @@ -20,8 +20,8 @@ public: bool ignoreLimits() const override { return true; } private: BlockIO executeBegin(ContextMutablePtr session_context); - BlockIO executeCommit(ContextMutablePtr session_context); - BlockIO executeRollback(ContextMutablePtr session_context); + static BlockIO executeCommit(ContextMutablePtr session_context); + static BlockIO executeRollback(ContextMutablePtr session_context); private: ContextMutablePtr query_context; diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 1ead2fdce87..921a1c10951 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index b6492286c3f..b1c50ff6d55 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -61,7 +61,7 @@ MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactio txn = nullptr; } -MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder &) +MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder &) // NOLINT { assert(txn == nullptr); return *this; diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 2a236d39a4a..8fc04d0d769 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -117,7 +117,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"used_storages", std::make_shared(std::make_shared())}, {"used_table_functions", std::make_shared(std::make_shared())}, - {"transaction_id", TransactionID::getDataType()} + {"transaction_id", getTransactionIDDataType()}, }; } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 2b3a603e681..9844eede4d0 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace ProfileEvents { diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index b8799522b5a..0624ddd116c 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -1,6 +1,7 @@ #include -#include +#include #include +#include #include namespace DB @@ -20,9 +21,9 @@ TransactionLog & TransactionLog::instance() TransactionLog::TransactionLog() : log(&Poco::Logger::get("TransactionLog")) { - latest_snapshot = 1; - csn_counter = 1; - local_tid_counter = 1; + latest_snapshot = Tx::MaxReservedCSN; + csn_counter = Tx::MaxReservedCSN; + local_tid_counter = Tx::MaxReservedLocalTID; } Snapshot TransactionLog::getLatestSnapshot() const @@ -37,7 +38,7 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() std::lock_guard lock{running_list_mutex}; Snapshot snapshot = latest_snapshot.load(); LocalTID ltid = 1 + local_tid_counter.fetch_add(1); - txn = std::make_shared(snapshot, ltid, UUIDHelpers::Nil); + txn = std::make_shared(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); diff --git a/src/Common/TransactionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp similarity index 84% rename from src/Common/TransactionMetadata.cpp rename to src/Interpreters/TransactionVersionMetadata.cpp index c3513380a49..f7d2c885f55 100644 --- a/src/Common/TransactionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -1,35 +1,18 @@ -#include -#include +#include #include #include #include #include +//#include + namespace DB { namespace ErrorCodes { -extern const int SERIALIZATION_ERROR; -extern const int LOGICAL_ERROR; -} - -DataTypePtr TransactionID::getDataType() -{ - DataTypes types; - types.push_back(std::make_shared()); - types.push_back(std::make_shared()); - types.push_back(std::make_shared()); - return std::make_shared(std::move(types)); -} - -TIDHash TransactionID::getHash() const -{ - SipHash hash; - hash.update(start_csn); - hash.update(local_tid); - hash.update(host_id); - return hash.get64(); + extern const int SERIALIZATION_ERROR; + extern const int LOGICAL_ERROR; } /// It can be used for introspection purposes only @@ -62,8 +45,8 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error if (!locked) { throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: " - "Transaction {} tried to remove data part, " - "but it's locked ({}) by another transaction {} which is currently removing this part. {}", + "Transaction {} tried to remove data part, " + "but it's locked ({}) by another transaction {} which is currently removing this part. {}", tid, expected_max_lock_value, getMaxTID(), error_context); } @@ -112,11 +95,14 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) { + //Poco::Logger * log = &Poco::Logger::get("WTF"); assert(mintid); CSN min = mincsn.load(std::memory_order_relaxed); TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); CSN max = maxcsn.load(std::memory_order_relaxed); + //LOG_TRACE(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid); + [[maybe_unused]] bool had_mincsn = min; [[maybe_unused]] bool had_maxtid = max_lock; [[maybe_unused]] bool had_maxcsn = max; @@ -151,13 +137,14 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// Data part has mintid/maxtid, but does not have mincsn/maxcsn. /// It means that some transaction is creating/removing the part right now or has done it recently - /// and we don't know if it was already committed ot not. + /// and we don't know if it was already committed or not. assert(!had_mincsn || (had_maxtid && !had_maxcsn)); assert(!current_tid || (mintid != current_tid && max_lock != current_tid.getHash())); /// Before doing CSN lookup, let's check some extra conditions. - /// If snapshot_version <= some_tid.start_csn, then changes of transaction with some_tid - /// are definitely not visible for us, so we don't need to check if it was committed. + /// 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 <= mintid.start_csn) return false; @@ -169,16 +156,18 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// But for long-running writing transactions we will always do /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. min = TransactionLog::instance().getCSN(mintid); + //LOG_TRACE(log, "Got min {}", min); if (!min) return false; /// Part creation is not committed yet - /// We don't need to check if CSNs are already writen or not, - /// because once writen CSN cannot be changed, so it's safe to overwrite it (with tha same value). + /// 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). mincsn.store(min, std::memory_order_relaxed); if (max_lock) { max = TransactionLog::instance().getCSN(max_lock); + //LOG_TRACE(log, "Got ax {}", max); if (max) maxcsn.store(max, std::memory_order_relaxed); } @@ -221,4 +210,13 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) return max <= oldest_snapshot_version; } +DataTypePtr getTransactionIDDataType() +{ + DataTypes types; + types.push_back(std::make_shared()); + types.push_back(std::make_shared()); + types.push_back(std::make_shared()); + return std::make_shared(std::move(types)); +} + } diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h new file mode 100644 index 00000000000..94d6a1905e9 --- /dev/null +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -0,0 +1,36 @@ +#pragma once +#include + +namespace DB +{ + +struct VersionMetadata +{ + const TransactionID mintid = Tx::EmptyTID; + TransactionID maxtid = Tx::EmptyTID; + + std::atomic maxtid_lock = 0; + + std::atomic mincsn = Tx::UnknownCSN; + std::atomic maxcsn = Tx::UnknownCSN; + + bool isVisible(const MergeTreeTransaction & txn); + bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); + + TransactionID getMinTID() const { return mintid; } + TransactionID getMaxTID() const; + + void lockMaxTID(const TransactionID & tid, const String & error_context = {}); + void unlockMaxTID(const TransactionID & tid); + + bool isMaxTIDLocked() const; + + /// It can be called only from MergeTreeTransaction or on server startup + void setMinTID(const TransactionID & tid); + + bool canBeRemoved(Snapshot oldest_snapshot_version); +}; + +DataTypePtr getTransactionIDDataType(); + +} diff --git a/src/Parsers/ASTTransactionControl.cpp b/src/Parsers/ASTTransactionControl.cpp index d12c9d6d6e4..2eb74903522 100644 --- a/src/Parsers/ASTTransactionControl.cpp +++ b/src/Parsers/ASTTransactionControl.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - void ASTTransactionControl::formatImpl(const FormatSettings & format /*state*/, FormatState &, FormatStateStacked /*frame*/) const { switch (action) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 12e071abe86..0a318276059 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 707c80510ce..0e646829176 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2725,7 +2725,7 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( void MergeTreeData::restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock) { - auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); + auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); //-V1018 assert(part->getState() != DataPartState::Committed); addPartContributionToColumnAndSecondaryIndexSizes(part); addPartContributionToDataVolume(part); diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 2f2b639b380..950d680f157 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace DB { @@ -84,8 +84,8 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"projections", std::make_shared(std::make_shared())}, {"visible", std::make_shared()}, - {"mintid", TransactionID::getDataType()}, - {"maxtid", TransactionID::getDataType()}, + {"mintid", getTransactionIDDataType()}, + {"maxtid", getTransactionIDDataType()}, {"mincsn", std::make_shared()}, {"maxcsn", std::make_shared()}, } diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index f506962a36a..c48f86f4aee 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: long, no-parallel +# Test is too heavy, avoid parallel run in Flaky Check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index fe055805d93..1f463a25c20 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -8,3 +8,4 @@ 4 all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 4 all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 4 all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +5 1 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 1d9311e4e86..ca114643130 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -18,6 +18,7 @@ begin transaction; insert into txn_counters(n) values (3); select 3, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; select 4, name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 5, transactionID().3 == serverUUID(); commit; drop table txn_counters; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 2bc7185e3fa..663939eb269 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +# shellcheck disable=SC2015 + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 0a4647f927db65f1b65006d1c5df5c63c26dc296 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Nov 2021 21:14:14 +0300 Subject: [PATCH 015/372] support alter partition --- .../TransactionVersionMetadata.cpp | 7 + src/Interpreters/executeQuery.cpp | 7 +- src/Storages/IStorage.h | 5 - src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 121 +++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 46 ++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 16 +-- src/Storages/StorageProxy.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 11 +- ...alter_partition_isolation_stress.reference | 6 + .../01169_alter_partition_isolation_stress.sh | 113 ++++++++++++++++ .../01170_alter_partition_isolation.reference | 30 +++++ .../01170_alter_partition_isolation.sh | 70 ++++++++++ .../01171_mv_select_insert_isolation_long.sh | 10 ++ tests/queries/0_stateless/transactions.lib | 14 ++ 17 files changed, 391 insertions(+), 71 deletions(-) create mode 100644 tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference create mode 100755 tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh create mode 100644 tests/queries/0_stateless/01170_alter_partition_isolation.reference create mode 100755 tests/queries/0_stateless/01170_alter_partition_isolation.sh create mode 100755 tests/queries/0_stateless/transactions.lib diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index f7d2c885f55..cbcff6057a5 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -44,6 +44,13 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); if (!locked) { + if (tid == Tx::PrehistoricTID && expected_max_lock_value == Tx::PrehistoricTID.getHash()) + { + /// Don't need to lock part for queries without transaction + //FIXME Transactions: why is it possible? + return; + } + throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: " "Transaction {} tried to remove data part, " "but it's locked ({}) by another transaction {} which is currently removing this part. {}", diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index dde6ff7dd72..b558e58997d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -172,10 +172,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)); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fa5f2c28b06..3bdd535b585 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -538,11 +538,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; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0a318276059..d6b25a5a6ef 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -226,6 +226,7 @@ public: */ enum class State { + ///TODO Transactions: rename Committed to Active, because it becomes confusing Temporary, /// the part is generating now, it is not in data_parts list PreCommitted, /// the part is in data_parts, but not used for SELECTs Committed, /// active data part, used by current and upcoming SELECTs diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cfb5a32ae9e..d14a4d9260f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3142,9 +3142,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::Committed, 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::Committed, partition_id}; auto lock = lockParts(); return DataPartsVector( @@ -3152,19 +3166,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 & partition_ids) const +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set & 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::Committed, 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::Committed, 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; } @@ -3295,10 +3327,10 @@ void MergeTreeData::checkAlterPartitionIsPossible( } } -void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) +void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context) { - const String partition_id = getPartitionIDFromQuery(partition, getContext()); - auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + const String partition_id = getPartitionIDFromQuery(partition, local_context); + auto parts_to_remove = getVisibleDataPartsVectorInPartition(local_context, partition_id); UInt64 partition_size = 0; @@ -3337,7 +3369,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::Committed, partition_id); + parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); auto disk = getStoragePolicy()->getDiskByName(name); if (!disk) @@ -3382,7 +3414,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::Committed, partition_id); + parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); auto volume = getStoragePolicy()->getVolumeByName(name); if (!volume) @@ -3454,7 +3486,7 @@ Pipe MergeTreeData::alterPartition( } else { - checkPartitionCanBeDropped(command.partition); + checkPartitionCanBeDropped(command.partition, query_context); dropPartition(command.partition, command.detach, query_context); } } @@ -3503,7 +3535,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); @@ -3564,7 +3596,7 @@ BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_co if (partitions.empty()) data_parts = getDataPartsVector(); else - data_parts = getDataPartsVectorInPartitions(MergeTreeDataPartState::Committed, getPartitionIDsFromQuery(partitions, local_context)); + data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(partitions, local_context)); return backupDataParts(data_parts); } @@ -3771,26 +3803,54 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } -DataPartsVector MergeTreeData::getDataPartsVector(ContextPtr local_context) const +DataPartsVector MergeTreeData::getVisibleDataPartsVector(ContextPtr local_context) const { - return getVisibleDataPartsVector(local_context->getCurrentTransaction()); + DataPartsVector res; + if (const auto * txn = local_context->getCurrentTransaction().get()) + { + res = getDataPartsVector({DataPartState::Committed, DataPartState::Outdated}); + filterVisibleDataParts(res, txn->getSnapshot(), txn->tid); + } + else + { + res = getDataPartsVector(); + } + return res; } MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const { - if (!txn) - return getDataPartsVector(); + DataPartsVector res; + if (txn) + { + res = getDataPartsVector({DataPartState::Committed, DataPartState::Outdated}); + filterVisibleDataParts(res, txn->getSnapshot(), txn->tid); + } + else + { + res = getDataPartsVector(); + } + return res; +} - DataPartsVector maybe_visible_parts = getDataPartsVector({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(Snapshot snapshot_version, TransactionID current_tid) const +{ + auto res = getDataPartsVector({DataPartState::Committed, DataPartState::Outdated}); + filterVisibleDataParts(res, snapshot_version, current_tid); + return res; +} + +void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts, Snapshot snapshot_version, TransactionID current_tid) const +{ if (maybe_visible_parts.empty()) - return maybe_visible_parts; + return; auto it = maybe_visible_parts.begin(); auto it_last = maybe_visible_parts.end() - 1; String visible_parts_str; while (it <= it_last) { - if ((*it)->versions.isVisible(*txn)) + if ((*it)->versions.isVisible(snapshot_version, current_tid)) { visible_parts_str += (*it)->name; visible_parts_str += " "; @@ -3804,9 +3864,8 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me } size_t new_size = it_last - maybe_visible_parts.begin() + 1; - LOG_TRACE(log, "Got {} parts visible for {}: {}", new_size, txn->tid, visible_parts_str); + LOG_TEST(log, "Got {} parts visible in snapshot {} (TID {}): {}", new_size, snapshot_version, current_tid, visible_parts_str); maybe_visible_parts.resize(new_size); - return maybe_visible_parts; } @@ -4238,7 +4297,7 @@ MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affo return res; } -MergeTreeData::DataParts MergeTreeData::getDataParts() const +MergeTreeData::DataParts MergeTreeData::getDataPartsForInternalUsage() const { return getDataParts({DataPartState::Committed}); } @@ -4879,7 +4938,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); } - auto parts = getDataPartsVector(query_context); + auto parts = getVisibleDataPartsVector(query_context); // If minmax_count_projection is a valid candidate, check its completeness. if (minmax_conut_projection_candidate) @@ -5237,7 +5296,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 / ""; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bf00db2c5c1..903cf0f979c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -425,6 +425,7 @@ 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; @@ -436,24 +437,18 @@ public: /// 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; - - void validateDetachedPartName(const String & name) const; - - void dropDetached(const ASTPtr & partition, bool part, ContextPtr context); - - MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, - ContextPtr context, PartsTemporaryRename & renamed_parts); - - /// Returns Committed parts - DataParts getDataParts() const; + /// Returns parts in Committed state (NOT in terms of transactions, should be used carefully) + DataParts getDataPartsForInternalUsage() const; DataPartsVector getDataPartsVector() const; - DataPartsVector getDataPartsVector(ContextPtr local_context) const; - DataPartsVector getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const; + void filterVisibleDataParts(DataPartsVector & maybe_visible_parts, Snapshot snapshot_version, TransactionID current_tid) const; - /// Returns a committed part with the given name or a part containing it. If there is no such part, returns nullptr. + /// Returns parts that visible with current snapshot + DataPartsVector getVisibleDataPartsVector(ContextPtr local_context) const; + DataPartsVector getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const; + DataPartsVector getVisibleDataPartsVector(Snapshot snapshot_version, TransactionID current_tid) const; + + /// Returns a part in Committed 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; @@ -463,8 +458,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 & partition_ids) const; + DataPartsVector getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const; + DataPartsVector getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set & 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); @@ -484,6 +479,18 @@ public: /// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition. std::optional getMinPartDataVersion() const; + + /// Returns all detached parts + DetachedPartsInfo getDetachedParts() const; + + void validateDetachedPartName(const String & name) const; + + 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; @@ -656,7 +663,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); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3a6944df633..7fa55005f93 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -395,7 +395,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) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1fc61012e6f..c766c775ac7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -130,7 +130,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( return std::make_unique(); const auto & settings = context->getSettingsRef(); - auto parts = data.getDataPartsVector(context); + auto parts = data.getVisibleDataPartsVector(context); if (!query_info.projection) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e1165dbe3c8..ec7504715e8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -87,7 +87,7 @@ StorageMergeTree::StorageMergeTree( { loadDataParts(has_force_restore_data_flag); - if (!attach && !getDataParts().empty()) + if (!attach && !getDataPartsForInternalUsage().empty()) throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA); increment.set(getMaxBlockNumber()); @@ -258,7 +258,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); - auto parts_to_remove = getDataPartsVector(); + auto parts_to_remove = getVisibleDataPartsVector(local_context); removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), parts_to_remove, true); LOG_INFO(log, "Removed {} parts.", parts_to_remove.size()); @@ -713,9 +713,9 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( { /// Cannot merge parts if some of them is not visible in current snapshot /// TODO We can use simplified visibility rules (without CSN lookup) here - if (left && !left->versions.isVisible(*tx)) + if (left && !left->versions.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; - if (right && !right->versions.isVisible(*tx)) + if (right && !right->versions.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; } @@ -1288,7 +1288,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); String partition_id = getPartitionIDFromQuery(partition, local_context); - parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + parts_to_remove = getVisibleDataPartsVectorInPartition(local_context, partition_id); /// TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), parts_to_remove, true); @@ -1370,7 +1370,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); - DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); MutableDataPartsVector dst_parts; static const String TMP_PREFIX = "tmp_replace_from_"; @@ -1455,7 +1455,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); - DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); MutableDataPartsVector dst_parts; static const String TMP_PREFIX = "tmp_move_from_"; @@ -1535,7 +1535,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ if (const auto & check_query = query->as(); check_query.partition) { String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); - data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } else data_parts = getDataPartsVector(); diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 304f84c02eb..c3f4536da82 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -145,7 +145,6 @@ public: CheckResults checkData(const ASTPtr & query , ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } - void checkPartitionCanBeDropped(const ASTPtr & partition) override { getNested()->checkPartitionCanBeDropped(partition); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 306d4d4f7e9..3a0aa3089a2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -405,7 +405,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( if (!attach) { - if (!getDataParts().empty()) + if (!getDataPartsForInternalUsage().empty()) throw Exception("Data directory for table already contains data parts" " - probably it was unclean DROP table or manual intervention." " You must either clear directory by hand or use ATTACH TABLE" @@ -2452,7 +2452,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); - auto local_active_parts = getDataParts(); + auto local_active_parts = getDataPartsForInternalUsage(); DataPartsVector parts_to_remove_from_working_set; @@ -4187,7 +4187,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg { ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; - for (const auto & data_part : getDataParts()) + for (const auto & data_part : getDataPartsForInternalUsage()) { max_added_blocks[data_part->info.partition_id] = std::max(max_added_blocks[data_part->info.partition_id], data_part->info.max_block); @@ -4293,6 +4293,7 @@ void StorageReplicatedMergeTree::foreachCommittedParts(Func && func, bool select max_added_blocks = getMaxAddedBlocks(); auto lock = lockParts(); + /// TODO Transactions: should we count visible parts only? for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) { if (part->isEmpty()) @@ -6246,7 +6247,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( String partition_id = getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); @@ -7068,7 +7069,7 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context if (const auto & check_query = query->as(); check_query.partition) { String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); - data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } else data_parts = getDataPartsVector(); diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference new file mode 100644 index 00000000000..bf9c6e88bb2 --- /dev/null +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference @@ -0,0 +1,6 @@ +1 1 +2 1 +3 1 +4 1 +1 +10 100 diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh new file mode 100755 index 00000000000..f05a3ec2b24 --- /dev/null +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -0,0 +1,113 @@ +#!/usr/bin/env bash +# Tags: long + +# shellcheck disable=SC2015 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0"; + +function thread_insert() +{ + set -e + trap "exit 0" INT + while true; do + action="ROLLBACK" + if (( RANDOM % 2 )); then + action="COMMIT" + fi + val=$RANDOM + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES ($val, 1); + INSERT INTO src VALUES ($val, 2); + COMMIT;" + sleep 0.$RANDOM; + done +} + + +# NOTE +# ALTER PARTITION query stops merges, +# but serialization error is still possible if some merge was assigned (and committed) between BEGIN and ALTER. +function thread_partition_src_to_dst() +{ + set -e + count=0 + sum=0 + for i in {1..20}; do + out=$( + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES ($i, 3); + INSERT INTO dst SELECT * FROM src; + ALTER TABLE src DROP PARTITION ID 'all'; + SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; + COMMIT;" 2>&1) ||: + + echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" ||: + echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1)) && sum=$((sum+i)) + done +} + +function thread_partition_dst_to_src() +{ + set -e + for i in {1..20}; do + action="ROLLBACK" + if (( i % 2 )); then + action="COMMIT" + fi + $CLICKHOUSE_CLIENT --multiquery --query " + SYSTEM STOP MERGES dst; + BEGIN TRANSACTION; + INSERT INTO dst VALUES ($i, 4); + INSERT INTO src SELECT * FROM dst; + ALTER TABLE dst DROP PARTITION ID 'all'; + SYSTEM START MERGES dst; + SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=4) != (toUInt8($i/2 + 1), (select sum(number) from numbers(1, $i) where number % 2 or number=$i))) FORMAT Null; + $action;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" + done +} + +function thread_select() +{ + set -e + trap "exit 0" INT + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + -- no duplicates + SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; + SELECT type, throwIf(count(n) != countDistinct(n)) FROM dst GROUP BY type FORMAT Null; + -- rows inserted by thread_insert moved together + SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null; + COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" + done +} + +thread_insert & PID_1=$! +thread_select & PID_2=$! + +thread_partition_src_to_dst & PID_3=$! +thread_partition_dst_to_src & PID_4=$! +wait $PID_3 && wait $PID_4 + +kill -INT $PID_1 +kill -INT $PID_2 +wait + +$CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" + + +$CLICKHOUSE_CLIENT --query "DROP TABLE src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE dst"; diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.reference b/tests/queries/0_stateless/01170_alter_partition_isolation.reference new file mode 100644 index 00000000000..fc772355a57 --- /dev/null +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.reference @@ -0,0 +1,30 @@ +1 1 +2 3 +3 2 +3 4 +4 3 + +5 3 +5 5 + +6 3 +6 5 +6 6 +7 8 +8 3 +8 5 +8 7 +8 9 +SERIALIZATION_ERROR +INVALID_TRANSACTION +9 8 + +10 8 + +11 8 +11 11 +11 12 +12 8 +12 8 +12 11 +12 12 diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.sh b/tests/queries/0_stateless/01170_alter_partition_isolation.sh new file mode 100755 index 00000000000..4174f8215fe --- /dev/null +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.sh @@ -0,0 +1,70 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./transactions.lib +. "$CURDIR"/transactions.lib + +$CLICKHOUSE_CLIENT -q "drop table if exists mt" +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n" + +tx 1 "begin transaction" +tx 1 "insert into mt values (1)" +tx 2 "begin transaction" +tx 2 "insert into mt values (2)" +tx 1 "select 1, n from mt order by n" +tx 1 "alter table mt drop partition id 'all'" +tx 2 "insert into mt values (4)" +tx 1 "insert into mt values (3)" +tx 1 "select 2, n from mt order by n" +tx 2 "select 3, n from mt order by n" +tx 2 "alter table mt drop partition id 'all'" +tx 2 "insert into mt values (5)" +tx 1 "select 4, n from mt order by n" +tx 2 "commit" +tx 1 "commit" + +echo '' +$CLICKHOUSE_CLIENT -q "select 5, n from mt order by n" +echo '' + +tx 4 "begin transaction" +tx 4 "insert into mt values (6)" +tx 3 "begin transaction" +tx 3 "insert into mt values (7)" +tx 4 "select 6, n from mt order by n" +tx 4 "alter table mt drop partition id 'all'" +tx 3 "insert into mt values (9)" +tx 4 "insert into mt values (8)" +tx 4 "select 7, n from mt order by n" +tx 3 "select 8, n from mt order by n" +tx 3 "alter table mt drop partition id 'all'" | grep -Eo "SERIALIZATION_ERROR" | uniq +tx 3 "insert into mt values (10)" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 4 "select 9, n from mt order by n" +tx 3 "rollback" +tx 4 "commit" + +echo '' +$CLICKHOUSE_CLIENT -q "select 10, n from mt order by n" +echo '' + +$CLICKHOUSE_CLIENT -q "drop table if exists another_mt" +$CLICKHOUSE_CLIENT -q "create table another_mt (n int) engine=MergeTree order by n" + +tx 5 "begin transaction" +tx 5 "insert into another_mt values (11)" +tx 6 "begin transaction" +tx 6 "insert into mt values (12)" +tx 6 "insert into another_mt values (13)" +tx 5 "alter table another_mt move partition id 'all' to table mt" +tx 6 "alter table another_mt replace partition id 'all' from mt" +tx 5 "alter table another_mt attach partition id 'all' from mt" +tx 5 "commit" +tx 6 "commit" + +$CLICKHOUSE_CLIENT -q "select 11, n from mt order by n" +$CLICKHOUSE_CLIENT -q "select 12, n from another_mt order by n" + +$CLICKHOUSE_CLIENT -q "drop table another_mt" +$CLICKHOUSE_CLIENT -q "drop table mt" diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index c48f86f4aee..672a49df5fc 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -22,6 +22,7 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO src VALUES (0, 0)" # some transactions will fail due to constraint function thread_insert_commit() { + set -e for i in {1..100}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; @@ -34,6 +35,7 @@ function thread_insert_commit() function thread_insert_rollback() { + set -e for _ in {1..100}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; @@ -46,11 +48,17 @@ function thread_insert_rollback() # make merges more aggressive function thread_optimize() { + set -e trap "exit 0" INT while true; do optimize_query="OPTIMIZE TABLE src" + partition_id=$(( RANDOM % 2 )) if (( RANDOM % 2 )); then optimize_query="OPTIMIZE TABLE dst" + partition_id="all" + fi + if (( RANDOM % 2 )); then + optimize_query="$optimize_query PARTITION ID '$partition_id'" fi if (( RANDOM % 2 )); then optimize_query="$optimize_query FINAL" @@ -71,6 +79,7 @@ function thread_optimize() function thread_select() { + set -e trap "exit 0" INT while true; do $CLICKHOUSE_CLIENT --multiquery --query " @@ -86,6 +95,7 @@ function thread_select() function thread_select_insert() { + set -e trap "exit 0" INT while true; do $CLICKHOUSE_CLIENT --multiquery --query " diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib new file mode 100755 index 00000000000..2d3eafc784a --- /dev/null +++ b/tests/queries/0_stateless/transactions.lib @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +function tx() +{ + tx_num=$1 + query=$2 + + url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" + session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_$tx_num" + url="${url_without_session}session_id=$session&database=$CLICKHOUSE_DATABASE" + + ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "$query" +} + From f2dbeee6c2aa3b6e391defbada3f785d28cf66d6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Nov 2021 23:03:15 +0300 Subject: [PATCH 016/372] fix test --- .../0_stateless/01169_alter_partition_isolation_stress.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index f05a3ec2b24..693c332dd1c 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -53,7 +53,8 @@ function thread_partition_src_to_dst() COMMIT;" 2>&1) ||: echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" ||: - echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1)) && sum=$((sum+i)) + echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1)) + echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || sum=$((sum+i)) done } From 600087265806c40b1b9c388993b716ed1f8ed308 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Nov 2021 15:06:22 +0300 Subject: [PATCH 017/372] fix tests --- .../01169_alter_partition_isolation_stress.reference | 2 ++ .../0_stateless/01169_alter_partition_isolation_stress.sh | 8 ++++++-- .../0_stateless/01170_alter_partition_isolation.sh | 3 +++ 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference index bf9c6e88bb2..12b941eab50 100644 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.reference @@ -4,3 +4,5 @@ 4 1 1 10 100 +1 1 1 +2 1 1 diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 693c332dd1c..4506065484d 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-replicated-database # shellcheck disable=SC2015 @@ -18,17 +18,18 @@ function thread_insert() { set -e trap "exit 0" INT + val=1 while true; do action="ROLLBACK" if (( RANDOM % 2 )); then action="COMMIT" fi - val=$RANDOM $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; INSERT INTO src VALUES ($val, 1); INSERT INTO src VALUES ($val, 2); COMMIT;" + val=$((val+1)) sleep 0.$RANDOM; done } @@ -90,6 +91,8 @@ function thread_select() SELECT type, throwIf(count(n) != countDistinct(n)) FROM dst GROUP BY type FORMAT Null; -- rows inserted by thread_insert moved together SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null; + -- all rows are inserted in insert_thread + SELECT type, throwIf(count(n) != max(n)), throwIf(sum(n) != max(n)*(max(n)+1)/2) FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type FORMAT Null; COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" done } @@ -108,6 +111,7 @@ wait $CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" $CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" +$CLICKHOUSE_CLIENT -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.sh b/tests/queries/0_stateless/01170_alter_partition_isolation.sh index 4174f8215fe..2db178fb6d1 100755 --- a/tests/queries/0_stateless/01170_alter_partition_isolation.sh +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.sh @@ -1,4 +1,7 @@ #!/usr/bin/env bash +# Tags: no-fasttest, no-replicated-database +# Looks like server does not listen https port in fasttest +# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 52885db5d741951a2e08ab38cbe104cf90332250 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 Nov 2021 19:51:03 +0300 Subject: [PATCH 018/372] remove strange multimap from mutations --- .../MergeTree/MergeTreeMutationEntry.cpp | 45 ++++++++++++++++--- .../MergeTree/MergeTreeMutationEntry.h | 10 +++-- src/Storages/StorageMergeTree.cpp | 23 +++++----- src/Storages/StorageMergeTree.h | 4 +- 4 files changed, 59 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 2aefb3df2be..0f71742fb09 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -10,7 +11,39 @@ namespace DB { -MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, Int64 tmp_number) +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +String MergeTreeMutationEntry::versionToFileName(UInt64 block_number_) +{ + assert(block_number_); + return fmt::format("mutation_{}.txt", block_number_); +} + +UInt64 MergeTreeMutationEntry::tryParseFileName(const String & file_name_) +{ + UInt64 maybe_block_number = 0; + ReadBufferFromString file_name_buf(file_name_); + if (!checkString("mutation_", file_name_buf)) + return 0; + if (!tryReadIntText(maybe_block_number, file_name_buf)) + return 0; + if (!checkString(".txt", file_name_buf)) + return 0; + assert(maybe_block_number); + return maybe_block_number; +} + +UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) +{ + if (UInt64 maybe_block_number = tryParseFileName(file_name_)) + return maybe_block_number; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse mutation version from file name, expected 'mutation_.txt', got '{}'", file_name_); +} + +MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -35,10 +68,11 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP } } -void MergeTreeMutationEntry::commit(Int64 block_number_) +void MergeTreeMutationEntry::commit(UInt64 block_number_) { + assert(block_number_); block_number = block_number_; - String new_file_name = "mutation_" + toString(block_number) + ".txt"; + String new_file_name = versionToFileName(block_number); disk->moveFile(path_prefix + file_name, path_prefix + new_file_name); is_temp = false; file_name = new_file_name; @@ -62,10 +96,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat , file_name(file_name_) , is_temp(false) { - ReadBufferFromString file_name_buf(file_name); - file_name_buf >> "mutation_" >> block_number >> ".txt"; - assertEOF(file_name_buf); - + block_number = parseFileName(file_name); auto buf = disk->readFile(path_prefix + file_name); *buf >> "format version: 1\n"; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index e01ce4320b3..7554a03836e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -21,7 +21,7 @@ struct MergeTreeMutationEntry String file_name; bool is_temp = false; - Int64 block_number = 0; + UInt64 block_number = 0; String latest_failed_part; MergeTreePartInfo latest_failed_part_info; @@ -29,15 +29,19 @@ struct MergeTreeMutationEntry String latest_fail_reason; /// Create a new entry and write it to a temporary file. - MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, Int64 tmp_number); + MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; /// Commit entry and rename it to a permanent file. - void commit(Int64 block_number_); + void commit(UInt64 block_number_); void removeFile(); + static String versionToFileName(UInt64 block_number_); + static UInt64 tryParseFileName(const String & file_name_); + static UInt64 parseFileName(const String & file_name_); + /// Load an existing entry. MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ec7504715e8..d9231b25a70 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -412,8 +412,9 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String version = increment.get(); entry.commit(version); mutation_file_name = entry.file_name; - auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); - current_mutations_by_version.emplace(version, insertion.first->second); + bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); LOG_INFO(log, "Added mutation: {}", mutation_file_name); } @@ -618,16 +619,18 @@ std::vector StorageMergeTree::getMutationsStatus() cons CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { LOG_TRACE(log, "Killing mutation {}", mutation_id); + UInt64 mutation_version = MergeTreeMutationEntry::tryParseFileName(mutation_id); + if (!mutation_version) + return CancellationCode::NotFound; std::optional to_kill; { std::lock_guard lock(currently_processing_in_background_mutex); - auto it = current_mutations_by_id.find(mutation_id); - if (it != current_mutations_by_id.end()) + auto it = current_mutations_by_version.find(mutation_version); + if (it != current_mutations_by_version.end()) { to_kill.emplace(std::move(it->second)); - current_mutations_by_id.erase(it); - current_mutations_by_version.erase(to_kill->block_number); + current_mutations_by_version.erase(it); } } @@ -668,10 +671,11 @@ void StorageMergeTree::loadMutations() if (startsWith(it->name(), "mutation_")) { MergeTreeMutationEntry entry(disk, path, it->name()); - Int64 block_number = entry.block_number; + UInt64 block_number = entry.block_number; LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); - auto insertion = current_mutations_by_id.emplace(it->name(), std::move(entry)); - current_mutations_by_version.emplace(block_number, insertion.first->second); + auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -1111,7 +1115,6 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) for (size_t i = 0; i < to_delete_count; ++i) { mutations_to_delete.push_back(std::move(it->second)); - current_mutations_by_id.erase(mutations_to_delete.back().file_name); it = current_mutations_by_version.erase(it); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index dd737dbd5fa..9edef38fef4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -131,9 +131,7 @@ private: /// This set have to be used with `currently_processing_in_background_mutex`. DataParts currently_merging_mutating_parts; - - std::map current_mutations_by_id; - std::multimap current_mutations_by_version; + std::map current_mutations_by_version; std::atomic shutdown_called {false}; From 6e1c16c2e7df8d091b03dcebfcb53ddc04012fc8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 14 Dec 2021 23:06:34 +0300 Subject: [PATCH 019/372] add support for mutations --- src/Common/TransactionID.h | 42 +++-- src/Interpreters/MergeTreeTransaction.cpp | 69 ++++++-- src/Interpreters/MergeTreeTransaction.h | 10 +- src/Interpreters/TransactionLog.cpp | 2 +- .../TransactionVersionMetadata.cpp | 39 ++-- src/Interpreters/TransactionVersionMetadata.h | 1 + src/Storages/IStorage.h | 5 + .../MergeTree/MergeMutateSelectedEntry.h | 6 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 3 +- .../MergeTree/MergeTreeMutationEntry.cpp | 8 +- .../MergeTree/MergeTreeMutationEntry.h | 9 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 7 +- src/Storages/StorageMergeTree.cpp | 166 ++++++++++++------ src/Storages/StorageMergeTree.h | 5 +- .../01168_mutations_isolation.reference | 20 +++ .../0_stateless/01168_mutations_isolation.sh | 57 ++++++ 17 files changed, 340 insertions(+), 111 deletions(-) create mode 100644 tests/queries/0_stateless/01168_mutations_isolation.reference create mode 100755 tests/queries/0_stateless/01168_mutations_isolation.sh diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index fa745efcf43..9037601ffad 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -17,6 +17,17 @@ using Snapshot = CSN; using LocalTID = UInt64; using TIDHash = UInt64; +namespace Tx +{ + const CSN UnknownCSN = 0; + const CSN PrehistoricCSN = 1; + const CSN CommittingCSN = 2; /// TODO do we really need it? + const CSN MaxReservedCSN = 16; + + const LocalTID PrehistoricLocalTID = 1; + const LocalTID MaxReservedLocalTID = 16; +} + struct TransactionID { CSN start_csn = 0; @@ -33,31 +44,28 @@ struct TransactionID return !(*this == rhs); } - operator bool() const + TIDHash getHash() const; + + bool isEmpty() const { - assert(local_tid || (start_csn == 0 && host_id == UUIDHelpers::Nil)); - return local_tid; + assert((local_tid == 0) == (start_csn == 0 && host_id == UUIDHelpers::Nil)); + return local_tid == 0; } - TIDHash getHash() const; + bool isPrehistoric() const + { + assert((local_tid == Tx::PrehistoricLocalTID) == (start_csn == Tx::PrehistoricCSN)); + return local_tid == Tx::PrehistoricLocalTID; + } }; namespace Tx { + const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; + const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil}; -const CSN UnknownCSN = 0; -const CSN PrehistoricCSN = 1; -const CSN MaxReservedCSN = 16; - -const LocalTID PrehistoricLocalTID = 1; -const LocalTID MaxReservedLocalTID = 16; - -const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; -const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil}; - -/// So far, that changes will never become visible -const CSN RolledBackCSN = std::numeric_limits::max(); - + /// So far, that changes will never become visible + const CSN RolledBackCSN = std::numeric_limits::max(); } } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index b9fb94de8e3..b2f1d8ae150 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -6,6 +6,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_TRANSACTION; + extern const int LOGICAL_ERROR; +} + MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id) : tid({snapshot_, local_tid_, host_id}) , snapshot(snapshot_) @@ -15,9 +21,10 @@ MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_ti MergeTreeTransaction::State MergeTreeTransaction::getState() const { - if (csn == Tx::UnknownCSN) + CSN c = csn.load(); + if (c == Tx::UnknownCSN || c == Tx::CommittingCSN) return RUNNING; - if (csn == Tx::RolledBackCSN) + if (c == Tx::RolledBackCSN) return ROLLED_BACK; return COMMITTED; } @@ -64,42 +71,77 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part) { - assert(csn == Tx::UnknownCSN); + 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); + storages.insert(storage); creating_parts.push_back(new_part); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) { - assert(csn == Tx::UnknownCSN); + 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); + storages.insert(storage); removing_parts.push_back(part_to_remove); } +void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) +{ + mutations.emplace_back(table, mutation_id); +} + bool MergeTreeTransaction::isReadOnly() const { return creating_parts.empty() && removing_parts.empty(); } -void MergeTreeTransaction::beforeCommit() const +void MergeTreeTransaction::beforeCommit() { - assert(csn == Tx::UnknownCSN); + for (const auto & table_and_mutation : mutations) + table_and_mutation.first->waitForMutation(table_and_mutation.second); + + CSN expected = Tx::UnknownCSN; + bool can_commit = csn.compare_exchange_strong(expected, Tx::CommittingCSN); + if (can_commit) + return; + + if (expected == Tx::RolledBackCSN) + throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", expected); } void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept { - assert(csn == Tx::UnknownCSN); - csn = assigned_csn; + [[maybe_unused]] CSN prev_value = csn.exchange(assigned_csn); + assert(prev_value == Tx::CommittingCSN); for (const auto & part : creating_parts) part->versions.mincsn.store(csn); for (const auto & part : removing_parts) part->versions.maxcsn.store(csn); } -void MergeTreeTransaction::rollback() noexcept +bool MergeTreeTransaction::rollback() noexcept { - assert(csn == Tx::UnknownCSN); - csn = Tx::RolledBackCSN; + CSN expected = Tx::UnknownCSN; + bool need_rollback = csn.compare_exchange_strong(expected, Tx::RolledBackCSN); + + if (!need_rollback) + { + /// TODO add assertions for the case when this method called from background operation + return false; + } + + for (const auto & table_and_mutation : mutations) + table_and_mutation.first->killMutation(table_and_mutation.second); + for (const auto & part : creating_parts) part->versions.mincsn.store(Tx::RolledBackCSN); @@ -114,14 +156,11 @@ void MergeTreeTransaction::rollback() noexcept if (part->versions.getMinTID() != tid) const_cast(part->storage).restoreAndActivatePart(part); - /// FIXME seems like session holds shared_ptr to Transaction and transaction holds shared_ptr to parts preventing cleanup + return true; } void MergeTreeTransaction::onException() { - if (csn) - return; - TransactionLog::instance().rollbackTransaction(shared_from_this()); } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 921a1c10951..6733dba4d8f 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -35,6 +35,8 @@ public: void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part); void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove); + 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); @@ -46,9 +48,9 @@ public: String dumpDescription() const; private: - void beforeCommit() const; + void beforeCommit(); void afterCommit(CSN assigned_csn) noexcept; - void rollback() noexcept; + bool rollback() noexcept; Snapshot snapshot; @@ -56,10 +58,12 @@ private: DataPartsVector creating_parts; DataPartsVector removing_parts; - CSN csn; + std::atomic csn; /// FIXME it's ugly std::list::iterator snapshot_in_use_it; + + std::vector> mutations; }; using MergeTreeTransactionPtr = std::shared_ptr; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 0624ddd116c..f084c9dbc3c 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -87,7 +87,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { LOG_TRACE(log, "Rolling back transaction {}", txn->tid); - txn->rollback(); + if (txn->rollback()) { std::lock_guard lock{running_list_mutex}; bool removed = running_list.erase(txn->tid.getHash()); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index cbcff6057a5..c8e6b9dcfba 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -38,7 +38,20 @@ TransactionID VersionMetadata::getMaxTID() const void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context) { - assert(tid); + TIDHash locked_by = 0; + if (tryLockMaxTID(tid, &locked_by)) + return; + + throw Exception(ErrorCodes::SERIALIZATION_ERROR, + "Serialization error: " + "Transaction {} tried to remove data part, " + "but it's locked ({}) by another transaction {} which is currently removing this part. {}", + tid, locked_by, getMaxTID(), error_context); +} + +bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id) +{ + assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash expected_max_lock_value = 0; bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); @@ -48,21 +61,21 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error { /// Don't need to lock part for queries without transaction //FIXME Transactions: why is it possible? - return; + return true; } - throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: " - "Transaction {} tried to remove data part, " - "but it's locked ({}) by another transaction {} which is currently removing this part. {}", - tid, expected_max_lock_value, getMaxTID(), error_context); + if (locked_by_id) + *locked_by_id = expected_max_lock_value; + return false; } maxtid = tid; + return true; } void VersionMetadata::unlockMaxTID(const TransactionID & tid) { - assert(tid); + assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash locked_by = maxtid_lock.load(); @@ -91,7 +104,7 @@ void VersionMetadata::setMinTID(const TransactionID & tid) { /// TODO Transactions: initialize it in constructor on part creation and remove this method /// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times - assert(!mintid || mintid == tid); + assert(mintid.isEmpty() || mintid == tid); const_cast(mintid) = tid; } @@ -103,7 +116,7 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) { //Poco::Logger * log = &Poco::Logger::get("WTF"); - assert(mintid); + assert(!mintid.isEmpty()); CSN min = mincsn.load(std::memory_order_relaxed); TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); CSN max = maxcsn.load(std::memory_order_relaxed); @@ -115,6 +128,8 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current [[maybe_unused]] bool had_maxcsn = max; assert(!had_maxcsn || had_maxtid); assert(!had_maxcsn || had_mincsn); + assert(min == Tx::UnknownCSN || min == Tx::PrehistoricCSN || Tx::MaxReservedCSN < min); + assert(max == Tx::UnknownCSN || max == Tx::PrehistoricCSN || Tx::MaxReservedCSN < max); /// Fast path: @@ -126,7 +141,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return false; if (max && max <= snapshot_version) return false; - if (current_tid && max_lock && max_lock == current_tid.getHash()) + if (!current_tid.isEmpty() && max_lock && max_lock == current_tid.getHash()) return false; /// Otherwise, part is definitely visible if: @@ -137,7 +152,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return true; if (min && min <= snapshot_version && max && snapshot_version < max) return true; - if (current_tid && mintid == current_tid) + if (!current_tid.isEmpty() && mintid == current_tid) return true; /// End of fast path. @@ -146,7 +161,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// 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_mincsn || (had_maxtid && !had_maxcsn)); - assert(!current_tid || (mintid != current_tid && max_lock != current_tid.getHash())); + assert(current_tid.isEmpty() || (mintid != current_tid && max_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 diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 94d6a1905e9..48049da2a23 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -20,6 +20,7 @@ struct VersionMetadata TransactionID getMinTID() const { return mintid; } TransactionID getMaxTID() const; + bool tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id = nullptr); void lockMaxTID(const TransactionID & tid, const String & error_context = {}); void unlockMaxTID(const TransactionID & tid); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6481922067b..c5d9d1c0657 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -471,6 +471,11 @@ 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); + } + /// Cancel a part move to shard. virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/) { diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index 64136205157..488dfba7618 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -39,10 +39,14 @@ struct MergeMutateSelectedEntry FutureMergedMutatedPartPtr future_part; CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; - MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, MutationCommandsConstPtr commands_) + //TransactionID mutation_tid; + MergeTreeTransactionPtr txn; + MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = nullptr) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) + , txn(txn_) {} }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee3cc893f03..48b5dece723 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2446,7 +2446,7 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, MergeTreeTra bool MergeTreeData::renameTempPartAndReplace( MutableDataPartPtr & part, MergeTreeTransaction * txn, SimpleIncrement * increment, Transaction * out_transaction, - std::unique_lock & lock, DataPartsVector * out_covered_parts, MergeTreeDeduplicationLog * deduplication_log) + std::unique_lock & lock, DataPartsVector * out_covered_parts, MergeTreeDeduplicationLog * deduplication_log, bool) { if (out_transaction && &out_transaction->data != this) throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 615d2a45826..d5b8901527d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -521,9 +521,10 @@ public: 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, - DataPartsVector * out_covered_parts = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr); + DataPartsVector * out_covered_parts = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr, bool add_to_txn = true); /// Remove parts from working set immediately (without wait for background diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 0f71742fb09..744a38f5e52 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -43,16 +44,20 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse mutation version from file name, expected 'mutation_.txt', got '{}'", file_name_); } -MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number) +MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, + const TransactionID & tid_) : 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_) + , csn(Tx::UnknownCSN) { try { + /// TODO Transactions: write (and read) tid auto out = disk->writeFile(path_prefix + file_name); *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; @@ -112,6 +117,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat *buf >> "\n"; assertEOF(*buf); + csn = Tx::PrehistoricCSN; } MergeTreeMutationEntry::~MergeTreeMutationEntry() diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 7554a03836e..5fb92b9954d 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -28,8 +29,14 @@ struct MergeTreeMutationEntry time_t latest_fail_time = 0; String latest_fail_reason; + /// ID of transaction which has created mutation. + TransactionID tid = Tx::PrehistoricTID; + CSN csn; + /// Create a new entry and write it to a temporary file. - MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number); + MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, + const TransactionID & tid_); + MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index ceca96370c4..4e2ca8aee54 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -1,7 +1,7 @@ #include #include - +#include namespace DB { @@ -85,7 +85,8 @@ bool MutatePlainMergeTreeTask::executeStep() new_part = mutate_task->getFuture().get(); - storage.renameTempPartAndReplace(new_part, nullptr); //FIXME + /// FIXME Transaction: it's too optimistic, better to lock parts before starting transaction + storage.renameTempPartAndReplace(new_part, merge_mutate_entry->txn.get()); storage.updateMutationEntriesErrors(future_part, true, ""); write_part_log({}); @@ -94,6 +95,8 @@ bool MutatePlainMergeTreeTask::executeStep() } catch (...) { + if (merge_mutate_entry->txn) + merge_mutate_entry->txn->onException(); storage.updateMutationEntriesErrors(future_part, false, getCurrentExceptionMessage(false)); write_part_log(ExecutionStatus::fromCurrentException()); return false; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 069c7c6efa0..1d676f2e4a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -58,6 +58,22 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove; } +static MergeTreeTransactionPtr tryGetTransactionForMutation(const MergeTreeMutationEntry & mutation, Poco::Logger * log = nullptr) +{ + assert(!mutation.tid.isEmpty()); + if (mutation.tid.isPrehistoric()) + return {}; + + auto txn = TransactionLog::instance().tryGetRunningTransaction(mutation.tid.getHash()); + if (txn) + return txn; + + if (log) + LOG_WARNING(log, "Cannot find transaction {} which had started mutation {}, probably it finished", mutation.tid, mutation.file_name); + + return {}; +} + StorageMergeTree::StorageMergeTree( const StorageID & table_id_, @@ -280,7 +296,6 @@ void StorageMergeTree::alter( StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); auto maybe_mutation_commands = commands.getMutationCommands(new_metadata, local_context->getSettingsRef().materialize_ttl_after_modify, local_context); - String mutation_file_name; Int64 mutation_version = -1; commands.apply(new_metadata, local_context); @@ -302,13 +317,13 @@ void StorageMergeTree::alter( DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); if (!maybe_mutation_commands.empty()) - mutation_version = startMutation(maybe_mutation_commands, mutation_file_name); + mutation_version = startMutation(maybe_mutation_commands, local_context); } /// Always execute required mutations synchronously, because alters /// should be executed in sequential order. if (!maybe_mutation_commands.empty()) - waitForMutation(mutation_version, mutation_file_name); + waitForMutation(mutation_version); } { @@ -399,24 +414,35 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } -Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String & mutation_file_name) +Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context) { /// Choose any disk, because when we load mutations we search them at each disk /// where storage can be placed. See loadMutations(). auto disk = getStoragePolicy()->getAnyDisk(); + TransactionID current_tid = Tx::PrehistoricTID; + String additional_info; + auto txn = query_context->getCurrentTransaction(); + if (txn) + { + current_tid = txn->tid; + additional_info = fmt::format(" (TID: {}; TIDH: {})", current_tid, current_tid.getHash()); + } + Int64 version; { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid); version = increment.get(); entry.commit(version); - mutation_file_name = entry.file_name; + String mutation_id = entry.file_name; + if (txn) + txn->addMutation(shared_from_this(), mutation_id); bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - LOG_INFO(log, "Added mutation: {}", mutation_file_name); + LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); return version; @@ -462,9 +488,15 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re mutation_wait_event.notify_all(); } -void StorageMergeTree::waitForMutation(Int64 version, const String & file_name) +void StorageMergeTree::waitForMutation(Int64 version) { - LOG_INFO(log, "Waiting mutation: {}", file_name); + waitForMutation(MergeTreeMutationEntry::versionToFileName(version)); +} + +void StorageMergeTree::waitForMutation(const String & mutation_id) +{ + UInt64 version = MergeTreeMutationEntry::parseFileName(mutation_id); + LOG_INFO(log, "Waiting mutation: {}", mutation_id); { auto check = [version, this]() { @@ -480,29 +512,20 @@ void StorageMergeTree::waitForMutation(Int64 version, const String & file_name) /// At least we have our current mutation std::set mutation_ids; - mutation_ids.insert(file_name); + mutation_ids.insert(mutation_id); auto mutation_status = getIncompleteMutationsStatus(version, &mutation_ids); - try - { - checkMutationStatus(mutation_status, mutation_ids); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } + checkMutationStatus(mutation_status, mutation_ids); - LOG_INFO(log, "Mutation {} done", file_name); + LOG_INFO(log, "Mutation {} done", mutation_id); } void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { - String mutation_file_name; - Int64 version = startMutation(commands, mutation_file_name); + Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0) - waitForMutation(version, mutation_file_name); + waitForMutation(version); } std::optional StorageMergeTree::getIncompleteMutationsStatus(Int64 mutation_version, std::set * mutation_ids) const @@ -518,7 +541,9 @@ std::optional StorageMergeTree::getIncompleteMutationsS const auto & mutation_entry = current_mutation_it->second; - auto data_parts = getDataPartsVector(); + auto txn = tryGetTransactionForMutation(mutation_entry, log); + assert(txn || mutation_entry.tid.isPrehistoric()); + auto data_parts = getVisibleDataPartsVector(txn); for (const auto & data_part : data_parts) { Int64 data_version = getUpdatedDataVersion(data_part, lock); @@ -542,6 +567,17 @@ std::optional StorageMergeTree::getIncompleteMutationsS mutation_ids->insert(it->second.file_name); } } + else if (txn) + { + /// Part is locked by concurrent transaction, most likely it will never be mutated + TIDHash part_locked = data_part->versions.maxtid_lock.load(); + if (part_locked && part_locked != mutation_entry.tid.getHash()) + { + result.latest_failed_part = data_part->name; + result.latest_fail_reason = fmt::format("Serialization error: part {} is locked by transaction {}", data_part->name, part_locked); + result.latest_fail_time = time(nullptr); + } + } return result; } @@ -617,6 +653,12 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) if (!to_kill) return CancellationCode::NotFound; + if (auto txn = tryGetTransactionForMutation(*to_kill, log)) + { + LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id); + TransactionLog::instance().rollbackTransaction(txn); + } + getContext()->getMergeList().cancelPartMutations(getStorageID(), {}, to_kill->block_number); to_kill->removeFile(); LOG_TRACE(log, "Cancelled part mutations and removed mutation file {}", mutation_id); @@ -900,11 +942,29 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( continue; } - auto commands = MutationCommands::create(); + TransactionID first_mutation_tid = mutations_begin_it->second.tid; + MergeTreeTransactionPtr txn = tryGetTransactionForMutation(mutations_begin_it->second, log); + /// FIXME Transactions: we should kill mutations, but cannot do it here while holding currently_processing_in_background_mutex + /// TIDs are not persistent, so it cannot happen for now + assert(txn || first_mutation_tid.isPrehistoric()); + if (txn) + { + /// Mutate visible parts only + /// NOTE Do not mutate visible parts in Outdated state, because it does not make sense: + /// mutation will fail anyway due to serialization error. + if (!part->versions.isVisible(*txn)) + continue; + } + + auto commands = MutationCommands::create(); size_t current_ast_elements = 0; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { + /// Do not squash mutation from different transactions to be able to commit/rollback them independently. + if (first_mutation_tid != it->second.tid) + break; + size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -988,7 +1048,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( future_part->type = part->getType(); tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands); + return std::make_shared(future_part, std::move(tagger), commands, txn); } } @@ -1112,54 +1172,52 @@ UInt64 StorageMergeTree::getCurrentMutationVersion( size_t StorageMergeTree::clearOldMutations(bool truncate) { - const auto settings = getSettings(); - if (!truncate && !settings->finished_mutations_to_keep) - return 0; + size_t finished_mutations_to_keep = truncate ? 0 : getSettings()->finished_mutations_to_keep; std::vector mutations_to_delete; { std::unique_lock lock(currently_processing_in_background_mutex); - if (!truncate && current_mutations_by_version.size() <= settings->finished_mutations_to_keep) + if (current_mutations_by_version.size() <= finished_mutations_to_keep) return 0; auto end_it = current_mutations_by_version.end(); auto begin_it = current_mutations_by_version.begin(); size_t to_delete_count = std::distance(begin_it, end_it); - if (!truncate) + if (std::optional min_version = getMinPartDataVersion()) + end_it = current_mutations_by_version.upper_bound(*min_version); + + size_t done_count = std::distance(begin_it, end_it); + if (done_count <= finished_mutations_to_keep) + return 0; + + auto part_versions_with_names = getSortedPartVersionsWithNames(lock); + + for (auto it = begin_it; it != end_it; ++it) { - if (std::optional min_version = getMinPartDataVersion()) - end_it = current_mutations_by_version.upper_bound(*min_version); + const PartVersionWithName needle{static_cast(it->first), ""}; + auto versions_it = std::lower_bound( + part_versions_with_names.begin(), part_versions_with_names.end(), needle); - size_t done_count = std::distance(begin_it, end_it); - if (done_count <= settings->finished_mutations_to_keep) - return 0; - - auto part_versions_with_names = getSortedPartVersionsWithNames(lock); - - for (auto it = begin_it; it != end_it; ++it) + if (versions_it != part_versions_with_names.begin()) { - const PartVersionWithName needle{static_cast(it->first), ""}; - auto versions_it = std::lower_bound( - part_versions_with_names.begin(), part_versions_with_names.end(), needle); - - if (versions_it != part_versions_with_names.begin()) - { - done_count = std::distance(begin_it, it); - break; - } + done_count = std::distance(begin_it, it); + break; } - - if (done_count <= settings->finished_mutations_to_keep) - return 0; - - to_delete_count = done_count - settings->finished_mutations_to_keep; } + if (done_count <= finished_mutations_to_keep) + return 0; + + to_delete_count = done_count - finished_mutations_to_keep; + auto it = begin_it; for (size_t i = 0; i < to_delete_count; ++i) { + const auto & tid = it->second.tid; + if (!tid.isPrehistoric() && !TransactionLog::instance().getCSN(tid)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove mutation {}, because transaction {} is not committed. It's a bug"); mutations_to_delete.push_back(std::move(it->second)); it = current_mutations_by_version.erase(it); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f4fe4e57959..7b805d189f2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -168,9 +168,10 @@ private: /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. - Int64 startMutation(const MutationCommands & commands, String & mutation_file_name); + Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); /// Wait until mutation with version will finish mutation for all parts - void waitForMutation(Int64 version, const String & file_name); + void waitForMutation(Int64 version); + void waitForMutation(const String & mutation_id) override; friend struct CurrentlyMergingPartsTagger; diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference new file mode 100644 index 00000000000..582a42418d6 --- /dev/null +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -0,0 +1,20 @@ +1 10 all_1_1_0_4 +1 30 all_3_3_0_4 +2 1 all_1_1_0 +2 2 all_2_2_0 +Serialization error +INVALID_TRANSACTION +3 1 all_1_1_0 +Serialization error +4 2 all_1_1_0_8 +4 5 all_11_11_0 +4 6 all_7_7_0_8 +5 2 all_1_1_0_8 +5 5 all_11_11_0 +5 6 all_7_7_0_8 +SERIALIZATION_ERROR +6 2 all_1_1_0_12 +6 6 all_7_7_0_12 +7 20 all_1_1_0_14 +7 60 all_7_7_0_14 +7 80 all_13_13_0_14 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh new file mode 100755 index 00000000000..9791e2aa079 --- /dev/null +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-replicated-database +# Looks like server does not listen https port in fasttest +# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./transactions.lib +. "$CURDIR"/transactions.lib + +$CLICKHOUSE_CLIENT -q "drop table if exists mt" +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by tuple()" + +$CLICKHOUSE_CLIENT -q "insert into mt values (1)" + +tx 1 "begin transaction" +tx 2 "begin transaction" +tx 1 "insert into mt values (2)" +tx 2 "insert into mt values (3)" +tx 2 "alter table mt update n=n*10 where 1 settings mutations_sync=1" +tx 2 "select 1, n, _part from mt order by n" +tx 1 "select 2, n, _part from mt order by n" +tx 1 "alter table mt update n=n+1 where 1 settings mutations_sync=1" | grep -Eo "Serialization error" | uniq +tx 1 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 2 "rollback" + + +tx 3 "begin transaction" +tx 3 "select 3, n, _part from mt order by n" +tx 4 "begin transaction" +tx 3 "insert into mt values (2)" +tx 4 "insert into mt values (3)" +tx 4 "alter table mt update n=n*2 where 1" +tx 3 "alter table mt update n=n+42 where 1" +tx 3 "insert into mt values (4)" +tx 4 "insert into mt values (5)" +tx 3 "commit" | grep -Eo "Serialization error" | uniq +tx 4 "commit" + + +tx 5 "begin transaction" +tx 5 "select 4, n, _part from mt order by n" +tx 6 "begin transaction" +tx 6 "alter table mt delete where n%2=1 settings mutations_sync=1" +tx 5 "select 5, n, _part from mt order by n" +tx 5 "alter table mt drop partition id 'all'" | grep -Eo "SERIALIZATION_ERROR" | uniq +tx 6 "select 6, n, _part from mt order by n" +tx 5 "rollback" +tx 6 "insert into mt values (8)" +tx 6 "alter table mt update n=n*10 where 1" +tx 6 "commit" + +tx 8 "begin transaction" +tx 8 "select 7, n, _part from mt order by n" + +$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" From 89b77d30569c6a86d466d95d242ab9d83488774a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 15 Dec 2021 01:36:45 +0300 Subject: [PATCH 020/372] Update 02117_show_create_table_system.reference --- .../0_stateless/02117_show_create_table_system.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 2b391cd292e..563ccc1147c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -33,7 +33,7 @@ CREATE TABLE system.numbers\n(\n `number` UInt64\n)\nENGINE = SystemNumbers() CREATE TABLE system.numbers_mt\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `visible` UInt8,\n `mintid` Tuple(UInt64, UInt64, UUID),\n `maxtid` Tuple(UInt64, UInt64, UUID),\n `mincsn` UInt64,\n `maxcsn` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From e185ad260b5208509afc5f21e95da8ad65585e0a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Dec 2021 21:19:29 +0300 Subject: [PATCH 021/372] fix skipping of some mutations --- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/StorageMergeTree.cpp | 13 ++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 22 ------------------- src/Storages/StorageReplicatedMergeTree.h | 1 - ...02004_invalid_partition_mutation_stuck.sql | 2 +- 6 files changed, 33 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bd3e79b72cc..c31f399522f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3867,6 +3867,27 @@ std::unordered_set MergeTreeData::getPartitionIDsFromQuery(const ASTs & return partition_ids; } +std::set MergeTreeData::getPartitionIdsAffectedByCommands( + const MutationCommands & commands, ContextPtr query_context) const +{ + std::set affected_partition_ids; + + for (const auto & command : commands) + { + if (!command.partition) + { + affected_partition_ids.clear(); + break; + } + + affected_partition_ids.insert( + getPartitionIDFromQuery(command.partition, query_context) + ); + } + + return affected_partition_ids; +} + MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector( const DataPartStates & affordable_states, DataPartStateVector * out_states, bool require_projection_parts) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d5b8901527d..f43bf2cc643 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -706,6 +706,7 @@ public: /// For ATTACH/DETACH/DROP PARTITION. String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const; std::unordered_set getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const; + std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, ContextPtr query_context) const; /// Extracts MergeTreeData of other *MergeTree* storage /// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 791fe7e239b..35df7ce1c58 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -529,6 +529,9 @@ void StorageMergeTree::waitForMutation(const String & mutation_id) void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { + /// Validate partition IDs (if any) before starting mutation + getPartitionIdsAffectedByCommands(commands, query_context); + Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0) @@ -966,6 +969,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto commands = MutationCommands::create(); size_t current_ast_elements = 0; + auto last_mutation_to_apply = mutations_end_it; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutation from different transactions to be able to commit/rollback them independently. @@ -1006,7 +1010,8 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( MergeTreeMutationEntry & entry = it->second; entry.latest_fail_time = time(nullptr); entry.latest_fail_reason = getCurrentExceptionMessage(false); - continue; + /// NOTE we should not skip mutations, because exception may be retryable (e.g. MEMORY_LIMIT_EXCEEDED) + break; } } @@ -1015,8 +1020,10 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( current_ast_elements += commands_size; commands->insert(commands->end(), it->second.commands.begin(), it->second.commands.end()); + last_mutation_to_apply = it; } + assert(commands->empty() == (last_mutation_to_apply == mutations_end_it)); if (!commands->empty()) { bool is_partition_affected = false; @@ -1041,13 +1048,13 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( /// Shall not create a new part, but will do that later if mutation with higher version appear. /// This is needed in order to not produce excessive mutations of non-related parts. auto block_range = std::make_pair(part->info.min_block, part->info.max_block); - updated_version_by_block_range[block_range] = current_mutations_by_version.rbegin()->first; + updated_version_by_block_range[block_range] = last_mutation_to_apply->first; were_some_mutations_for_some_parts_skipped = true; continue; } auto new_part_info = part->info; - new_part_info.mutation = current_mutations_by_version.rbegin()->first; + new_part_info.mutation = last_mutation_to_apply->first; future_part->parts.push_back(part); future_part->part_info = new_part_info; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2c891436234..b8653d72646 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4533,28 +4533,6 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer } -std::set StorageReplicatedMergeTree::getPartitionIdsAffectedByCommands( - const MutationCommands & commands, ContextPtr query_context) const -{ - std::set affected_partition_ids; - - for (const auto & command : commands) - { - if (!command.partition) - { - affected_partition_ids.clear(); - break; - } - - affected_partition_ids.insert( - getPartitionIDFromQuery(command.partition, query_context) - ); - } - - return affected_partition_ids; -} - - PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bcd364df30e..6861d89f070 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -717,7 +717,6 @@ private: std::unique_ptr getDefaultSettings() const override; - std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, ContextPtr query_context) const; PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const; diff --git a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql index 481a5565095..71c8b9af652 100644 --- a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql +++ b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql @@ -28,6 +28,6 @@ PARTITION BY p ORDER BY t SETTINGS number_of_free_entries_in_pool_to_execute_mutation=0; INSERT INTO data VALUES (1, now()); -ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 341 } +ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 248 } ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '1'; ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '2'; From f330ac31c3ce68a405c37485b80cb5ca0d30a500 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Dec 2021 00:49:06 +0300 Subject: [PATCH 022/372] fix test --- src/Interpreters/MergeTreeTransaction.cpp | 2 +- .../TransactionVersionMetadata.cpp | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 17 ++++++---- .../01168_mutations_isolation.reference | 14 +++++++++ .../0_stateless/01168_mutations_isolation.sh | 31 +++++++++++++++++-- 5 files changed, 57 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index b2f1d8ae150..91e4b51ea07 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -85,7 +85,7 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP { CSN c = csn.load(); if (c == Tx::RolledBackCSN) - throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); + throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled");//FIXME else if (c != Tx::UnknownCSN) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", c); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index c8e6b9dcfba..59d3b1ef4b9 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -38,6 +38,7 @@ TransactionID VersionMetadata::getMaxTID() const void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context) { + //LOG_TRACE(&Poco::Logger::get("WTF"), "Trying to lock maxtid by {}: {}\n{}", tid, error_context, StackTrace().toString()); TIDHash locked_by = 0; if (tryLockMaxTID(tid, &locked_by)) return; @@ -75,6 +76,7 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_ void VersionMetadata::unlockMaxTID(const TransactionID & tid) { + //LOG_TRACE(&Poco::Logger::get("WTF"), "Unlocking maxtid by {}", tid); assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash locked_by = maxtid_lock.load(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c31f399522f..31d4d5a8098 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1663,11 +1663,15 @@ size_t MergeTreeData::clearEmptyParts() auto parts = getDataPartsVector(); for (const auto & part : parts) { - if (part->rows_count == 0) - { - dropPartNoWaitNoThrow(part->name); - ++cleared_count; - } + if (part->rows_count != 0) + continue; + + /// Do not drop empty part if it may be visible for some transaction (otherwise it may cause conflicts) + if (!part->versions.canBeRemoved(TransactionLog::instance().getOldestSnapshot())) + continue; + + dropPartNoWaitNoThrow(part->name); + ++cleared_count; } return cleared_count; } @@ -2734,7 +2738,8 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( void MergeTreeData::restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock) { auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); //-V1018 - assert(part->getState() != DataPartState::Committed); + if (part->getState() == DataPartState::Committed) + return; addPartContributionToColumnAndSecondaryIndexSizes(part); addPartContributionToDataVolume(part); modifyPartState(part, DataPartState::Committed); diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index 582a42418d6..56e7264b174 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -16,5 +16,19 @@ SERIALIZATION_ERROR 6 2 all_1_1_0_12 6 6 all_7_7_0_12 7 20 all_1_1_0_14 +7 40 all_15_15_0 7 60 all_7_7_0_14 7 80 all_13_13_0_14 +8 20 all_1_15_1_14 +8 40 all_1_15_1_14 +8 60 all_1_15_1_14 +8 80 all_1_15_1_14 +INVALID_TRANSACTION +9 21 all_1_15_1_18 +9 41 all_1_15_1_18 +9 61 all_1_15_1_18 +9 81 all_1_15_1_18 +10 22 all_1_15_1_19 +10 42 all_1_15_1_19 +10 62 all_1_15_1_19 +10 82 all_1_15_1_19 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 9791e2aa079..d4d76e91ee0 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -49,9 +49,36 @@ tx 6 "select 6, n, _part from mt orde tx 5 "rollback" tx 6 "insert into mt values (8)" tx 6 "alter table mt update n=n*10 where 1" +tx 6 "insert into mt values (40)" tx 6 "commit" -tx 8 "begin transaction" -tx 8 "select 7, n, _part from mt order by n" + +tx 7 "begin transaction" +tx 7 "select 7, n, _part from mt order by n" +tx 8 "begin transaction" +tx 8 "alter table mt update n = 0 where 1" +$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_16.txt' format Null" +tx 7 "optimize table mt final" +tx 7 "select 8, n, _part from mt order by n" +tx 8 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 8 "rollback" +tx 10 "begin transaction" +tx 10 "alter table mt update n = 0 where 1" +tx 7 "alter table mt update n=n+1 where 1" +tx 10 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 10 "rollback" +tx 7 "commit" + + +tx 11 "begin transaction" +tx 11 "select 9, n, _part from mt order by n" +tx 12 "begin transaction" +tx 11 "alter table mt update n=n+1 where 1" +tx 12 "alter table mt update n=n+1 where 1" +tx 11 "commit" >/dev/null +tx 12 "commit" >/dev/null + +tx 11 "begin transaction" +tx 11 "select 10, n, _part from mt order by n" $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" From a15444031ca9d12d76ebe0aa10fd26b402b68351 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 20 Dec 2021 21:53:05 +0300 Subject: [PATCH 023/372] add test --- src/Interpreters/MergeTreeTransaction.cpp | 3 - src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 1 - .../01167_isolation_hermitage.reference | 59 +++++++ .../0_stateless/01167_isolation_hermitage.sh | 166 ++++++++++++++++++ .../01168_mutations_isolation.reference | 61 ++++--- .../0_stateless/01168_mutations_isolation.sh | 40 +++-- .../01170_alter_partition_isolation.reference | 28 +-- tests/queries/0_stateless/transactions.lib | 40 ++++- 10 files changed, 332 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/01167_isolation_hermitage.reference create mode 100755 tests/queries/0_stateless/01167_isolation_hermitage.sh diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 91e4b51ea07..662edd2cac8 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -134,10 +134,7 @@ bool MergeTreeTransaction::rollback() noexcept bool need_rollback = csn.compare_exchange_strong(expected, Tx::RolledBackCSN); if (!need_rollback) - { - /// TODO add assertions for the case when this method called from background operation return false; - } for (const auto & table_and_mutation : mutations) table_and_mutation.first->killMutation(table_and_mutation.second); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2fab4431134..6bfbaa0809c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1684,8 +1684,8 @@ size_t MergeTreeData::clearEmptyParts() if (part->rows_count != 0) continue; - /// Do not drop empty part if it may be visible for some transaction (otherwise it may cause conflicts) - if (!part->versions.canBeRemoved(TransactionLog::instance().getOldestSnapshot())) + /// Do not try to drop empty part if it's locked by some transaction and do not try to drop uncommitted parts. + if (part->versions.maxtid_lock.load() || !part->versions.isVisible(TransactionLog::instance().getLatestSnapshot())) continue; dropPartNoWaitNoThrow(part->name); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 3293f200f3e..38b9a5790e6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -538,7 +538,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer Int64 version = startMutation(commands, query_context); - if (query_context->getSettingsRef().mutations_sync > 0) + if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) waitForMutation(version); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d960d4ceb17..858b550c38e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -245,7 +245,6 @@ private: std::unique_ptr getDefaultSettings() const override; - friend class MergeTreeProjectionBlockOutputStream; friend class MergeTreeSink; friend class MergeTreeData; friend class MergePlainMergeTreeTask; diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.reference b/tests/queries/0_stateless/01167_isolation_hermitage.reference new file mode 100644 index 00000000000..4488809f3ed --- /dev/null +++ b/tests/queries/0_stateless/01167_isolation_hermitage.reference @@ -0,0 +1,59 @@ +Serialization error +INVALID_TRANSACTION +INVALID_TRANSACTION +1 1 11 +1 2 21 +tx4 2 1 10 +tx4 2 2 20 +tx4 3 1 10 +tx4 3 2 20 +4 1 10 +4 2 20 +tx6 5 1 10 +tx6 5 2 20 +tx6 6 1 10 +tx6 6 2 20 +7 1 11 +7 2 20 +Serialization error +tx7 8 1 11 +tx7 8 2 20 +INVALID_TRANSACTION +INVALID_TRANSACTION +10 1 11 +10 2 20 +Serialization error +tx11 11 1 10 +tx11 11 2 20 +INVALID_TRANSACTION +tx11 12 1 10 +tx11 12 2 20 +INVALID_TRANSACTION +13 1 11 +13 2 19 +16 1 10 +16 2 20 +16 3 30 +Serialization error +INVALID_TRANSACTION +INVALID_TRANSACTION +18 1 20 +18 2 30 +tx16 19 1 10 +tx16 19 2 20 +tx17 20 1 10 +tx17 20 2 20 +Serialization error +INVALID_TRANSACTION +21 1 11 +21 2 20 +tx18 22 1 10 +tx19 23 1 10 +tx19 24 2 20 +tx18 25 2 20 +26 1 12 +26 2 18 +29 1 10 +29 2 20 +29 3 30 +29 4 42 diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.sh b/tests/queries/0_stateless/01167_isolation_hermitage.sh new file mode 100755 index 00000000000..b08ac405ee8 --- /dev/null +++ b/tests/queries/0_stateless/01167_isolation_hermitage.sh @@ -0,0 +1,166 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-replicated-database +# Looks like server does not listen https port in fasttest +# FIXME Replicated database executes ALTERs in separate context, so transaction info is lost + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./transactions.lib +. "$CURDIR"/transactions.lib +set -e + +# https://github.com/ept/hermitage + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (id int, value int) engine=MergeTree order by id" + +function reset_table() +{ + $CLICKHOUSE_CLIENT -q "truncate table test;" + $CLICKHOUSE_CLIENT -q "insert into test (id, value) values (1, 10);" + $CLICKHOUSE_CLIENT -q "insert into test (id, value) values (2, 20);" +} + +# TODO update test after implementing Read Committed + +# G0 +reset_table +tx 1 "begin transaction" +tx 2 "begin transaction" +tx 1 "alter table test update value=11 where id=1" +tx 2 "alter table test update value=12 where id=1" | grep -Eo "Serialization error" | uniq +tx 1 "alter table test update value=21 where id=2" +tx 1 "commit" +tx 2 "alter table test update value=22 where id=2" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 2 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 2 "rollback" +$CLICKHOUSE_CLIENT -q "select 1, * from test order by id" + +# G1a +reset_table +tx_async 3 "begin transaction" +tx_async 4 "begin transaction" +tx_async 3 "alter table test update value=101 where id=1" +tx_async 4 "select 2, * from test order by id" +tx_async 3 "alter table test update value=11 where id=1" +tx_async 3 "rollback" +tx_async 4 "select 3, * from test order by id" +tx_async 4 "commit" +tx_wait 3 +tx_wait 4 +$CLICKHOUSE_CLIENT -q "select 4, * from test order by id" + +# G1b +reset_table +tx_async 5 "begin transaction" +tx_async 6 "begin transaction" +tx_async 5 "alter table test update value=101 where id=1" +tx_async 6 "select 5, * from test order by id" +tx_async 5 "alter table test update value=11 where id=1" +tx_async 5 "commit" +tx_async 6 "select 6, * from test order by id" +tx_async 6 "commit" +tx_wait 5 +tx_wait 6 +$CLICKHOUSE_CLIENT -q "select 7, * from test order by id" + +# G1c +# NOTE both transactions will succeed if we implement skipping of unaffected partitions/parts +reset_table +tx 7 "begin transaction" +tx 8 "begin transaction" +tx 7 "alter table test update value=11 where id=1" +tx 8 "alter table test update value=22 where id=2" | grep -Eo "Serialization error" | uniq +tx 7 "select 8, * from test order by id" +tx 8 "select 9, * from test order by id" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 7 "commit" +tx 8 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 8 "rollback" +$CLICKHOUSE_CLIENT -q "select 10, * from test order by id" + +# OTV +reset_table +tx 9 "begin transaction" +tx 10 "begin transaction" +tx 11 "begin transaction" +tx 9 "alter table test update value = 11 where id = 1" +tx 9 "alter table test update value = 19 where id = 2" +tx 10 "alter table test update value = 12 where id = 1" | grep -Eo "Serialization error" | uniq +tx 9 "commit" +tx 11 "select 11, * from test order by id" +tx 10 "alter table test update value = 18 where id = 2" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 11 "select 12, * from test order by id" +tx 10 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 10 "rollback" +tx 11 "commit" +$CLICKHOUSE_CLIENT -q "select 13, * from test order by id" + +# PMP +reset_table +tx_async 12 "begin transaction" +tx_async 13 "begin transaction" +tx_async 12 "select 14, * from test where value = 30" +tx_async 13 "insert into test (id, value) values (3, 30)" +tx_async 13 "commit" +tx_async 12 "select 15, * from test where value = 30" +tx_async 12 "commit" +tx_wait 12 +tx_wait 13 +$CLICKHOUSE_CLIENT -q "select 16, * from test order by id" + +# PMP write +reset_table +tx 14 "begin transaction" +tx 15 "begin transaction" +tx 14 "alter table test update value = value + 10 where 1" +tx 15 "alter table test delete where value = 20" | grep -Eo "Serialization error" | uniq +tx 14 "commit" +tx 15 "select 17, * from test order by id" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 15 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 15 "rollback" +$CLICKHOUSE_CLIENT -q "select 18, * from test order by id" + +# P4 +reset_table +tx 16 "begin transaction" +tx 17 "begin transaction" +tx 16 "select 19, * from test order by id" +tx 17 "select 20, * from test order by id" +tx 16 "alter table test update value = 11 where id = 1" +tx 17 "alter table test update value = 11 where id = 1" | grep -Eo "Serialization error" | uniq +tx 16 "commit" +tx 17 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq +tx 17 "rollback" +$CLICKHOUSE_CLIENT -q "select 21, * from test order by id" + +# G-single +reset_table +tx_async 18 "begin transaction" +tx_async 19 "begin transaction" +tx_async 18 "select 22, * from test where id = 1" +tx_async 19 "select 23, * from test where id = 1" +tx_async 19 "select 24, * from test where id = 2" +tx_async 19 "alter table test update value = 12 where id = 1" +tx_async 19 "alter table test update value = 18 where id = 2" +tx_async 19 "commit" +tx_async 18 "select 25, * from test where id = 2" +tx_async 18 "commit" +tx_wait 18 +tx_wait 19 +$CLICKHOUSE_CLIENT -q "select 26, * from test order by id" + +# G2 +reset_table +tx_async 20 "begin transaction" +tx_async 21 "begin transaction" +tx_async 20 "select 27, * from test where value % 3 = 0" +tx_async 21 "select 28, * from test where value % 3 = 0" +tx_async 20 "insert into test (id, value) values (3, 30)" +tx_async 21 "insert into test (id, value) values (4, 42)" +tx_async 20 "commit" +tx_async 21 "commit" +tx_wait 20 +tx_wait 21 +$CLICKHOUSE_CLIENT -q "select 29, * from test order by id" + diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index 56e7264b174..f7a1a707cfe 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -1,34 +1,37 @@ -1 10 all_1_1_0_4 -1 30 all_3_3_0_4 -2 1 all_1_1_0 -2 2 all_2_2_0 +tx2 1 10 all_1_1_0_4 +tx2 1 30 all_3_3_0_4 +tx1 2 1 all_1_1_0 +tx1 2 2 all_2_2_0 Serialization error INVALID_TRANSACTION -3 1 all_1_1_0 +tx3 3 1 all_1_1_0 Serialization error -4 2 all_1_1_0_8 -4 5 all_11_11_0 -4 6 all_7_7_0_8 -5 2 all_1_1_0_8 -5 5 all_11_11_0 -5 6 all_7_7_0_8 +INVALID_TRANSACTION +INVALID_TRANSACTION +tx5 4 2 all_1_1_0_8 +tx5 4 5 all_10_10_0 +tx5 4 6 all_7_7_0_8 +tx5 5 2 all_1_1_0_8 +tx5 5 5 all_10_10_0 +tx5 5 6 all_7_7_0_8 SERIALIZATION_ERROR -6 2 all_1_1_0_12 -6 6 all_7_7_0_12 -7 20 all_1_1_0_14 -7 40 all_15_15_0 -7 60 all_7_7_0_14 -7 80 all_13_13_0_14 -8 20 all_1_15_1_14 -8 40 all_1_15_1_14 -8 60 all_1_15_1_14 -8 80 all_1_15_1_14 +tx6 6 2 all_1_1_0_11 +tx6 6 6 all_7_7_0_11 +tx7 7 20 all_1_1_0_13 +tx7 7 40 all_14_14_0 +tx7 7 60 all_7_7_0_13 +tx7 7 80 all_12_12_0_13 +tx7 8 20 all_1_14_1_13 +tx7 8 40 all_1_14_1_13 +tx7 8 60 all_1_14_1_13 +tx7 8 80 all_1_14_1_13 +Serialization error INVALID_TRANSACTION -9 21 all_1_15_1_18 -9 41 all_1_15_1_18 -9 61 all_1_15_1_18 -9 81 all_1_15_1_18 -10 22 all_1_15_1_19 -10 42 all_1_15_1_19 -10 62 all_1_15_1_19 -10 82 all_1_15_1_19 +tx11 9 21 all_1_14_1_17 +tx11 9 41 all_1_14_1_17 +tx11 9 61 all_1_14_1_17 +tx11 9 81 all_1_14_1_17 +tx13 10 22 all_1_14_1_18 +tx13 10 42 all_1_14_1_18 +tx13 10 62 all_1_14_1_18 +tx13 10 82 all_1_14_1_18 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index d4d76e91ee0..e6049e50131 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -18,10 +18,10 @@ tx 1 "begin transaction" tx 2 "begin transaction" tx 1 "insert into mt values (2)" tx 2 "insert into mt values (3)" -tx 2 "alter table mt update n=n*10 where 1 settings mutations_sync=1" +tx 2 "alter table mt update n=n*10 where 1" tx 2 "select 1, n, _part from mt order by n" tx 1 "select 2, n, _part from mt order by n" -tx 1 "alter table mt update n=n+1 where 1 settings mutations_sync=1" | grep -Eo "Serialization error" | uniq +tx 1 "alter table mt update n=n+1 where 1" | grep -Eo "Serialization error" | uniq tx 1 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq tx 2 "rollback" @@ -32,17 +32,17 @@ tx 4 "begin transaction" tx 3 "insert into mt values (2)" tx 4 "insert into mt values (3)" tx 4 "alter table mt update n=n*2 where 1" -tx 3 "alter table mt update n=n+42 where 1" -tx 3 "insert into mt values (4)" +tx 3 "alter table mt update n=n+42 where 1" | grep -Eo "Serialization error" | uniq +tx 3 "insert into mt values (4)" | grep -Eo "INVALID_TRANSACTION" | uniq tx 4 "insert into mt values (5)" -tx 3 "commit" | grep -Eo "Serialization error" | uniq +tx 3 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq tx 4 "commit" tx 5 "begin transaction" tx 5 "select 4, n, _part from mt order by n" tx 6 "begin transaction" -tx 6 "alter table mt delete where n%2=1 settings mutations_sync=1" +tx 6 "alter table mt delete where n%2=1" tx 5 "select 5, n, _part from mt order by n" tx 5 "alter table mt drop partition id 'all'" | grep -Eo "SERIALIZATION_ERROR" | uniq tx 6 "select 6, n, _part from mt order by n" @@ -56,29 +56,31 @@ tx 6 "commit" tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" -tx 8 "alter table mt update n = 0 where 1" -$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_16.txt' format Null" +tx 8 "alter table mt update n = 0 where 1" & +$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" +wait tx 7 "optimize table mt final" tx 7 "select 8, n, _part from mt order by n" -tx 8 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq tx 8 "rollback" tx 10 "begin transaction" -tx 10 "alter table mt update n = 0 where 1" +tx 10 "alter table mt update n = 0 where 1" | grep -Eo "Serialization error" | uniq tx 7 "alter table mt update n=n+1 where 1" tx 10 "commit" | grep -Eo "INVALID_TRANSACTION" | uniq tx 10 "rollback" tx 7 "commit" -tx 11 "begin transaction" -tx 11 "select 9, n, _part from mt order by n" -tx 12 "begin transaction" -tx 11 "alter table mt update n=n+1 where 1" -tx 12 "alter table mt update n=n+1 where 1" -tx 11 "commit" >/dev/null -tx 12 "commit" >/dev/null +tx_async 11 "begin transaction" +tx_async 11 "select 9, n, _part from mt order by n" +tx_async 12 "begin transaction" +tx_async 11 "alter table mt update n=n+1 where 1" >/dev/null & +tx_async 12 "alter table mt update n=n+1 where 1" >/dev/null +tx_async 11 "commit" >/dev/null +tx_async 12 "commit" >/dev/null +tx_wait 11 +tx_wait 12 -tx 11 "begin transaction" -tx 11 "select 10, n, _part from mt order by n" +tx 13 "begin transaction" +tx 13 "select 10, n, _part from mt order by n" $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.reference b/tests/queries/0_stateless/01170_alter_partition_isolation.reference index fc772355a57..f384fc748d4 100644 --- a/tests/queries/0_stateless/01170_alter_partition_isolation.reference +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.reference @@ -1,23 +1,23 @@ -1 1 -2 3 -3 2 -3 4 -4 3 +tx1 1 1 +tx1 2 3 +tx2 3 2 +tx2 3 4 +tx1 4 3 5 3 5 5 -6 3 -6 5 -6 6 -7 8 -8 3 -8 5 -8 7 -8 9 +tx4 6 3 +tx4 6 5 +tx4 6 6 +tx4 7 8 +tx3 8 3 +tx3 8 5 +tx3 8 7 +tx3 8 9 SERIALIZATION_ERROR INVALID_TRANSACTION -9 8 +tx4 9 8 10 8 diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index 2d3eafc784a..be8745e68a5 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -1,14 +1,48 @@ #!/usr/bin/env bash +# Useful to run function tx() { tx_num=$1 query=$2 + session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" + query_id="${session}_${RANDOM}" url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" - session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_$tx_num" - url="${url_without_session}session_id=$session&database=$CLICKHOUSE_DATABASE" + url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" - ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "$query" + ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" } +function tx_wait() { + tx_num=$1 + + session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" + + # wait for previous query in transaction + count=0 + while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '$session%'") -gt 0 ]]; do + sleep 0.5 + count=$((count+1)) + if [ "$count" -gt 120 ]; then + echo "timeout while waiting for $tx_num" + break + fi + done; +} + +function tx_async() +{ + tx_num=$1 + query=$2 + + tx_wait $tx_num + + session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" + query_id="${session}_${RANDOM}" + url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" + url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" + + # run query asynchronously + ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" & +} From df12fdf612e65bc1dbc6fc42fc9201c326e73ace Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Dec 2021 19:34:02 +0300 Subject: [PATCH 024/372] fix tests --- src/Storages/StorageMergeTree.cpp | 5 ++-- .../0_stateless/01168_mutations_isolation.sh | 2 +- tests/queries/0_stateless/transactions.lib | 23 ++++++++++++++++++- 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 38b9a5790e6..5a41b81252e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1218,7 +1218,7 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) auto versions_it = std::lower_bound( part_versions_with_names.begin(), part_versions_with_names.end(), needle); - if (versions_it != part_versions_with_names.begin()) + if (versions_it != part_versions_with_names.begin() || !it->second.tid.isPrehistoric()) { done_count = std::distance(begin_it, it); break; @@ -1235,7 +1235,8 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) { const auto & tid = it->second.tid; if (!tid.isPrehistoric() && !TransactionLog::instance().getCSN(tid)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove mutation {}, because transaction {} is not committed. It's a bug"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove mutation {}, because transaction {} is not committed. It's a bug", + it->first, tid); mutations_to_delete.push_back(std::move(it->second)); it = current_mutations_by_version.erase(it); } diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index e6049e50131..e034b467358 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -56,7 +56,7 @@ tx 6 "commit" tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" -tx 8 "alter table mt update n = 0 where 1" & +tx 8 "alter table mt update n = 0 where 1" >/dev/null & $CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" wait tx 7 "optimize table mt final" diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index be8745e68a5..d4d349d8827 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -1,6 +1,8 @@ #!/usr/bin/env bash -# Useful to run +# shellcheck disable=SC2015 + +# Useful to run queries in parallel sessions function tx() { tx_num=$1 @@ -14,12 +16,24 @@ function tx() ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" } +# Waits for the last query in session to finish function tx_wait() { tx_num=$1 session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" + # try get pid of previous query + query_pid="" + tmp_file_name="${CLICKHOUSE_TMP}/tmp_tx_${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}" + query_id_and_pid=$(grep -F "$session" "$tmp_file_name" 2>/dev/null | tail -1) ||: + read -r query_id query_pid <<< "$query_id_and_pid" ||: + # wait for previous query in transaction + if [ -n "$query_pid" ]; then + timeout 5 tail --pid=$query_pid -f /dev/null && return ||: + fi + + # there is no pid (or maybe we got wrong one), so wait using system.processes (it's less reliable) count=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '$session%'") -gt 0 ]]; do sleep 0.5 @@ -31,6 +45,7 @@ function tx_wait() { done; } +# Wait for previous query in session to finish, starts new one asynchronously function tx_async() { tx_num=$1 @@ -43,6 +58,12 @@ function tx_async() url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" + # We cannot be sure that query will actually start execution and appear in system.processes before the next call to tx_wait + # Also we cannot use global map in bash to store last query_id for each tx_num, so we use tmp file... + tmp_file_name="${CLICKHOUSE_TMP}/tmp_tx_${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}" + # run query asynchronously ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" & + query_pid=$! + echo -e "$query_id\t$query_pid" >> "$tmp_file_name" } From b54178e7232b9d141b9149c8433864f7bac23122 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Dec 2021 22:02:27 +0300 Subject: [PATCH 025/372] fix tests --- src/Storages/MergeTree/MergeTreeData.cpp | 1 + .../0_stateless/01167_isolation_hermitage.sh | 4 ++-- .../01168_mutations_isolation.reference | 24 +++++++++---------- .../0_stateless/01168_mutations_isolation.sh | 8 ++++--- tests/queries/0_stateless/transactions.lib | 9 +++++++ 5 files changed, 29 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6bfbaa0809c..43b17f3c706 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1688,6 +1688,7 @@ size_t MergeTreeData::clearEmptyParts() if (part->versions.maxtid_lock.load() || !part->versions.isVisible(TransactionLog::instance().getLatestSnapshot())) continue; + LOG_TRACE(log, "Will drop empty part {}", part->name); dropPartNoWaitNoThrow(part->name); ++cleared_count; } diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.sh b/tests/queries/0_stateless/01167_isolation_hermitage.sh index b08ac405ee8..09b34694da6 100755 --- a/tests/queries/0_stateless/01167_isolation_hermitage.sh +++ b/tests/queries/0_stateless/01167_isolation_hermitage.sh @@ -138,7 +138,7 @@ $CLICKHOUSE_CLIENT -q "select 21, * from test order by id" reset_table tx_async 18 "begin transaction" tx_async 19 "begin transaction" -tx_async 18 "select 22, * from test where id = 1" +tx_sync 18 "select 22, * from test where id = 1" tx_async 19 "select 23, * from test where id = 1" tx_async 19 "select 24, * from test where id = 2" tx_async 19 "alter table test update value = 12 where id = 1" @@ -154,7 +154,7 @@ $CLICKHOUSE_CLIENT -q "select 26, * from test order by id" reset_table tx_async 20 "begin transaction" tx_async 21 "begin transaction" -tx_async 20 "select 27, * from test where value % 3 = 0" +tx_sync 20 "select 27, * from test where value % 3 = 0" tx_async 21 "select 28, * from test where value % 3 = 0" tx_async 20 "insert into test (id, value) values (3, 30)" tx_async 21 "insert into test (id, value) values (4, 42)" diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index f7a1a707cfe..9daf4ac4c2d 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -21,17 +21,17 @@ tx7 7 20 all_1_1_0_13 tx7 7 40 all_14_14_0 tx7 7 60 all_7_7_0_13 tx7 7 80 all_12_12_0_13 -tx7 8 20 all_1_14_1_13 -tx7 8 40 all_1_14_1_13 -tx7 8 60 all_1_14_1_13 -tx7 8 80 all_1_14_1_13 +tx7 8 20 all_19_19_0 +tx7 8 40 all_17_17_0 +tx7 8 60 all_18_18_0 +tx7 8 80 all_16_16_0 Serialization error INVALID_TRANSACTION -tx11 9 21 all_1_14_1_17 -tx11 9 41 all_1_14_1_17 -tx11 9 61 all_1_14_1_17 -tx11 9 81 all_1_14_1_17 -tx13 10 22 all_1_14_1_18 -tx13 10 42 all_1_14_1_18 -tx13 10 62 all_1_14_1_18 -tx13 10 82 all_1_14_1_18 +tx11 9 21 all_19_19_0_21 +tx11 9 41 all_17_17_0_21 +tx11 9 61 all_18_18_0_21 +tx11 9 81 all_16_16_0_21 +tx13 10 22 all_19_19_0_22 +tx13 10 42 all_17_17_0_22 +tx13 10 62 all_18_18_0_22 +tx13 10 82 all_16_16_0_22 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index e034b467358..7b56de20c8e 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -43,6 +43,7 @@ tx 5 "begin transaction" tx 5 "select 4, n, _part from mt order by n" tx 6 "begin transaction" tx 6 "alter table mt delete where n%2=1" +tx 6 "alter table mt drop part 'all_10_10_0_11'" tx 5 "select 5, n, _part from mt order by n" tx 5 "alter table mt drop partition id 'all'" | grep -Eo "SERIALIZATION_ERROR" | uniq tx 6 "select 6, n, _part from mt order by n" @@ -56,10 +57,11 @@ tx 6 "commit" tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" -tx 8 "alter table mt update n = 0 where 1" >/dev/null & +tx_async 8 "alter table mt update n = 0 where 1" >/dev/null $CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" -wait -tx 7 "optimize table mt final" +tx 7 "alter table mt detach partition id 'all'" +tx_wait 8 +tx 7 "alter table mt attach partition id 'all'" tx 7 "select 8, n, _part from mt order by n" tx 8 "rollback" tx 10 "begin transaction" diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index d4d349d8827..33b884ba6c8 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -67,3 +67,12 @@ function tx_async() query_pid=$! echo -e "$query_id\t$query_pid" >> "$tmp_file_name" } + +# Wait for previous query in session to finish, execute the next one synchronously +function tx_sync() +{ + tx_num=$1 + query=$2 + tx_wait $tx_num + tx $tx_num $query +} From 0fc8c259c79c73c1dde50b7677257cbcb22698bc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Dec 2021 16:14:17 +0300 Subject: [PATCH 026/372] fix tests --- .../01168_mutations_isolation.reference | 24 +++++++++---------- .../0_stateless/01168_mutations_isolation.sh | 5 ++-- tests/queries/0_stateless/transactions.lib | 6 ++--- 3 files changed, 17 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index 9daf4ac4c2d..f7a1a707cfe 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -21,17 +21,17 @@ tx7 7 20 all_1_1_0_13 tx7 7 40 all_14_14_0 tx7 7 60 all_7_7_0_13 tx7 7 80 all_12_12_0_13 -tx7 8 20 all_19_19_0 -tx7 8 40 all_17_17_0 -tx7 8 60 all_18_18_0 -tx7 8 80 all_16_16_0 +tx7 8 20 all_1_14_1_13 +tx7 8 40 all_1_14_1_13 +tx7 8 60 all_1_14_1_13 +tx7 8 80 all_1_14_1_13 Serialization error INVALID_TRANSACTION -tx11 9 21 all_19_19_0_21 -tx11 9 41 all_17_17_0_21 -tx11 9 61 all_18_18_0_21 -tx11 9 81 all_16_16_0_21 -tx13 10 22 all_19_19_0_22 -tx13 10 42 all_17_17_0_22 -tx13 10 62 all_18_18_0_22 -tx13 10 82 all_16_16_0_22 +tx11 9 21 all_1_14_1_17 +tx11 9 41 all_1_14_1_17 +tx11 9 61 all_1_14_1_17 +tx11 9 81 all_1_14_1_17 +tx13 10 22 all_1_14_1_18 +tx13 10 42 all_1_14_1_18 +tx13 10 62 all_1_14_1_18 +tx13 10 82 all_1_14_1_18 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 7b56de20c8e..9a829d7a5e4 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -57,11 +57,10 @@ tx 6 "commit" tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" -tx_async 8 "alter table mt update n = 0 where 1" >/dev/null +tx_async 8 "alter table mt update n = 0 where 1" >/dev/null $CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" -tx 7 "alter table mt detach partition id 'all'" tx_wait 8 -tx 7 "alter table mt attach partition id 'all'" +tx 7 "optimize table mt final" tx 7 "select 8, n, _part from mt order by n" tx 8 "rollback" tx 10 "begin transaction" diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index 33b884ba6c8..521c56754bc 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -51,7 +51,7 @@ function tx_async() tx_num=$1 query=$2 - tx_wait $tx_num + tx_wait "$tx_num" session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" query_id="${session}_${RANDOM}" @@ -73,6 +73,6 @@ function tx_sync() { tx_num=$1 query=$2 - tx_wait $tx_num - tx $tx_num $query + tx_wait "$tx_num" + tx "$tx_num" "$query" } From f0b3c8121284ae3ca8e5fd079cc792810b6f7fc1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Dec 2021 14:23:35 +0300 Subject: [PATCH 027/372] add log in zookeeper --- src/Common/TransactionID.h | 2 +- src/Interpreters/Context.cpp | 2 + src/Interpreters/Context.h | 3 + src/Interpreters/TransactionLog.cpp | 210 +++++++++++++++++- src/Interpreters/TransactionLog.h | 27 ++- .../0_stateless/01168_mutations_isolation.sh | 3 +- 6 files changed, 232 insertions(+), 15 deletions(-) diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 9037601ffad..991dab66e71 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -22,7 +22,7 @@ namespace Tx const CSN UnknownCSN = 0; const CSN PrehistoricCSN = 1; const CSN CommittingCSN = 2; /// TODO do we really need it? - const CSN MaxReservedCSN = 16; + const CSN MaxReservedCSN = 2; const LocalTID PrehistoricLocalTID = 1; const LocalTID MaxReservedLocalTID = 16; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 24a5b220fb1..2fca325917f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -435,6 +435,8 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) void Context::initGlobal() { + assert(!global_context_instance); + global_context_instance = shared_from_this(); DatabaseCatalog::init(shared_from_this()); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 69ea9353b54..b3fdc8fa8e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -303,6 +303,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. @@ -629,6 +630,8 @@ public: ContextMutablePtr getGlobalContext() const; + static ContextPtr getGlobalContextInstance() { return global_context_instance; } + bool hasGlobalContext() const { return !global_context.expired(); } bool isGlobalContext() const { diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index f084c9dbc3c..94cb8cc1827 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -1,6 +1,11 @@ #include #include +#include +#include +#include +#include #include +#include #include #include @@ -21,11 +26,188 @@ TransactionLog & TransactionLog::instance() TransactionLog::TransactionLog() : log(&Poco::Logger::get("TransactionLog")) { - latest_snapshot = Tx::MaxReservedCSN; - csn_counter = Tx::MaxReservedCSN; + global_context = Context::getGlobalContextInstance(); + zookeeper_path = "/test/clickhouse/txn_log"; + + loadLogFromZooKeeper(); + + updating_thread = ThreadFromGlobalPool(&TransactionLog::runUpdatingThread, this); +} + +TransactionLog::~TransactionLog() +{ + stop_flag.store(true); + log_updated_event->set(); + updating_thread.join(); +} + +UInt64 TransactionLog::parseCSN(const String & csn_node_name) +{ + ReadBufferFromString buf{csn_node_name}; + assertString("csn-", buf); + UInt64 res; + readText(res, buf); + assertEOF(buf); + return res; +} + +TransactionID TransactionLog::parseTID(const String & csn_node_content) +{ + TransactionID tid = Tx::EmptyTID; + if (csn_node_content.empty()) + return tid; + + ReadBufferFromString buf{csn_node_content}; + assertChar('(', buf); + readText(tid.start_csn, buf); + assertString(", ", buf); + readText(tid.local_tid, buf); + assertString(", ", buf); + readText(tid.host_id, buf); + assertChar(')', buf); + assertEOF(buf); + return tid; +} + +String TransactionLog::writeTID(const TransactionID & tid) +{ + WriteBufferFromOwnString buf; + writeChar('(', buf); + writeText(tid.start_csn, buf); + writeCString(", ", buf); + writeText(tid.local_tid, buf); + writeCString(", ", buf); + writeText(tid.host_id, buf); + writeChar(')', buf); + return buf.str(); +} + + +void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_iterator end) +{ + std::vector> 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, *beg, last_entry); + futures.reserve(entries_count); + for (auto it = beg; it != end; ++it) + futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path) / *it)); + + std::vector> 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 = parseCSN(*it); + TransactionID tid = parseTID(res.data); + loaded.emplace_back(tid.getHash(), csn); + LOG_TEST(log, "Got entry {} -> {}", tid, csn); + } + futures.clear(); + + /// Use noexcept here to exit on unexpected exceptions (SIGABRT is better that broken state in memory) + auto insert = [&]() noexcept + { + for (const auto & entry : loaded) + if (entry.first != Tx::EmptyTID.getHash()) + tid_to_csn.emplace(entry.first, entry.second); + last_loaded_entry = last_entry; + latest_snapshot = loaded.back().second; + }; + + MemoryTracker::LockExceptionInThread blocker(VariableContext::Global); + std::lock_guard lock{commit_mutex}; + insert(); +} + +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 + "/csn-", "", zkutil::CreateMode::PersistentSequential); + if (code != Coordination::Error::ZOK) + { + assert(code == Coordination::Error::ZNONODE); + zookeeper->createAncestors(zookeeper_path); + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + for (size_t i = 0; i <= Tx::MaxReservedCSN; ++i) + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/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, 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 == parseCSN(last_loaded_entry)); local_tid_counter = Tx::MaxReservedLocalTID; } +void TransactionLog::runUpdatingThread() +{ + while (true) + { + try + { + log_updated_event->wait(); + if (stop_flag.load()) + return; + + if (!zookeeper) + zookeeper = global_context->getZooKeeper(); + + loadNewEntries(); + } + catch (const Coordination::Exception & e) + { + LOG_ERROR(log, getCurrentExceptionMessage(true)); + /// TODO better backoff + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + if (Coordination::isHardwareError(e.code)) + zookeeper = nullptr; + log_updated_event->set(); + } + catch (...) + { + LOG_ERROR(log, getCurrentExceptionMessage(true)); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + log_updated_event->set(); + } + } +} + +void TransactionLog::loadNewEntries() +{ + Strings entries_list = zookeeper->getChildren(zookeeper_path, 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 == parseCSN(last_loaded_entry)); + latest_snapshot.notify_all(); +} + + Snapshot TransactionLog::getLatestSnapshot() const { return latest_snapshot.load(); @@ -61,16 +243,22 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) } else { - LOG_TRACE(log, "Committing transaction {}{}", txn->tid, txn->dumpDescription()); - std::lock_guard lock{commit_mutex}; - new_csn = 1 + csn_counter.fetch_add(1); - bool inserted = tid_to_csn.try_emplace(txn->tid.getHash(), new_csn).second; /// Commit point - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); - latest_snapshot.store(new_csn, std::memory_order_relaxed); - } + LOG_TEST(log, "Committing transaction {}{}", txn->tid, txn->dumpDescription()); + /// TODO handle connection loss + /// TODO support batching + String path_created = zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point + new_csn = parseCSN(path_created.substr(zookeeper_path.size() + 1)); + LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn); - LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn); + /// Wait for committed changes to become actually visible, so the next transaction will see changes + /// TODO it's optional, add a setting for this + auto current_latest_snapshot = latest_snapshot.load(); + while (current_latest_snapshot < new_csn) + { + latest_snapshot.wait(current_latest_snapshot); + current_latest_snapshot = latest_snapshot.load(); + } + } txn->afterCommit(new_csn); diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index c2edeafb523..44c6ccf53cc 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -1,6 +1,8 @@ #pragma once #include #include +#include +#include #include #include #include @@ -8,6 +10,8 @@ namespace DB { +using ZooKeeperPtr = std::shared_ptr; + class TransactionLog final : private boost::noncopyable { public: @@ -15,6 +19,8 @@ public: TransactionLog(); + ~TransactionLog(); + Snapshot getLatestSnapshot() const; /// Allocated TID, returns transaction object @@ -32,10 +38,20 @@ public: Snapshot getOldestSnapshot() const; private: + void loadLogFromZooKeeper(); + void runUpdatingThread(); + + void loadEntries(Strings::const_iterator beg, Strings::const_iterator end); + void loadNewEntries(); + + static UInt64 parseCSN(const String & csn_node_name); + static TransactionID parseTID(const String & csn_node_content); + static String writeTID(const TransactionID & tid); + + ContextPtr global_context; Poco::Logger * log; std::atomic latest_snapshot; - std::atomic csn_counter; std::atomic local_tid_counter; /// FIXME Transactions: it's probably a bad idea to use global mutex here @@ -45,6 +61,15 @@ private: mutable std::mutex running_list_mutex; std::unordered_map running_list; std::list snapshots_in_use; + + String zookeeper_path; + ZooKeeperPtr zookeeper; + String last_loaded_entry; + zkutil::EventPtr log_updated_event = std::make_shared(); + + std::atomic_bool stop_flag = false; + ThreadFromGlobalPool updating_thread; + }; } diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 9a829d7a5e4..6b0fb15d7f7 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -59,10 +59,9 @@ tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" tx_async 8 "alter table mt update n = 0 where 1" >/dev/null $CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" -tx_wait 8 +tx_sync 8 "rollback" tx 7 "optimize table mt final" tx 7 "select 8, n, _part from mt order by n" -tx 8 "rollback" tx 10 "begin transaction" tx 10 "alter table mt update n = 0 where 1" | grep -Eo "Serialization error" | uniq tx 7 "alter table mt update n=n+1 where 1" From beb03d75ca71dcef7843820a667e49451f532bbf Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 28 Dec 2021 16:43:19 +0300 Subject: [PATCH 028/372] Update 01528_clickhouse_local_prepare_parts.sh --- .../queries/0_stateless/01528_clickhouse_local_prepare_parts.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh index 95ecbf09cf5..e33f75b2b06 100755 --- a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh +++ b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0d91887cdc105f7eda79b98e077b69f7a9bd1802 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Dec 2021 16:15:28 +0300 Subject: [PATCH 029/372] save versions of parts --- src/Common/TransactionID.cpp | 27 ++++++ src/Common/TransactionID.h | 6 ++ src/Functions/FunctionConstantBase.h | 4 +- .../FunctionsTransactionCounters.cpp | 56 +++++------ src/IO/ReadHelpers.cpp | 6 ++ src/IO/ReadHelpers.h | 1 + src/IO/WriteBufferFromFileDescriptor.cpp | 1 - src/Interpreters/MergeTreeTransaction.cpp | 9 ++ src/Interpreters/TransactionLog.cpp | 16 +--- src/Interpreters/TransactionLog.h | 2 +- .../TransactionVersionMetadata.cpp | 85 +++++++++++++++++ src/Interpreters/TransactionVersionMetadata.h | 7 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 79 +++++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 + src/Storages/MergeTree/MergeTreeData.cpp | 95 +++++++++++++++---- .../01172_transaction_counters.reference | 7 ++ .../01172_transaction_counters.sql | 11 +++ 17 files changed, 354 insertions(+), 62 deletions(-) diff --git a/src/Common/TransactionID.cpp b/src/Common/TransactionID.cpp index 4cf93636c11..8a9894fbe53 100644 --- a/src/Common/TransactionID.cpp +++ b/src/Common/TransactionID.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include namespace DB { @@ -13,4 +15,29 @@ TIDHash TransactionID::getHash() const 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; +} + } diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 991dab66e71..2652667ddd1 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -25,6 +25,7 @@ namespace Tx const CSN MaxReservedCSN = 2; const LocalTID PrehistoricLocalTID = 1; + const LocalTID DummyLocalTID = 1; const LocalTID MaxReservedLocalTID = 16; } @@ -57,12 +58,17 @@ struct TransactionID 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}; /// So far, that changes will never become visible const CSN RolledBackCSN = std::numeric_limits::max(); diff --git a/src/Functions/FunctionConstantBase.h b/src/Functions/FunctionConstantBase.h index 2d237c77256..c178b3a256e 100644 --- a/src/Functions/FunctionConstantBase.h +++ b/src/Functions/FunctionConstantBase.h @@ -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: diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp index f2547734e52..f553d0a8be1 100644 --- a/src/Functions/FunctionsTransactionCounters.cpp +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -1,6 +1,10 @@ +#include +#include +#include #include #include #include +#include namespace DB @@ -9,44 +13,40 @@ namespace DB namespace { -class FunctionTransactionID : public IFunction +class FunctionTransactionID : public FunctionConstantBase { public: static constexpr auto name = "transactionID"; - - static FunctionPtr create(ContextPtr context) - { - return std::make_shared(context->getCurrentTransaction()); - } - - explicit FunctionTransactionID(MergeTreeTransactionPtr && txn_) : txn(txn_) - { - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes &) const override - { - return getTransactionIDDataType(); - } - - bool isDeterministic() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override + 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 result_type->createColumnConst(input_rows_count, res); + return res; } -private: - MergeTreeTransactionPtr txn; + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return getTransactionIDDataType(); } + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionTransactionID(ContextPtr context) : FunctionConstantBase(getValue(context->getCurrentTransaction()), context->isDistributed()) {} +}; + +class FunctionTransactionLatestSnapshot : public FunctionConstantBase +{ +public: + static constexpr auto name = "transactionLatestSnapshot"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionTransactionLatestSnapshot(ContextPtr context) : FunctionConstantBase(TransactionLog::instance().getLatestSnapshot(), context->isDistributed()) {} +}; + +class FunctionTransactionOldestSnapshot : public FunctionConstantBase +{ +public: + static constexpr auto name = "transactionOldestSnapshot"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionTransactionOldestSnapshot(ContextPtr context) : FunctionConstantBase(TransactionLog::instance().getOldestSnapshot(), context->isDistributed()) {} }; } @@ -54,6 +54,8 @@ private: void registerFunctionsTransactionCounters(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index b0a6838b81e..ec66630415e 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -138,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) { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b2ad4035cdc..4223bce2d1c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -163,6 +163,7 @@ void readVectorBinary(std::vector & 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); diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index b91114995e8..d3ca4a9fc32 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -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); diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 662edd2cac8..82c2429210a 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -79,6 +79,7 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart storages.insert(storage); creating_parts.push_back(new_part); + new_part->storeVersionMetadata(); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) @@ -91,6 +92,7 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP storages.insert(storage); removing_parts.push_back(part_to_remove); + part_to_remove->storeVersionMetadata(); } void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) @@ -123,9 +125,16 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept [[maybe_unused]] CSN prev_value = csn.exchange(assigned_csn); assert(prev_value == Tx::CommittingCSN); for (const auto & part : creating_parts) + { part->versions.mincsn.store(csn); + part->storeVersionMetadata(); + } + for (const auto & part : removing_parts) + { part->versions.maxcsn.store(csn); + part->storeVersionMetadata(); + } } bool MergeTreeTransaction::rollback() noexcept diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 94cb8cc1827..99a4c44fb5c 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -58,13 +58,7 @@ TransactionID TransactionLog::parseTID(const String & csn_node_content) return tid; ReadBufferFromString buf{csn_node_content}; - assertChar('(', buf); - readText(tid.start_csn, buf); - assertString(", ", buf); - readText(tid.local_tid, buf); - assertString(", ", buf); - readText(tid.host_id, buf); - assertChar(')', buf); + tid = TransactionID::read(buf); assertEOF(buf); return tid; } @@ -72,13 +66,7 @@ TransactionID TransactionLog::parseTID(const String & csn_node_content) String TransactionLog::writeTID(const TransactionID & tid) { WriteBufferFromOwnString 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::write(tid, buf); return buf.str(); } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 44c6ccf53cc..3b9660d46eb 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -22,6 +22,7 @@ public: ~TransactionLog(); Snapshot getLatestSnapshot() const; + Snapshot getOldestSnapshot() const; /// Allocated TID, returns transaction object MergeTreeTransactionPtr beginTransaction(); @@ -35,7 +36,6 @@ public: MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); - Snapshot getOldestSnapshot() const; private: void loadLogFromZooKeeper(); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 59d3b1ef4b9..755ad161d24 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -3,6 +3,9 @@ #include #include #include +#include +#include +#include //#include @@ -13,6 +16,7 @@ namespace ErrorCodes { extern const int SERIALIZATION_ERROR; extern const int LOGICAL_ERROR; + extern const int CANNOT_PARSE_TEXT; } /// It can be used for introspection purposes only @@ -234,6 +238,87 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) return max <= oldest_snapshot_version; } +void VersionMetadata::write(WriteBuffer & buf) const +{ + writeCString("version: 1", buf); + writeCString("\nmintid: ", buf); + TransactionID::write(mintid, buf); + if (CSN min = mincsn.load()) + { + writeCString("\nmincsn: ", buf); + writeText(min, buf); + } + + if (!maxtid.isEmpty()) + { + writeCString("\nmaxtid: ", buf); + TransactionID::write(maxtid, buf); + if (CSN max = maxcsn.load()) + { + writeCString("\nmaxcsn: ", buf); + writeText(max, buf); + } + } +} + +void VersionMetadata::read(ReadBuffer & buf) +{ + assertString("version: 1", buf); + assertString("\nmintid: ", buf); + mintid = TransactionID::read(buf); + if (buf.eof()) + return; + + String name; + constexpr size_t size = 8; + name.resize(size); + + assertChar('\n', buf); + buf.readStrict(name.data(), size); + if (name == "mincsn: ") + { + UInt64 min; + readText(min, buf); + mincsn = min; + if (buf.eof()) + return; + } + + assertChar('\n', buf); + buf.readStrict(name.data(), size); + if (name == "maxtid: ") + { + maxtid = TransactionID::read(buf); + maxtid_lock = maxtid.getHash(); + if (buf.eof()) + return; + } + + assertChar('\n', buf); + buf.readStrict(name.data(), size); + if (name == "maxcsn: ") + { + if (maxtid.isEmpty()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Found maxcsn in metadata file, but maxtid is {}", maxtid); + UInt64 max; + readText(max, buf); + maxcsn = max; + } + + assertEOF(buf); +} + +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; diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 48049da2a23..ed25a205d6a 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -6,7 +6,7 @@ namespace DB struct VersionMetadata { - const TransactionID mintid = Tx::EmptyTID; + TransactionID mintid = Tx::EmptyTID; TransactionID maxtid = Tx::EmptyTID; std::atomic maxtid_lock = 0; @@ -30,6 +30,11 @@ struct VersionMetadata void setMinTID(const TransactionID & tid); bool canBeRemoved(Snapshot oldest_snapshot_version); + + void write(WriteBuffer & buf) const; + void read(ReadBuffer & buf); + + String toString(bool one_line = true) const; }; DataTypePtr getTransactionIDDataType(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 83328594363..f7f139ea3aa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1092,6 +1092,84 @@ void IMergeTreeDataPart::loadColumns(bool require) setColumns(loaded_columns, infos); } + +void IMergeTreeDataPart::storeVersionMetadata() const +{ + assert(!versions.mintid.isEmpty()); + if (versions.mintid.isPrehistoric() && (versions.maxtid.isEmpty() || versions.maxtid.isPrehistoric())) + return; + + 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 out = volume->getDisk()->writeFile(tmp_version_file_name, 4096, WriteMode::Rewrite); + versions.write(*out); + out->finalize(); + out->sync(); + } + + SyncGuardPtr sync_guard; + if (storage.getSettings()->fsync_part_directory) + sync_guard = volume->getDisk()->getDirectorySyncGuard(getFullRelativePath()); + disk->moveFile(tmp_version_file_name, version_file_name); +} + +void IMergeTreeDataPart::loadVersionMetadata() const +{ + 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); + versions.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 + /// We must remove part in this case, but we cannot distinguish it from the first case. + /// TODO Write something to some checksummed file if part was created with transaction, + /// so part will be ether broken or known to be created by transaction. + /// 4. Fsyncs in storeVersionMetadata() work incorrectly. + + if (!disk->exists(tmp_version_file_name)) + { + /// Case 1 (or 3). + /// We do not have version metadata and transactions history for old parts, + /// so let's consider that such parts were created by some ancient transaction + /// and were committed with some prehistoric CSN. + versions.setMinTID(Tx::PrehistoricTID); + versions.mincsn = 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. + versions.setMinTID(Tx::DummyTID); + versions.mincsn = Tx::RolledBackCSN; + remove_tmp_file(); +} + + bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const { /// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds @@ -1276,6 +1354,7 @@ void IMergeTreeDataPart::remove() const disk->removeSharedFileIfExists(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, *keep_shared_data); disk->removeSharedFileIfExists(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, *keep_shared_data); + disk->removeSharedFileIfExists(fs::path(to) / TXN_VERSION_METADATA_FILE_NAME, *keep_shared_data); disk->removeDirectory(to); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 440d0848b1e..fcee542c847 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -404,6 +404,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"; + /// Checks that all TTLs (table min/max, column ttls, so on) for part /// calculated. Part without calculated TTL may exist if TTL was added after /// part creation (using alter query with materialize_ttl setting). @@ -413,6 +415,8 @@ public: /// Required for distinguish different copies of the same part on S3 String getUniqueId() const; + void storeVersionMetadata() const; + void loadVersionMetadata() const; protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 43b17f3c706..369e6369759 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1277,14 +1277,83 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) for (auto & part : duplicate_parts_to_remove) part->remove(); - for (const auto & part : data_parts_by_state_and_info) + auto deactivate_part = [&] (DataPartIteratorByStateAndInfo it) { - /// 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. - /// TODO Transactions: distinguish "prehistoric" parts from uncommitted parts in case of hard restart - part->versions.setMinTID(Tx::PrehistoricTID); - part->versions.mincsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); + (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); + modifyPartState(it, DataPartState::Outdated); + removePartContributionToDataVolume(*it); + }; + + /// All parts are in "Committed" 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::Committed; + }) == data_parts_by_state_and_info.end()); + + auto iter = data_parts_by_state_and_info.begin(); + while (iter != data_parts_by_state_and_info.end() && (*iter)->getState() == DataPartState::Committed) + { + const DataPartPtr & part = *iter; + part->loadVersionMetadata(); + VersionMetadata & versions = part->versions; + + /// Check if CSNs were witten after committing transaction, update and write if needed. + bool versions_updated = false; + if (!versions.mintid.isEmpty() && !part->versions.mincsn) + { + auto min = TransactionLog::instance().getCSN(versions.mintid); + if (!min) + { + /// Transaction that created this part was not committed. Remove part. + min = Tx::RolledBackCSN; + } + LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has mintid={}, setting mincsn={}", + part->name, versions.mintid, min); + versions.mincsn = min; + versions_updated = true; + } + if (!versions.maxtid.isEmpty() && !part->versions.maxcsn) + { + auto max = TransactionLog::instance().getCSN(versions.maxtid); + if (max) + { + LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has maxtid={}, setting maxcsn={}", + part->name, versions.maxtid, max); + versions.maxcsn = max; + } + else + { + /// Transaction that tried to remove this part was not committed. Clear maxtid. + LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: clearing maxtid={}", + part->name, versions.maxtid); + versions.unlockMaxTID(versions.maxtid); + } + versions_updated = true; + } + + /// Sanity checks + bool csn_order = !versions.maxcsn || versions.mincsn <= versions.maxcsn; + bool min_start_csn_order = versions.mintid.start_csn <= versions.mincsn; + bool max_start_csn_order = versions.maxtid.start_csn <= versions.maxcsn; + bool mincsn_known = versions.mincsn; + if (!csn_order || !min_start_csn_order || !max_start_csn_order || !mincsn_known) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} has invalid versions metadata: {}", part->name, versions.toString()); + + if (versions_updated) + part->storeVersionMetadata(); + + /// Deactivate part if creation was not committed or if removal was. + if (versions.mincsn == Tx::RolledBackCSN || versions.maxcsn) + { + auto next_it = std::next(iter); + deactivate_part(iter); + iter = next_it; + } + else + { + ++iter; + } } /// Delete from the set of current parts those parts that are covered by another part (those parts that @@ -1297,15 +1366,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto prev_jt = data_parts_by_state_and_info.begin(); auto curr_jt = std::next(prev_jt); - auto deactivate_part = [&] (DataPartIteratorByStateAndInfo it) - { - (*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed); - modifyPartState(it, DataPartState::Outdated); - (*it)->versions.lockMaxTID(Tx::PrehistoricTID); - (*it)->versions.maxcsn.store(Tx::PrehistoricCSN, std::memory_order_relaxed); - removePartContributionToDataVolume(*it); - }; - (*prev_jt)->assertState({DataPartState::Committed}); while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Committed) @@ -3245,6 +3305,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() @@ -5228,6 +5290,7 @@ 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); 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(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 1f463a25c20..c1a551cb049 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -9,3 +9,10 @@ 4 all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 4 all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 5 1 +6 all_1_1_0 0 +6 all_3_3_0 1 +6 all_4_4_0 1 +7 all_1_1_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +7 all_3_3_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +7 all_4_4_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 +8 1 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index ca114643130..4ca0c3b3bf5 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -21,4 +21,15 @@ select 4, name, mincsn, maxtid, maxcsn from system.parts where database=currentD select 5, transactionID().3 == serverUUID(); commit; +detach table txn_counters; +attach table txn_counters; + +begin transaction; +insert into txn_counters(n) values (4); +select 6, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 7, name, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 8, transactionID().3 == serverUUID(); +commit; + + drop table txn_counters; From 158fbaaa29283717ade4f381efe826b809e0644c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 10 Jan 2022 20:07:01 +0300 Subject: [PATCH 030/372] fix --- src/IO/ReadHelpers.cpp | 1 + src/Storages/StorageMergeTree.cpp | 3 +-- .../0_stateless/01528_clickhouse_local_prepare_parts.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 610e6e4b9da..3bf31f11531 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -27,6 +27,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 diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d5215442a13..30d95054d3d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1201,7 +1201,6 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) auto end_it = current_mutations_by_version.end(); auto begin_it = current_mutations_by_version.begin(); - size_t to_delete_count = std::distance(begin_it, end_it); if (std::optional min_version = getMinPartDataVersion()) end_it = current_mutations_by_version.upper_bound(*min_version); @@ -1228,7 +1227,7 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) if (done_count <= finished_mutations_to_keep) return 0; - to_delete_count = done_count - finished_mutations_to_keep; + size_t to_delete_count = done_count - finished_mutations_to_keep; auto it = begin_it; for (size_t i = 0; i < to_delete_count; ++i) diff --git a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh index e33f75b2b06..48edf41edd7 100755 --- a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh +++ b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: disabled CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e1a41fc69413a1dad8c398981bcb537201eb9f8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 14 Jan 2022 17:03:00 +0300 Subject: [PATCH 031/372] add system log for introspection --- docker/test/stateless/run.sh | 2 +- src/Interpreters/Context.cpp | 11 +++ src/Interpreters/Context.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 4 +- src/Interpreters/MergeTreeTransaction.cpp | 21 ++--- src/Interpreters/SystemLog.cpp | 5 ++ src/Interpreters/SystemLog.h | 3 + src/Interpreters/TransactionLog.cpp | 45 ++++++++-- src/Interpreters/TransactionLog.h | 2 + .../TransactionVersionMetadata.cpp | 72 +++++++++++---- src/Interpreters/TransactionVersionMetadata.h | 26 +++++- src/Interpreters/TransactionsInfoLog.cpp | 88 +++++++++++++++++++ src/Interpreters/TransactionsInfoLog.h | 54 ++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- tests/config/config.d/transactions.xml | 7 ++ .../configs/no_system_log.xml | 1 + .../01172_transaction_counters.reference | 22 +++++ .../01172_transaction_counters.sql | 11 +++ 19 files changed, 342 insertions(+), 44 deletions(-) create mode 100644 src/Interpreters/TransactionsInfoLog.cpp create mode 100644 src/Interpreters/TransactionsInfoLog.h diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index d6d9f189e89..307d9ccea7c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -159,7 +159,7 @@ tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: # Replace the engine with Ordinary to avoid extra symlinks stuff in artifacts. # (so that clickhouse-local --path can read it w/o extra care). sed -i -e "s/ATTACH DATABASE _ UUID '[^']*'/ATTACH DATABASE system/" -e "s/Atomic/Ordinary/" /var/lib/clickhouse/metadata/system.sql -for table in text_log query_log zookeeper_log trace_log; do +for table in text_log query_log zookeeper_log trace_log transactions_info_log; do sed -i "s/ATTACH TABLE _ UUID '[^']*'/ATTACH TABLE $table/" /var/lib/clickhouse/metadata/system/${table}.sql tar -chf /test_output/${table}_dump.tar /var/lib/clickhouse/metadata/system.sql /var/lib/clickhouse/metadata/system/${table}.sql /var/lib/clickhouse/data/system/${table} ||: done diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c24187ae5f7..3a661f26a36 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2383,6 +2383,17 @@ std::shared_ptr Context::getZooKeeperLog() const } +std::shared_ptr Context::getTransactionsInfoLog() const +{ + auto lock = getLock(); + + if (!shared->system_logs) + return {}; + + return shared->system_logs->transactions_info_log; +} + + CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index cd503aef7c1..295d35ce6cf 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -80,6 +80,7 @@ class AsynchronousMetricLog; class OpenTelemetrySpanLog; class ZooKeeperLog; class SessionLog; +class TransactionsInfoLog; struct MergeTreeSettings; class StorageS3Settings; class IDatabase; @@ -790,6 +791,7 @@ public: std::shared_ptr getOpenTelemetrySpanLog() const; std::shared_ptr getZooKeeperLog() const; std::shared_ptr getSessionLog() const; + std::shared_ptr getTransactionsInfoLog() const; /// Returns an object used to log operations with parts if it possible. /// Provide table name to make required checks. diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 123ff6ba2ca..e3b5478bc76 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -446,7 +447,8 @@ BlockIO InterpreterSystemQuery::execute() [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); }, [&] { if (auto query_views_log = getContext()->getQueryViewsLog()) query_views_log->flush(true); }, [&] { if (auto zookeeper_log = getContext()->getZooKeeperLog()) zookeeper_log->flush(true); }, - [&] { if (auto session_log = getContext()->getSessionLog()) session_log->flush(true); } + [&] { if (auto session_log = getContext()->getSessionLog()) session_log->flush(true); }, + [&] { if (auto transactions_info_log = getContext()->getTransactionsInfoLog()) transactions_info_log->flush(true); } ); break; } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 82c2429210a..ae221e4b1a0 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -33,7 +33,7 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - new_part->versions.setMinTID(tid); + new_part->versions.setMinTID(tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); if (txn) txn->addNewPart(storage, new_part); } @@ -41,10 +41,8 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - String error_context = fmt::format("Table: {}, part name: {}", - part_to_remove->storage.getStorageID().getNameForLogs(), - part_to_remove->name); - part_to_remove->versions.lockMaxTID(tid, error_context); + TransactionInfoContext context{storage->getStorageID(), part_to_remove->name}; + part_to_remove->versions.lockMaxTID(tid, context); if (txn) txn->removeOldPart(storage, part_to_remove); } @@ -53,17 +51,16 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - new_part->versions.setMinTID(tid); + TransactionInfoContext context{storage->getStorageID(), new_part->name}; + new_part->versions.setMinTID(tid, context); if (txn) txn->addNewPart(storage, new_part); - String error_context = fmt::format("Table: {}, covering part name: {}", - new_part->storage.getStorageID().getNameForLogs(), - new_part->name); - error_context += ", part_name: {}"; + context.covering_part = std::move(context.part_name); for (const auto & covered : covered_parts) { - covered->versions.lockMaxTID(tid, fmt::format(error_context, covered->name)); + context.part_name = covered->name; + covered->versions.lockMaxTID(tid, context); if (txn) txn->removeOldPart(storage, covered); } @@ -152,7 +149,7 @@ bool MergeTreeTransaction::rollback() noexcept part->versions.mincsn.store(Tx::RolledBackCSN); for (const auto & part : removing_parts) - part->versions.unlockMaxTID(tid); + part->versions.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); /// FIXME const_cast for (const auto & part : creating_parts) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index fc2a5b620e2..bbcc99923ea 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -130,6 +131,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf query_views_log = createSystemLog(global_context, "system", "query_views_log", config, "query_views_log"); zookeeper_log = createSystemLog(global_context, "system", "zookeeper_log", config, "zookeeper_log"); session_log = createSystemLog(global_context, "system", "session_log", config, "session_log"); + transactions_info_log = createSystemLog( + global_context, "system", "transactions_info_log", config, "transactions_info_log"); if (query_log) logs.emplace_back(query_log.get()); @@ -155,6 +158,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf logs.emplace_back(zookeeper_log.get()); if (session_log) logs.emplace_back(session_log.get()); + if (transactions_info_log) + logs.emplace_back(transactions_info_log.get()); try { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index d6342e3973e..865b48e5913 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -78,6 +78,7 @@ class OpenTelemetrySpanLog; class QueryViewsLog; class ZooKeeperLog; class SessionLog; +class TransactionsInfoLog; class ISystemLog @@ -125,6 +126,8 @@ struct SystemLogs std::shared_ptr zookeeper_log; /// Login, LogOut and Login failure events std::shared_ptr session_log; + /// Events related to transactions + std::shared_ptr transactions_info_log; std::vector logs; }; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 99a4c44fb5c..11b119c86c1 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -14,13 +15,34 @@ namespace DB namespace ErrorCodes { -extern const int LOGICAL_ERROR; + extern const int LOGICAL_ERROR; +} + +static void tryWriteEventToSystemLog(Poco::Logger * log, ContextPtr context, + TransactionsInfoLogElement::Type type, const TransactionID & tid, CSN csn = Tx::UnknownCSN) +try +{ + auto system_log = context->getTransactionsInfoLog(); + if (!system_log) + return; + + TransactionsInfoLogElement elem; + elem.type = type; + elem.tid = tid; + elem.csn = csn; + elem.fillCommonFields(nullptr); + system_log->add(elem); +} +catch (...) +{ + tryLogCurrentException(log); } TransactionLog & TransactionLog::instance() { - static TransactionLog inst; - return inst; + /// Use unique_ptr to avoid races on initialization retries if exceptions was thrown from ctor + static std::unique_ptr inst = std::make_unique(); + return *inst; } TransactionLog::TransactionLog() @@ -214,7 +236,10 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} exists", txn->tid.getHash(), txn->tid); txn->snapshot_in_use_it = snapshots_in_use.insert(snapshots_in_use.end(), snapshot); } - LOG_TRACE(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash()); + + LOG_TEST(log, "Beginning transaction {} ({})", txn->tid, txn->tid.getHash()); + tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::BEGIN, txn->tid); + return txn; } @@ -226,8 +251,9 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// TODO Transactions: reset local_tid_counter if (txn->isReadOnly()) { - LOG_TRACE(log, "Closing readonly transaction {}", txn->tid); + LOG_TEST(log, "Closing readonly transaction {}", txn->tid); new_csn = txn->snapshot; + tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::COMMIT, txn->tid, new_csn); } else { @@ -236,7 +262,9 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// TODO support batching String path_created = zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point new_csn = parseCSN(path_created.substr(zookeeper_path.size() + 1)); + LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn); + tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::COMMIT, txn->tid, new_csn); /// Wait for committed changes to become actually visible, so the next transaction will see changes /// TODO it's optional, add a setting for this @@ -257,13 +285,16 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) throw Exception(ErrorCodes::LOGICAL_ERROR, "I's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid); snapshots_in_use.erase(txn->snapshot_in_use_it); } + return new_csn; } void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { LOG_TRACE(log, "Rolling back transaction {}", txn->tid); - if (txn->rollback()) + if (!txn->rollback()) + return; + { std::lock_guard lock{running_list_mutex}; bool removed = running_list.erase(txn->tid.getHash()); @@ -271,6 +302,8 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no abort(); snapshots_in_use.erase(txn->snapshot_in_use_it); } + + tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::ROLLBACK, txn->tid); } MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & tid) diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 3b9660d46eb..90b36793935 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -10,6 +10,8 @@ namespace DB { +class TransactionsInfoLog; +using TransactionsInfoLogPtr = std::shared_ptr; using ZooKeeperPtr = std::shared_ptr; class TransactionLog final : private boost::noncopyable diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 755ad161d24..1e2b0596d93 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -6,8 +7,7 @@ #include #include #include - -//#include +#include namespace DB { @@ -19,6 +19,32 @@ namespace ErrorCodes extern const int CANNOT_PARSE_TEXT; } +static void tryWriteEventToSystemLog(Poco::Logger * log, + TransactionsInfoLogElement::Type type, const TransactionID & tid, + const TransactionInfoContext & context) +try +{ + auto system_log = Context::getGlobalContextInstance()->getTransactionsInfoLog(); + if (!system_log) + return; + + TransactionsInfoLogElement elem; + elem.type = type; + elem.tid = tid; + elem.fillCommonFields(&context); + system_log->add(elem); +} +catch (...) +{ + tryLogCurrentException(log); +} + +VersionMetadata::VersionMetadata() +{ + /// It would be better to make it static, but static loggers do not work for some reason (initialization order?) + log = &Poco::Logger::get("VersionMetadata"); +} + /// It can be used for introspection purposes only TransactionID VersionMetadata::getMaxTID() const { @@ -40,21 +66,26 @@ TransactionID VersionMetadata::getMaxTID() const return Tx::EmptyTID; } -void VersionMetadata::lockMaxTID(const TransactionID & tid, const String & error_context) +void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) { - //LOG_TRACE(&Poco::Logger::get("WTF"), "Trying to lock maxtid by {}: {}\n{}", tid, error_context, StackTrace().toString()); + LOG_TEST(log, "Trying to lock maxtid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); TIDHash locked_by = 0; - if (tryLockMaxTID(tid, &locked_by)) + if (tryLockMaxTID(tid, context, &locked_by)) return; + String part_desc; + if (context.covering_part.empty()) + part_desc = context.part_name; + else + part_desc = fmt::format("{} (covered by {})", context.part_name, context.covering_part); throw Exception(ErrorCodes::SERIALIZATION_ERROR, "Serialization error: " - "Transaction {} tried to remove data part, " - "but it's locked ({}) by another transaction {} which is currently removing this part. {}", - tid, locked_by, getMaxTID(), error_context); + "Transaction {} tried to remove data part {} from {}, " + "but it's locked by another transaction (TID: {}, TIDH: {}) which is currently removing this part.", + tid, part_desc, context.table.getNameForLogs(), getMaxTID(), locked_by); } -bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id) +bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id) { assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); @@ -66,6 +97,7 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_ { /// Don't need to lock part for queries without transaction //FIXME Transactions: why is it possible? + LOG_TEST(log, "Assuming maxtid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); return true; } @@ -75,12 +107,13 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, TIDHash * locked_ } maxtid = tid; + tryWriteEventToSystemLog(log, TransactionsInfoLogElement::LOCK_PART, tid, context); return true; } -void VersionMetadata::unlockMaxTID(const TransactionID & tid) +void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) { - //LOG_TRACE(&Poco::Logger::get("WTF"), "Unlocking maxtid by {}", tid); + LOG_TEST(log, "Unlocking maxtid by {}", tid); assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash locked_by = maxtid_lock.load(); @@ -99,6 +132,8 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid) bool unlocked = maxtid_lock.compare_exchange_strong(locked_by, 0); if (!unlocked) throw_cannot_unlock(); + + tryWriteEventToSystemLog(log, TransactionsInfoLogElement::UNLOCK_PART, tid, context); } bool VersionMetadata::isMaxTIDLocked() const @@ -106,12 +141,14 @@ bool VersionMetadata::isMaxTIDLocked() const return maxtid_lock.load() != 0; } -void VersionMetadata::setMinTID(const TransactionID & tid) +void VersionMetadata::setMinTID(const TransactionID & tid, const TransactionInfoContext & context) { /// TODO Transactions: initialize it in constructor on part creation and remove this method /// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times assert(mintid.isEmpty() || mintid == tid); const_cast(mintid) = tid; + + tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, context); } bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) @@ -121,13 +158,12 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) { - //Poco::Logger * log = &Poco::Logger::get("WTF"); assert(!mintid.isEmpty()); CSN min = mincsn.load(std::memory_order_relaxed); TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); CSN max = maxcsn.load(std::memory_order_relaxed); - //LOG_TRACE(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid); + //LOG_TEST(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid); [[maybe_unused]] bool had_mincsn = min; [[maybe_unused]] bool had_maxtid = max_lock; @@ -184,7 +220,6 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// But for long-running writing transactions we will always do /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. min = TransactionLog::instance().getCSN(mintid); - //LOG_TRACE(log, "Got min {}", min); if (!min) return false; /// Part creation is not committed yet @@ -195,7 +230,6 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current if (max_lock) { max = TransactionLog::instance().getCSN(max_lock); - //LOG_TRACE(log, "Got ax {}", max); if (max) maxcsn.store(max, std::memory_order_relaxed); } @@ -206,11 +240,13 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) { CSN min = mincsn.load(std::memory_order_relaxed); + /// We can safely remove part if its creation was rolled back if (min == Tx::RolledBackCSN) return true; if (!min) { + /// Cannot remove part if its creation not committed yet min = TransactionLog::instance().getCSN(mintid); if (min) mincsn.store(min, std::memory_order_relaxed); @@ -218,16 +254,19 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) return false; } + /// Part is probably visible for some transactions (part is too new or the oldest snapshot is too old) if (oldest_snapshot_version < min) return false; TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); + /// Part is active if (!max_lock) return false; CSN max = maxcsn.load(std::memory_order_relaxed); if (!max) { + /// Part removal is not committed yet max = TransactionLog::instance().getCSN(max_lock); if (max) maxcsn.store(max, std::memory_order_relaxed); @@ -235,6 +274,7 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) return false; } + /// We can safely remove part if all running transactions were started after part removal was committed return max <= oldest_snapshot_version; } diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index ed25a205d6a..6065bbe4ed1 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -1,9 +1,24 @@ #pragma once #include +#include + +namespace Poco +{ +class Logger; +} namespace DB { +struct TransactionInfoContext +{ + StorageID table = StorageID::createEmpty(); + String part_name; + String covering_part; + + TransactionInfoContext(StorageID id, String part) : table(std::move(id)), part_name(std::move(part)) {} +}; + struct VersionMetadata { TransactionID mintid = Tx::EmptyTID; @@ -20,14 +35,14 @@ struct VersionMetadata TransactionID getMinTID() const { return mintid; } TransactionID getMaxTID() const; - bool tryLockMaxTID(const TransactionID & tid, TIDHash * locked_by_id = nullptr); - void lockMaxTID(const TransactionID & tid, const String & error_context = {}); - void unlockMaxTID(const TransactionID & tid); + bool tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr); + void lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); + void unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); bool isMaxTIDLocked() const; /// It can be called only from MergeTreeTransaction or on server startup - void setMinTID(const TransactionID & tid); + void setMinTID(const TransactionID & tid, const TransactionInfoContext & context); bool canBeRemoved(Snapshot oldest_snapshot_version); @@ -35,6 +50,9 @@ struct VersionMetadata void read(ReadBuffer & buf); String toString(bool one_line = true) const; + + Poco::Logger * log; + VersionMetadata(); }; DataTypePtr getTransactionIDDataType(); diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp new file mode 100644 index 00000000000..ec80e8db942 --- /dev/null +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +NamesAndTypesList TransactionsInfoLogElement::getNamesAndTypes() +{ + auto type_enum = std::make_shared( + DataTypeEnum8::Values + { + {"Begin", static_cast(BEGIN)}, + {"Commit", static_cast(COMMIT)}, + {"Rollback", static_cast(ROLLBACK)}, + + {"AddPart", static_cast(ADD_PART)}, + {"LockPart", static_cast(LOCK_PART)}, + {"UnlockPart", static_cast(UNLOCK_PART)}, + }); + + return + { + {"type", std::move(type_enum)}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared(6)}, + {"thread_id", std::make_shared()}, + + {"query_id", std::make_shared()}, + {"tid", getTransactionIDDataType()}, + {"tid_hash", std::make_shared()}, + + {"csn", std::make_shared()}, + + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"uuid", std::make_shared()}, + {"part", std::make_shared()}, + }; +} + +void TransactionsInfoLogElement::fillCommonFields(const TransactionInfoContext * context) +{ + event_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + thread_id = getThreadId(); + + query_id = CurrentThread::getQueryId().toString(); + + if (!context) + return; + + table = context->table; + part_name = context->part_name; +} + +void TransactionsInfoLogElement::appendToBlock(MutableColumns & columns) const +{ + assert(type != UNKNOWN); + assert(!tid.isEmpty()); + size_t i = 0; + + columns[i++]->insert(type); + auto event_time_seconds = event_time / 1000000; + columns[i++]->insert(DateLUT::instance().toDayNum(event_time_seconds).toUnderType()); + columns[i++]->insert(event_time); + columns[i++]->insert(thread_id); + + columns[i++]->insert(query_id); + columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); + columns[i++]->insert(tid.getHash()); + + columns[i++]->insert(csn); + + columns[i++]->insert(table.database_name); + columns[i++]->insert(table.table_name); + columns[i++]->insert(table.uuid); + columns[i++]->insert(part_name); +} + +} diff --git a/src/Interpreters/TransactionsInfoLog.h b/src/Interpreters/TransactionsInfoLog.h new file mode 100644 index 00000000000..8b48b54aa88 --- /dev/null +++ b/src/Interpreters/TransactionsInfoLog.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include + +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 +{ + using SystemLog::SystemLog; +}; + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 84477b03cf7..caa2745ea3b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1156,13 +1156,15 @@ void IMergeTreeDataPart::loadVersionMetadata() const /// so part will be ether broken or known to be created by transaction. /// 4. Fsyncs in storeVersionMetadata() work incorrectly. + TransactionInfoContext txn_context{storage.getStorageID(), name}; + if (!disk->exists(tmp_version_file_name)) { /// Case 1 (or 3). /// We do not have version metadata and transactions history for old parts, /// so let's consider that such parts were created by some ancient transaction /// and were committed with some prehistoric CSN. - versions.setMinTID(Tx::PrehistoricTID); + versions.setMinTID(Tx::PrehistoricTID, txn_context); versions.mincsn = Tx::PrehistoricCSN; return; } @@ -1170,7 +1172,7 @@ void IMergeTreeDataPart::loadVersionMetadata() const /// Case 2. /// Content of *.tmp file may be broken, just use fake TID. /// Transaction was not committed if *.tmp file was not renamed, so we should complete rollback by removing part. - versions.setMinTID(Tx::DummyTID); + versions.setMinTID(Tx::DummyTID, txn_context); versions.mincsn = Tx::RolledBackCSN; remove_tmp_file(); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4c66a4d5aa5..cad6ad0c5cf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1329,7 +1329,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// Transaction that tried to remove this part was not committed. Clear maxtid. LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: clearing maxtid={}", part->name, versions.maxtid); - versions.unlockMaxTID(versions.maxtid); + versions.unlockMaxTID(versions.maxtid, TransactionInfoContext{getStorageID(), part->name}); } versions_updated = true; } @@ -4445,7 +4445,7 @@ void MergeTreeData::Transaction::rollback() DataPartPtr covering_part; DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, lock); for (auto & covered : covered_parts) - covered->versions.unlockMaxTID(Tx::PrehistoricTID); + covered->versions.unlockMaxTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name}); } } diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index 731a312ed58..b0d57ad651d 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,3 +1,10 @@ <_enable_experimental_mvcc_prototype_test_helper_dev>42 + + + system + transactions_info_log
+ 7500 +
+
diff --git a/tests/integration/test_MemoryTracking/configs/no_system_log.xml b/tests/integration/test_MemoryTracking/configs/no_system_log.xml index bd1b9f9a49e..3218dae4dc7 100644 --- a/tests/integration/test_MemoryTracking/configs/no_system_log.xml +++ b/tests/integration/test_MemoryTracking/configs/no_system_log.xml @@ -14,4 +14,5 @@ + diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index c1a551cb049..0ab65e5dc9a 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -16,3 +16,25 @@ 7 all_3_3_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 7 all_4_4_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 8 1 +1 1 AddPart 1 1 1 1 all_1_1_0 +2 1 Begin 1 1 1 1 +2 1 AddPart 1 1 1 1 all_2_2_0 +2 1 Rollback 1 1 1 1 +3 1 Begin 1 1 1 1 +3 1 AddPart 1 1 1 1 all_3_3_0 +3 1 Commit 1 1 1 0 +1 1 AddPart 1 1 1 1 all_1_1_0 +4 1 Begin 1 1 1 1 +4 1 AddPart 1 1 1 1 all_4_4_0 +4 1 Commit 1 1 1 0 +5 1 Begin 1 1 1 1 +5 1 AddPart 1 1 1 1 all_5_5_0 +5 1 LockPart 1 1 1 1 all_1_1_0 +5 1 LockPart 1 1 1 1 all_3_3_0 +5 1 LockPart 1 1 1 1 all_4_4_0 +5 1 LockPart 1 1 1 1 all_5_5_0 +5 1 UnlockPart 1 1 1 1 all_1_1_0 +5 1 UnlockPart 1 1 1 1 all_3_3_0 +5 1 UnlockPart 1 1 1 1 all_4_4_0 +5 1 UnlockPart 1 1 1 1 all_5_5_0 +5 1 Rollback 1 1 1 1 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 4ca0c3b3bf5..89a3a08f57b 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -31,5 +31,16 @@ select 7, name, maxtid, maxcsn from system.parts where database=currentDatabase( select 8, transactionID().3 == serverUUID(); commit; +begin transaction; +insert into txn_counters(n) values (5); +alter table txn_counters drop partition id 'all'; +rollback; + +system flush logs; +select indexOf((select arraySort(groupUniqArray(tid)) from system.transactions_info_log where database=currentDatabase() and table='txn_counters'), tid), + (toDecimal64(now64(6), 6) - toDecimal64(event_time, 6)) < 100, type, thread_id!=0, length(query_id)=length(queryID()), tid_hash!=0, csn=0, part +from system.transactions_info_log +where tid in (select tid from system.transactions_info_log where database=currentDatabase() and table='txn_counters' and not (tid.1=1 and tid.2=1)) +or (database=currentDatabase() and table='txn_counters') order by event_time; drop table txn_counters; From 7feb01480478bd46f4eed651f15649bc361f57d4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 14 Jan 2022 23:26:18 +0300 Subject: [PATCH 032/372] rebuild docker image --- docker/test/integration/base/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 91b26735fe5..2ee8d369dc2 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #33610 +# rebuild in #24258 # docker build -t clickhouse/integration-test . ARG FROM_TAG=latest FROM clickhouse/test-base:$FROM_TAG From 5132411e00de5c917afc23fcec7ea96786fb78da Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 15 Jan 2022 14:07:40 +0300 Subject: [PATCH 033/372] rebuild docker image --- docker/test/integration/base/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 2ee8d369dc2..81129233f68 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -57,7 +57,7 @@ RUN echo $'tickTime=2500 \n\ tickTime=2500 \n\ dataDir=/zookeeper \n\ clientPort=2181 \n\ -maxClientCnxns=80' > /opt/zookeeper/conf/zoo.cfg +maxClientCnxns=81' > /opt/zookeeper/conf/zoo.cfg RUN mkdir /zookeeper && chmod -R 777 /zookeeper ENV TZ=Europe/Moscow From 9a518d789f1b14a31c767da4e6309087491adcba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 15 Jan 2022 16:21:43 +0300 Subject: [PATCH 034/372] rebuild docker image --- docker/test/base/Dockerfile | 2 +- docker/test/integration/base/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 6beab2e5bb7..11881365318 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #33610 +# rebuild in #24258 # docker build -t clickhouse/test-base . ARG FROM_TAG=latest FROM clickhouse/test-util:$FROM_TAG diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 81129233f68..2ee8d369dc2 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -57,7 +57,7 @@ RUN echo $'tickTime=2500 \n\ tickTime=2500 \n\ dataDir=/zookeeper \n\ clientPort=2181 \n\ -maxClientCnxns=81' > /opt/zookeeper/conf/zoo.cfg +maxClientCnxns=80' > /opt/zookeeper/conf/zoo.cfg RUN mkdir /zookeeper && chmod -R 777 /zookeeper ENV TZ=Europe/Moscow From d237099729e38f8fd02b52ff01fb7e9785ceddc3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 15 Jan 2022 22:38:25 +0300 Subject: [PATCH 035/372] rebuild docker image --- docker/test/util/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index d9827260acb..c83643a4069 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #33610 +# rebuild in #24258 # docker build -t clickhouse/test-util . FROM ubuntu:20.04 From 74580121033a8f07e89d1a38bdabb6c7b61f556e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 19 Jan 2022 21:29:31 +0300 Subject: [PATCH 036/372] some fixes, add test with restart --- src/Interpreters/Context.cpp | 3 + src/Interpreters/MergeTreeTransaction.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 17 +++-- src/Interpreters/TransactionLog.h | 43 ++++++++++- .../TransactionVersionMetadata.cpp | 16 ++-- src/Interpreters/TransactionsInfoLog.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++ .../MergeTree/MutatePlainMergeTreeTask.cpp | 4 +- tests/integration/helpers/cluster.py | 3 +- .../integration/test_transactions/__init__.py | 0 .../configs/transactions.xml | 12 +++ tests/integration/test_transactions/test.py | 76 +++++++++++++++++++ 12 files changed, 162 insertions(+), 21 deletions(-) create mode 100644 tests/integration/test_transactions/__init__.py create mode 100644 tests/integration/test_transactions/configs/transactions.xml create mode 100644 tests/integration/test_transactions/test.py diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3a661f26a36..15ecd70a233 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -87,6 +87,7 @@ #include #include #include +#include #include @@ -341,6 +342,8 @@ struct ContextSharedPart if (common_executor) common_executor->wait(); + TransactionLog::shutdownIfAny(); + std::unique_ptr delete_system_logs; { auto lock = std::lock_guard(mutex); diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index ae221e4b1a0..6e6a69c81d8 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -148,7 +148,7 @@ bool MergeTreeTransaction::rollback() noexcept for (const auto & part : creating_parts) part->versions.mincsn.store(Tx::RolledBackCSN); - for (const auto & part : removing_parts) + for (const auto & part : removing_parts) /// TODO update metadata file part->versions.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); /// FIXME const_cast diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 11b119c86c1..9af21ac7e1f 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -38,12 +38,6 @@ catch (...) tryLogCurrentException(log); } -TransactionLog & TransactionLog::instance() -{ - /// Use unique_ptr to avoid races on initialization retries if exceptions was thrown from ctor - static std::unique_ptr inst = std::make_unique(); - return *inst; -} TransactionLog::TransactionLog() : log(&Poco::Logger::get("TransactionLog")) @@ -58,8 +52,15 @@ TransactionLog::TransactionLog() TransactionLog::~TransactionLog() { - stop_flag.store(true); + shutdown(); +} + +void TransactionLog::shutdown() +{ + if (stop_flag.exchange(true)) + return; log_updated_event->set(); + latest_snapshot.notify_all(); updating_thread.join(); } @@ -269,7 +270,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// Wait for committed changes to become actually visible, so the next transaction will see changes /// TODO it's optional, add a setting for this auto current_latest_snapshot = latest_snapshot.load(); - while (current_latest_snapshot < new_csn) + while (current_latest_snapshot < new_csn && !stop_flag) { latest_snapshot.wait(current_latest_snapshot); current_latest_snapshot = latest_snapshot.load(); diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 90b36793935..d6ab1b3d63c 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -10,19 +10,57 @@ 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 +class SingletonHelper : private boost::noncopyable +{ +public: + static Derived & instance() + { + Derived * ptr = instance_raw_ptr.load(); + if (likely(ptr)) + return *ptr; + + std::lock_guard lock{instance_mutex}; + if (!instance_holder.has_value()) + { + instance_holder.emplace(); + instance_raw_ptr = &instance_holder.value(); + } + return instance_holder.value(); + } + + static void shutdownIfAny() + { + std::lock_guard lock{instance_mutex}; + if (instance_holder.has_value()) + instance_holder->shutdown(); + } + +private: + static inline std::atomic instance_raw_ptr; + static inline std::optional instance_holder; + static inline std::mutex instance_mutex; +}; + class TransactionsInfoLog; using TransactionsInfoLogPtr = std::shared_ptr; using ZooKeeperPtr = std::shared_ptr; -class TransactionLog final : private boost::noncopyable +class TransactionLog final : public SingletonHelper { public: - static TransactionLog & instance(); + //static TransactionLog & instance(); TransactionLog(); ~TransactionLog(); + void shutdown(); + Snapshot getLatestSnapshot() const; Snapshot getOldestSnapshot() const; @@ -71,7 +109,6 @@ private: std::atomic_bool stop_flag = false; ThreadFromGlobalPool updating_thread; - }; } diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 1e2b0596d93..abe4a7b529d 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -113,7 +113,7 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) { - LOG_TEST(log, "Unlocking maxtid by {}", tid); + LOG_TEST(log, "Unlocking maxtid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash locked_by = maxtid_lock.load(); @@ -289,8 +289,10 @@ void VersionMetadata::write(WriteBuffer & buf) const writeText(min, buf); } - if (!maxtid.isEmpty()) + if (maxtid_lock) { + assert(!maxtid.isEmpty()); + assert(maxtid.getHash() == maxtid_lock); writeCString("\nmaxtid: ", buf); TransactionID::write(maxtid, buf); if (CSN max = maxcsn.load()) @@ -322,20 +324,22 @@ void VersionMetadata::read(ReadBuffer & buf) mincsn = min; if (buf.eof()) return; + + assertChar('\n', buf); + buf.readStrict(name.data(), size); } - assertChar('\n', buf); - buf.readStrict(name.data(), size); if (name == "maxtid: ") { maxtid = TransactionID::read(buf); maxtid_lock = maxtid.getHash(); if (buf.eof()) return; + + assertChar('\n', buf); + buf.readStrict(name.data(), size); } - assertChar('\n', buf); - buf.readStrict(name.data(), size); if (name == "maxcsn: ") { if (maxtid.isEmpty()) diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index ec80e8db942..890bddeedd7 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -64,7 +64,6 @@ void TransactionsInfoLogElement::fillCommonFields(const TransactionInfoContext * void TransactionsInfoLogElement::appendToBlock(MutableColumns & columns) const { assert(type != UNKNOWN); - assert(!tid.isEmpty()); size_t i = 0; columns[i++]->insert(type); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index caa2745ea3b..37524b93f82 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1122,6 +1122,7 @@ void IMergeTreeDataPart::storeVersionMetadata() const } 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"; @@ -1176,6 +1177,11 @@ void IMergeTreeDataPart::loadVersionMetadata() const versions.mincsn = Tx::RolledBackCSN; remove_tmp_file(); } +catch (Exception & e) +{ + e.addMessage("While loading version metadata from table {} part {}", storage.getStorageID().getNameForLogs(), name); + throw; +} bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 4e2ca8aee54..7388ba2790d 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -97,7 +97,9 @@ bool MutatePlainMergeTreeTask::executeStep() { if (merge_mutate_entry->txn) merge_mutate_entry->txn->onException(); - storage.updateMutationEntriesErrors(future_part, false, getCurrentExceptionMessage(false)); + String exception_message = getCurrentExceptionMessage(false); + LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message); + storage.updateMutationEntriesErrors(future_part, false, exception_message); write_part_log(ExecutionStatus::fromCurrentException()); return false; } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dd2028f36a3..d5383bd6ea8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2085,7 +2085,8 @@ class ClickHouseInstance: else: params = params.copy() - params["query"] = sql + if sql is not None: + params["query"] = sql auth = None if user and password: diff --git a/tests/integration/test_transactions/__init__.py b/tests/integration/test_transactions/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_transactions/configs/transactions.xml b/tests/integration/test_transactions/configs/transactions.xml new file mode 100644 index 00000000000..5c250f623f7 --- /dev/null +++ b/tests/integration/test_transactions/configs/transactions.xml @@ -0,0 +1,12 @@ + + <_enable_experimental_mvcc_prototype_test_helper_dev>42 + + + + + + system + transactions_info_log
+ 7500 +
+
diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py new file mode 100644 index 00000000000..963dbfdbf61 --- /dev/null +++ b/tests/integration/test_transactions/test.py @@ -0,0 +1,76 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/transactions.xml"], stay_alive=True, with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def tx(session, query): + params = {'session_id': 'session_{}'.format(session)} + return node.http_query(None, data=query, params=params) + + +def test_rollback_unfinished_on_restart(start_cluster): + node.query('create table mt (n int, m int) engine=MergeTree order by n partition by n % 2') + node.query('insert into mt values (1, 10), (2, 20)') + tid0 = "(1,1,'00000000-0000-0000-0000-000000000000')" + + tx(1, 'begin transaction') + tid1 = tx(1, 'select transactionID()').strip() + tx(1, "alter table mt drop partition id '1'") + tx(1, 'commit') + + tx(1, 'begin transaction') + tid2 = tx(1, 'select transactionID()').strip() + tx(1, 'insert into mt values (3, 30), (4, 40)') + tx(1, 'commit') + + node.query('system flush logs') + csn1 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid1)).strip() + csn2 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid2)).strip() + + tx(1, 'begin transaction') + tid3 = tx(1, 'select transactionID()').strip() + tx(1, 'insert into mt values (5, 50)') + tx(1, "alter table mt update m = m+n in partition id '1' where 1") + + tx(2, 'begin transaction') + tid4 = tx(2, 'select transactionID()').strip() + tx(2, "optimize table mt partition id '0' final settings optimize_throw_if_noop = 1") + + tx(3, 'begin transaction') + tid5 = tx(3, 'select transactionID()').strip() + tx(3, 'insert into mt values (6, 70)') + + node.restart_clickhouse(kill=True) + + assert node.query('select *, _part from mt order by n') == '2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n' + res = node.query("select name, active, mintid, 'csn' || toString(mincsn), maxtid, 'csn' || toString(maxcsn) from system.parts where table='mt' order by name") + res = res.replace(tid0, 'tid0') + res = res.replace(tid1, 'tid1').replace('csn' + csn1, 'csn_1') + res = res.replace(tid2, 'tid2').replace('csn' + csn2, 'csn_2') + res = res.replace(tid3, 'tid3') + res = res.replace(tid4, 'tid4') + res = res.replace(tid5, 'tid5') + assert res == "0_2_2_0\t1\ttid0\tcsn1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "0_2_4_1\t0\ttid4\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "0_7_7_0\t0\ttid5\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_3_3_0_6\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_5_5_0\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_5_5_0_6\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" + + + + + From ba5844f1d13978d61c1f320e3fef9ab96af4c5fa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 19 Jan 2022 23:16:05 +0300 Subject: [PATCH 037/372] fix build after merge --- src/Interpreters/TransactionLog.cpp | 2 +- src/Interpreters/TransactionVersionMetadata.cpp | 1 + src/Interpreters/TransactionsInfoLog.cpp | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 9af21ac7e1f..5223b6415bc 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -130,7 +130,7 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite latest_snapshot = loaded.back().second; }; - MemoryTracker::LockExceptionInThread blocker(VariableContext::Global); + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); std::lock_guard lock{commit_mutex}; insert(); } diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index abe4a7b529d..7bba0ab668f 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index 890bddeedd7..c126e38b5b3 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include From c021fb2bd5f3afcf6e1e09d9409fcf1efdd92ad8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jan 2022 18:23:44 +0300 Subject: [PATCH 038/372] fix runtime linking error (lol) --- src/Interpreters/SystemLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index a4eacd73605..44e599fd629 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -642,5 +642,6 @@ template class SystemLog; template class SystemLog; template class SystemLog; template class SystemLog; +template class SystemLog; } From 13857ceb697a59ff137b6adac7d104baedd8d4de Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jan 2022 21:15:23 +0300 Subject: [PATCH 039/372] fixes --- src/Interpreters/TransactionLog.cpp | 28 +++++++++++++++---- src/Interpreters/TransactionLog.h | 5 ++-- .../configs/transactions.xml | 1 + tests/integration/test_transactions/test.py | 4 +++ 4 files changed, 31 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 5223b6415bc..105a62c87f9 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -62,6 +62,16 @@ void TransactionLog::shutdown() 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::parseCSN(const String & csn_node_name) @@ -131,7 +141,7 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite }; LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - std::lock_guard lock{commit_mutex}; + std::lock_guard lock{mutex}; insert(); } @@ -184,7 +194,11 @@ void TransactionLog::runUpdatingThread() return; if (!zookeeper) - zookeeper = global_context->getZooKeeper(); + { + auto new_zookeeper = global_context->getZooKeeper(); + std::lock_guard lock{mutex}; + zookeeper = new_zookeeper; + } loadNewEntries(); } @@ -194,7 +208,10 @@ void TransactionLog::runUpdatingThread() /// TODO better backoff std::this_thread::sleep_for(std::chrono::milliseconds(1000)); if (Coordination::isHardwareError(e.code)) - zookeeper = nullptr; + { + std::lock_guard lock{mutex}; + zookeeper.reset(); + } log_updated_event->set(); } catch (...) @@ -261,7 +278,8 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) LOG_TEST(log, "Committing transaction {}{}", txn->tid, txn->dumpDescription()); /// TODO handle connection loss /// TODO support batching - String path_created = zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point + auto current_zookeeper = getZooKeeper(); + String path_created = current_zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point new_csn = parseCSN(path_created.substr(zookeeper_path.size() + 1)); LOG_INFO(log, "Transaction {} committed with CSN={}", txn->tid, new_csn); @@ -328,7 +346,7 @@ CSN TransactionLog::getCSN(const TIDHash & tid) const if (tid == Tx::PrehistoricTID.getHash()) return Tx::PrehistoricCSN; - std::lock_guard lock{commit_mutex}; + std::lock_guard lock{mutex}; auto it = tid_to_csn.find(tid); if (it == tid_to_csn.end()) return Tx::UnknownCSN; diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index d6ab1b3d63c..f2599a4d104 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -88,14 +88,15 @@ private: static TransactionID parseTID(const String & csn_node_content); static String writeTID(const TransactionID & tid); + ZooKeeperPtr getZooKeeper() const; + ContextPtr global_context; Poco::Logger * log; std::atomic latest_snapshot; std::atomic local_tid_counter; - /// FIXME Transactions: it's probably a bad idea to use global mutex here - mutable std::mutex commit_mutex; + mutable std::mutex mutex; std::unordered_map tid_to_csn; mutable std::mutex running_list_mutex; diff --git a/tests/integration/test_transactions/configs/transactions.xml b/tests/integration/test_transactions/configs/transactions.xml index 5c250f623f7..620515cd7bd 100644 --- a/tests/integration/test_transactions/configs/transactions.xml +++ b/tests/integration/test_transactions/configs/transactions.xml @@ -2,6 +2,7 @@ <_enable_experimental_mvcc_prototype_test_helper_dev>42 + 100500 diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 963dbfdbf61..97f1fa7b1f6 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -24,6 +24,9 @@ def test_rollback_unfinished_on_restart(start_cluster): node.query('insert into mt values (1, 10), (2, 20)') tid0 = "(1,1,'00000000-0000-0000-0000-000000000000')" + # it will hold a snapshot and avoid parts cleanup + tx(0, 'begin transaction') + tx(1, 'begin transaction') tid1 = tx(1, 'select transactionID()').strip() tx(1, "alter table mt drop partition id '1'") @@ -65,6 +68,7 @@ def test_rollback_unfinished_on_restart(start_cluster): "0_2_4_1\t0\ttid4\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "0_7_7_0\t0\ttid5\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_1_1_0\t0\ttid0\tcsn1\ttid1\tcsn_1\n" \ "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "1_3_3_0_6\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "1_5_5_0\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ From e0304c2a5871c89345c6784be1c6650972f89f92 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 28 Jan 2022 20:47:37 +0300 Subject: [PATCH 040/372] review fixes, write tid into mutation entry --- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/MergeTreeTransaction.cpp | 24 +-- .../TransactionVersionMetadata.cpp | 143 ++++++++++-------- src/Interpreters/TransactionVersionMetadata.h | 16 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 16 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 98 ++++++------ src/Storages/MergeTree/MergeTreeData.h | 5 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 14 ++ .../MergeTree/MergeTreePartsMover.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageMergeTree.cpp | 33 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/System/StorageSystemParts.cpp | 18 +-- .../System/StorageSystemPartsBase.cpp | 4 +- tests/config/config.d/transactions.xml | 3 + .../configs/transactions.xml | 1 + tests/integration/test_transactions/test.py | 27 +++- .../01172_transaction_counters.sql | 16 +- .../02117_show_create_table_system.reference | 2 +- 21 files changed, 247 insertions(+), 191 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3f33f99eca3..e3cf6751cf7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1764,8 +1764,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) - && typeid_cast(query_analyzer->aggregates()[0].function.get()) - && !context->getCurrentTransaction(); + && typeid_cast(query_analyzer->aggregates()[0].function.get()); if (optimize_trivial_count) { @@ -1773,7 +1772,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc const auto & func = desc.function; std::optional num_rows{}; - if (!query.prewhere() && !query.where()) + if (!query.prewhere() && !query.where() && !context->getCurrentTransaction()) { num_rows = storage->totalRows(settings); } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 6e6a69c81d8..430749dedb9 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -33,7 +33,7 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - new_part->versions.setMinTID(tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); + new_part->version.setCreationTID(tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); if (txn) txn->addNewPart(storage, new_part); } @@ -42,7 +42,7 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; TransactionInfoContext context{storage->getStorageID(), part_to_remove->name}; - part_to_remove->versions.lockMaxTID(tid, context); + part_to_remove->version.lockMaxTID(tid, context); if (txn) txn->removeOldPart(storage, part_to_remove); } @@ -52,7 +52,7 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; TransactionInfoContext context{storage->getStorageID(), new_part->name}; - new_part->versions.setMinTID(tid, context); + new_part->version.setCreationTID(tid, context); if (txn) txn->addNewPart(storage, new_part); @@ -60,7 +60,7 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage for (const auto & covered : covered_parts) { context.part_name = covered->name; - covered->versions.lockMaxTID(tid, context); + covered->version.lockMaxTID(tid, context); if (txn) txn->removeOldPart(storage, covered); } @@ -123,13 +123,13 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept assert(prev_value == Tx::CommittingCSN); for (const auto & part : creating_parts) { - part->versions.mincsn.store(csn); + part->version.creation_csn.store(csn); part->storeVersionMetadata(); } for (const auto & part : removing_parts) { - part->versions.maxcsn.store(csn); + part->version.removal_csn.store(csn); part->storeVersionMetadata(); } } @@ -146,17 +146,17 @@ bool MergeTreeTransaction::rollback() noexcept table_and_mutation.first->killMutation(table_and_mutation.second); for (const auto & part : creating_parts) - part->versions.mincsn.store(Tx::RolledBackCSN); + part->version.creation_csn.store(Tx::RolledBackCSN); for (const auto & part : removing_parts) /// TODO update metadata file - part->versions.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); + part->version.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); /// FIXME const_cast for (const auto & part : creating_parts) const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); for (const auto & part : removing_parts) - if (part->versions.getMinTID() != tid) + if (part->version.getCreationTID() != tid) const_cast(part->storage).restoreAndActivatePart(part); return true; @@ -180,9 +180,9 @@ String MergeTreeTransaction::dumpDescription() const for (const auto & part : removing_parts) { res += part->name; - res += fmt::format(" (created by {}, {})\n", part->versions.getMinTID(), part->versions.mincsn); - assert(!part->versions.mincsn || part->versions.mincsn <= snapshot); - assert(!part->versions.maxcsn); + res += fmt::format(" (created by {}, {})\n", part->version.getCreationTID(), part->version.creation_csn); + assert(!part->version.creation_csn || part->version.creation_csn <= snapshot); + assert(!part->version.removal_csn); } return res; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 7bba0ab668f..4b00a2e0625 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -47,9 +47,9 @@ VersionMetadata::VersionMetadata() } /// It can be used for introspection purposes only -TransactionID VersionMetadata::getMaxTID() const +TransactionID VersionMetadata::getRemovalTID() const { - TIDHash max_lock = maxtid_lock.load(); + TIDHash max_lock = removal_tid_lock.load(); if (max_lock) { if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) @@ -58,10 +58,10 @@ TransactionID VersionMetadata::getMaxTID() const return Tx::PrehistoricTID; } - if (maxcsn.load(std::memory_order_relaxed)) + if (removal_csn.load(std::memory_order_relaxed)) { - /// maxtid cannot be changed since we have maxcsn, so it's readonly - return maxtid; + /// removal_tid cannot be changed since we have removal_csn, so it's readonly + return removal_tid; } return Tx::EmptyTID; @@ -69,7 +69,7 @@ TransactionID VersionMetadata::getMaxTID() const void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) { - LOG_TEST(log, "Trying to lock maxtid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); + LOG_TEST(log, "Trying to lock removal_tid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); TIDHash locked_by = 0; if (tryLockMaxTID(tid, context, &locked_by)) return; @@ -83,7 +83,7 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInf "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(), getMaxTID(), locked_by); + tid, part_desc, context.table.getNameForLogs(), getRemovalTID(), locked_by); } bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id) @@ -91,14 +91,14 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash expected_max_lock_value = 0; - bool locked = maxtid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); + bool locked = removal_tid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); if (!locked) { if (tid == Tx::PrehistoricTID && expected_max_lock_value == Tx::PrehistoricTID.getHash()) { /// Don't need to lock part for queries without transaction //FIXME Transactions: why is it possible? - LOG_TEST(log, "Assuming maxtid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); + LOG_TEST(log, "Assuming removal_tid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); return true; } @@ -107,30 +107,30 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction return false; } - maxtid = tid; + removal_tid = tid; tryWriteEventToSystemLog(log, TransactionsInfoLogElement::LOCK_PART, tid, context); return true; } void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) { - LOG_TEST(log, "Unlocking maxtid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); + LOG_TEST(log, "Unlocking removal_tid by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); assert(!tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); - TIDHash locked_by = maxtid_lock.load(); + 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 maxtid, it's a bug. Current: {} {}, actual: {} {}", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unlock removal_tid, it's a bug. Current: {} {}, actual: {} {}", max_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID); }; if (locked_by != max_lock_value) throw_cannot_unlock(); - maxtid = Tx::EmptyTID; - bool unlocked = maxtid_lock.compare_exchange_strong(locked_by, 0); + removal_tid = Tx::EmptyTID; + bool unlocked = removal_tid_lock.compare_exchange_strong(locked_by, 0); if (!unlocked) throw_cannot_unlock(); @@ -139,15 +139,15 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionI bool VersionMetadata::isMaxTIDLocked() const { - return maxtid_lock.load() != 0; + return removal_tid_lock.load() != 0; } -void VersionMetadata::setMinTID(const TransactionID & tid, const TransactionInfoContext & context) +void VersionMetadata::setCreationTID(const TransactionID & tid, const TransactionInfoContext & context) { /// TODO Transactions: initialize it in constructor on part creation and remove this method /// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times - assert(mintid.isEmpty() || mintid == tid); - const_cast(mintid) = tid; + assert(creation_tid.isEmpty() || creation_tid == tid); + const_cast(creation_tid) = tid; tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, context); } @@ -159,18 +159,18 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) { - assert(!mintid.isEmpty()); - CSN min = mincsn.load(std::memory_order_relaxed); - TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); - CSN max = maxcsn.load(std::memory_order_relaxed); + assert(!creation_tid.isEmpty()); + CSN min = creation_csn.load(std::memory_order_relaxed); + TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); + CSN max = removal_csn.load(std::memory_order_relaxed); - //LOG_TEST(log, "Checking if mintid {} mincsn {} maxtidhash {} maxcsn {} visible for {} {}", mintid, min, max_lock, max, snapshot_version, current_tid); + //LOG_TEST(log, "Checking if creation_tid {} creation_csn {} removal_tidhash {} removal_csn {} visible for {} {}", creation_tid, min, max_lock, max, snapshot_version, current_tid); - [[maybe_unused]] bool had_mincsn = min; - [[maybe_unused]] bool had_maxtid = max_lock; - [[maybe_unused]] bool had_maxcsn = max; - assert(!had_maxcsn || had_maxtid); - assert(!had_maxcsn || had_mincsn); + [[maybe_unused]] bool had_creation_csn = min; + [[maybe_unused]] bool had_removal_tid = max_lock; + [[maybe_unused]] bool had_removal_csn = max; + assert(!had_removal_csn || had_removal_tid); + assert(!had_removal_csn || had_creation_csn); assert(min == Tx::UnknownCSN || min == Tx::PrehistoricCSN || Tx::MaxReservedCSN < min); assert(max == Tx::UnknownCSN || max == Tx::PrehistoricCSN || Tx::MaxReservedCSN < max); @@ -195,44 +195,44 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return true; if (min && min <= snapshot_version && max && snapshot_version < max) return true; - if (!current_tid.isEmpty() && mintid == current_tid) + if (!current_tid.isEmpty() && creation_tid == current_tid) return true; /// End of fast path. - /// Data part has mintid/maxtid, but does not have mincsn/maxcsn. + /// 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_mincsn || (had_maxtid && !had_maxcsn)); - assert(current_tid.isEmpty() || (mintid != current_tid && max_lock != current_tid.getHash())); + assert(!had_creation_csn || (had_removal_tid && !had_removal_csn)); + assert(current_tid.isEmpty() || (creation_tid != current_tid && max_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 <= mintid.start_csn) + if (snapshot_version <= creation_tid.start_csn) return false; - /// Check if mintid/maxtid transactions are committed and write CSNs + /// Check if creation_tid/removal_tid transactions are committed and write CSNs /// TODO Transactions: we probably need some optimizations here /// to avoid some CSN lookups or make the lookups cheaper. /// NOTE: Old enough committed parts always have written CSNs, /// so we can determine their visibility through fast path. /// But for long-running writing transactions we will always do /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. - min = TransactionLog::instance().getCSN(mintid); + min = TransactionLog::instance().getCSN(creation_tid); if (!min) 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). - mincsn.store(min, std::memory_order_relaxed); + creation_csn.store(min, std::memory_order_relaxed); if (max_lock) { max = TransactionLog::instance().getCSN(max_lock); if (max) - maxcsn.store(max, std::memory_order_relaxed); + removal_csn.store(max, std::memory_order_relaxed); } return min <= snapshot_version && (!max || snapshot_version < max); @@ -240,7 +240,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) { - CSN min = mincsn.load(std::memory_order_relaxed); + CSN min = creation_csn.load(std::memory_order_relaxed); /// We can safely remove part if its creation was rolled back if (min == Tx::RolledBackCSN) return true; @@ -248,9 +248,9 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) if (!min) { /// Cannot remove part if its creation not committed yet - min = TransactionLog::instance().getCSN(mintid); + min = TransactionLog::instance().getCSN(creation_tid); if (min) - mincsn.store(min, std::memory_order_relaxed); + creation_csn.store(min, std::memory_order_relaxed); else return false; } @@ -259,18 +259,18 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) if (oldest_snapshot_version < min) return false; - TIDHash max_lock = maxtid_lock.load(std::memory_order_relaxed); + TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); /// Part is active if (!max_lock) return false; - CSN max = maxcsn.load(std::memory_order_relaxed); + CSN max = removal_csn.load(std::memory_order_relaxed); if (!max) { /// Part removal is not committed yet max = TransactionLog::instance().getCSN(max_lock); if (max) - maxcsn.store(max, std::memory_order_relaxed); + removal_csn.store(max, std::memory_order_relaxed); else return false; } @@ -279,26 +279,31 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) return max <= 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::write(WriteBuffer & buf) const { writeCString("version: 1", buf); - writeCString("\nmintid: ", buf); - TransactionID::write(mintid, buf); - if (CSN min = mincsn.load()) + writeCString("\n" CREATION_TID_STR, buf); + TransactionID::write(creation_tid, buf); + if (CSN min = creation_csn.load()) { - writeCString("\nmincsn: ", buf); + writeCString("\n" CREATION_CSN_STR, buf); writeText(min, buf); } - if (maxtid_lock) + if (removal_tid_lock) { - assert(!maxtid.isEmpty()); - assert(maxtid.getHash() == maxtid_lock); - writeCString("\nmaxtid: ", buf); - TransactionID::write(maxtid, buf); - if (CSN max = maxcsn.load()) + assert(!removal_tid.isEmpty()); + assert(removal_tid.getHash() == removal_tid_lock); + writeCString("\n" REMOVAL_TID_STR, buf); + TransactionID::write(removal_tid, buf); + if (CSN max = removal_csn.load()) { - writeCString("\nmaxcsn: ", buf); + writeCString("\n" REMOVAL_CSN_STR, buf); writeText(max, buf); } } @@ -306,23 +311,27 @@ void VersionMetadata::write(WriteBuffer & buf) const 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("\nmintid: ", buf); - mintid = TransactionID::read(buf); + assertString("\n" CREATION_TID_STR, buf); + creation_tid = TransactionID::read(buf); if (buf.eof()) return; String name; - constexpr size_t size = 8; name.resize(size); assertChar('\n', buf); buf.readStrict(name.data(), size); - if (name == "mincsn: ") + if (name == CREATION_CSN_STR) { UInt64 min; readText(min, buf); - mincsn = min; + creation_csn = min; if (buf.eof()) return; @@ -330,10 +339,10 @@ void VersionMetadata::read(ReadBuffer & buf) buf.readStrict(name.data(), size); } - if (name == "maxtid: ") + if (name == REMOVAL_TID_STR) { - maxtid = TransactionID::read(buf); - maxtid_lock = maxtid.getHash(); + removal_tid = TransactionID::read(buf); + removal_tid_lock = removal_tid.getHash(); if (buf.eof()) return; @@ -341,13 +350,13 @@ void VersionMetadata::read(ReadBuffer & buf) buf.readStrict(name.data(), size); } - if (name == "maxcsn: ") + if (name == REMOVAL_CSN_STR) { - if (maxtid.isEmpty()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Found maxcsn in metadata file, but maxtid is {}", maxtid); + if (removal_tid.isEmpty()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Found removal_csn in metadata file, but removal_tid is {}", removal_tid); UInt64 max; readText(max, buf); - maxcsn = max; + removal_csn = max; } assertEOF(buf); diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 6065bbe4ed1..32607a521a2 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -21,19 +21,19 @@ struct TransactionInfoContext struct VersionMetadata { - TransactionID mintid = Tx::EmptyTID; - TransactionID maxtid = Tx::EmptyTID; + TransactionID creation_tid = Tx::EmptyTID; + TransactionID removal_tid = Tx::EmptyTID; - std::atomic maxtid_lock = 0; + std::atomic removal_tid_lock = 0; - std::atomic mincsn = Tx::UnknownCSN; - std::atomic maxcsn = Tx::UnknownCSN; + std::atomic creation_csn = Tx::UnknownCSN; + std::atomic removal_csn = Tx::UnknownCSN; bool isVisible(const MergeTreeTransaction & txn); bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); - TransactionID getMinTID() const { return mintid; } - TransactionID getMaxTID() const; + TransactionID getCreationTID() const { return creation_tid; } + TransactionID getRemovalTID() const; bool tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr); void lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); @@ -42,7 +42,7 @@ struct VersionMetadata bool isMaxTIDLocked() const; /// It can be called only from MergeTreeTransaction or on server startup - void setMinTID(const TransactionID & tid, const TransactionInfoContext & context); + void setCreationTID(const TransactionID & tid, const TransactionInfoContext & context); bool canBeRemoved(Snapshot oldest_snapshot_version); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ea9e61578db..d039daa3473 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1104,8 +1104,8 @@ void IMergeTreeDataPart::loadColumns(bool require) void IMergeTreeDataPart::storeVersionMetadata() const { - assert(!versions.mintid.isEmpty()); - if (versions.mintid.isPrehistoric() && (versions.maxtid.isEmpty() || versions.maxtid.isPrehistoric())) + assert(!version.creation_tid.isEmpty()); + if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) return; String version_file_name = fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME; @@ -1113,7 +1113,7 @@ void IMergeTreeDataPart::storeVersionMetadata() const DiskPtr disk = volume->getDisk(); { auto out = volume->getDisk()->writeFile(tmp_version_file_name, 4096, WriteMode::Rewrite); - versions.write(*out); + version.write(*out); out->finalize(); out->sync(); } @@ -1145,7 +1145,7 @@ try if (disk->exists(version_file_name)) { auto buf = openForReading(disk, version_file_name); - versions.read(*buf); + version.read(*buf); if (disk->exists(tmp_version_file_name)) remove_tmp_file(); return; @@ -1168,16 +1168,16 @@ try /// We do not have version metadata and transactions history for old parts, /// so let's consider that such parts were created by some ancient transaction /// and were committed with some prehistoric CSN. - versions.setMinTID(Tx::PrehistoricTID, txn_context); - versions.mincsn = Tx::PrehistoricCSN; + version.setCreationTID(Tx::PrehistoricTID, txn_context); + 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. - versions.setMinTID(Tx::DummyTID, txn_context); - versions.mincsn = Tx::RolledBackCSN; + version.setCreationTID(Tx::DummyTID, txn_context); + version.creation_csn = Tx::RolledBackCSN; remove_tmp_file(); } catch (Exception & e) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 44b4eff9927..f65d642dab8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -317,7 +317,7 @@ public: CompressionCodecPtr default_codec; - mutable VersionMetadata versions; + mutable VersionMetadata version; /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2c4acddd10b..1ef8619d240 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1314,55 +1314,55 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { const DataPartPtr & part = *iter; part->loadVersionMetadata(); - VersionMetadata & versions = part->versions; + VersionMetadata & versions = part->version; /// Check if CSNs were witten after committing transaction, update and write if needed. bool versions_updated = false; - if (!versions.mintid.isEmpty() && !part->versions.mincsn) + if (!versions.creation_tid.isEmpty() && !part->version.creation_csn) { - auto min = TransactionLog::instance().getCSN(versions.mintid); + auto min = TransactionLog::instance().getCSN(versions.creation_tid); if (!min) { /// Transaction that created this part was not committed. Remove part. min = Tx::RolledBackCSN; } - LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has mintid={}, setting mincsn={}", - part->name, versions.mintid, min); - versions.mincsn = min; + LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has creation_tid={}, setting creation_csn={}", + part->name, versions.creation_tid, min); + versions.creation_csn = min; versions_updated = true; } - if (!versions.maxtid.isEmpty() && !part->versions.maxcsn) + if (!versions.removal_tid.isEmpty() && !part->version.removal_csn) { - auto max = TransactionLog::instance().getCSN(versions.maxtid); + auto max = TransactionLog::instance().getCSN(versions.removal_tid); if (max) { - LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has maxtid={}, setting maxcsn={}", - part->name, versions.maxtid, max); - versions.maxcsn = max; + LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has removal_tid={}, setting removal_csn={}", + part->name, versions.removal_tid, max); + versions.removal_csn = max; } else { - /// Transaction that tried to remove this part was not committed. Clear maxtid. - LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: clearing maxtid={}", - part->name, versions.maxtid); - versions.unlockMaxTID(versions.maxtid, TransactionInfoContext{getStorageID(), part->name}); + /// 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, versions.removal_tid); + versions.unlockMaxTID(versions.removal_tid, TransactionInfoContext{getStorageID(), part->name}); } versions_updated = true; } /// Sanity checks - bool csn_order = !versions.maxcsn || versions.mincsn <= versions.maxcsn; - bool min_start_csn_order = versions.mintid.start_csn <= versions.mincsn; - bool max_start_csn_order = versions.maxtid.start_csn <= versions.maxcsn; - bool mincsn_known = versions.mincsn; - if (!csn_order || !min_start_csn_order || !max_start_csn_order || !mincsn_known) + bool csn_order = !versions.removal_csn || versions.creation_csn <= versions.removal_csn; + bool min_start_csn_order = versions.creation_tid.start_csn <= versions.creation_csn; + bool max_start_csn_order = versions.removal_tid.start_csn <= versions.removal_csn; + bool creation_csn_known = versions.creation_csn; + if (!csn_order || !min_start_csn_order || !max_start_csn_order || !creation_csn_known) throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} has invalid versions metadata: {}", part->name, versions.toString()); if (versions_updated) part->storeVersionMetadata(); /// Deactivate part if creation was not committed or if removal was. - if (versions.mincsn == Tx::RolledBackCSN || versions.maxcsn) + if (versions.creation_csn == Tx::RolledBackCSN || versions.removal_csn) { auto next_it = std::next(iter); deactivate_part(iter); @@ -1527,15 +1527,19 @@ 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->versions.canBeRemoved(TransactionLog::instance().getOldestSnapshot())) + if (!part->version.canBeRemoved(TransactionLog::instance().getOldestSnapshot())) 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); } @@ -1626,7 +1630,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)) @@ -1693,7 +1697,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re size_t MergeTreeData::clearOldWriteAheadLogs() { - DataPartsVector parts = getDataPartsVector(); + DataPartsVector parts = getDataPartsVectorForInternalUsage(); std::vector> all_block_numbers_on_disk; std::vector> block_numbers_on_disk; @@ -1756,14 +1760,14 @@ 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) continue; - /// Do not try to drop empty part if it's locked by some transaction and do not try to drop uncommitted parts. - if (part->versions.maxtid_lock.load() || !part->versions.isVisible(TransactionLog::instance().getLatestSnapshot())) + /// Do not try to drop uncommitted parts. + if (!part->version.isVisible(TransactionLog::instance().getLatestSnapshot())) continue; LOG_TRACE(log, "Will drop empty part {}", part->name); @@ -2241,7 +2245,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()) @@ -2630,8 +2634,8 @@ 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 maxtid, - /// because it's too optimistic. We should lock maxtid of covered parts at the beginning of operation. + /// 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) @@ -2704,7 +2708,7 @@ void MergeTreeData::removePartsFromWorkingSet(MergeTreeTransaction * txn, const for (const DataPartPtr & part : remove) { - if (part->versions.mincsn != Tx::RolledBackCSN) + if (part->version.creation_csn != Tx::RolledBackCSN) MergeTreeTransaction::removeOldPart(shared_from_this(), part, txn); if (part->getState() == IMergeTreeDataPart::State::Active) @@ -2829,7 +2833,7 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - if (!part->versions.isVisible(tid.start_csn, tid)) + if (!part->version.isVisible(tid.start_csn, tid)) continue; parts_to_remove.emplace_back(part); @@ -3700,7 +3704,7 @@ BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_co { DataPartsVector data_parts; if (partitions.empty()) - data_parts = getDataPartsVector(); + data_parts = getVisibleDataPartsVector(local_context); else data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(partitions, local_context)); return backupDataParts(data_parts); @@ -3911,12 +3915,12 @@ DataPartsVector MergeTreeData::getVisibleDataPartsVector(ContextPtr local_contex DataPartsVector res; if (const auto * txn = local_context->getCurrentTransaction().get()) { - res = getDataPartsVector({DataPartState::Active, DataPartState::Outdated}); + res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated}); filterVisibleDataParts(res, txn->getSnapshot(), txn->tid); } else { - res = getDataPartsVector(); + res = getDataPartsVectorForInternalUsage(); } return res; } @@ -3926,19 +3930,19 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me DataPartsVector res; if (txn) { - res = getDataPartsVector({DataPartState::Active, DataPartState::Outdated}); + res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated}); filterVisibleDataParts(res, txn->getSnapshot(), txn->tid); } else { - res = getDataPartsVector(); + res = getDataPartsVectorForInternalUsage(); } return res; } MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(Snapshot snapshot_version, TransactionID current_tid) const { - auto res = getDataPartsVector({DataPartState::Active, DataPartState::Outdated}); + auto res = getDataPartsVectorForInternalUsage({DataPartState::Active, DataPartState::Outdated}); filterVisibleDataParts(res, snapshot_version, current_tid); return res; } @@ -3953,7 +3957,7 @@ void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts String visible_parts_str; while (it <= it_last) { - if ((*it)->versions.isVisible(snapshot_version, current_tid)) + if ((*it)->version.isVisible(snapshot_version, current_tid)) { visible_parts_str += (*it)->name; visible_parts_str += " "; @@ -4002,7 +4006,7 @@ std::set MergeTreeData::getPartitionIdsAffectedByCommands( } -MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector( +MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage( const DataPartStates & affordable_states, DataPartStateVector * out_states, bool require_projection_parts) const { DataPartsVector res; @@ -4410,9 +4414,9 @@ 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( @@ -4464,7 +4468,7 @@ void MergeTreeData::Transaction::rollback() DataPartPtr covering_part; DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, lock); for (auto & covered : covered_parts) - covered->versions.unlockMaxTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name}); + covered->version.unlockMaxTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name}); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 31aac068ed8..760c76577c8 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -433,7 +433,7 @@ public: /// Returns sorted list of the parts with specified states /// out_states will contain snapshot of each part state - DataPartsVector getDataPartsVector( + 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) @@ -441,8 +441,7 @@ public: /// Returns parts in Active state DataParts getDataPartsForInternalUsage() const; - - DataPartsVector getDataPartsVector() const; + DataPartsVector getDataPartsVectorForInternalUsage() const; void filterVisibleDataParts(DataPartsVector & maybe_visible_parts, Snapshot snapshot_version, TransactionID current_tid) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 88b05da3175..e1e894c9a16 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -129,7 +129,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( { /// NOTE It will contain uncommitted parts and future parts. /// But It's ok since merge predicate allows to include in range visible parts only. - MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); + MergeTreeData::DataPartsVector data_parts = data.getDataPartsVectorForInternalUsage(); const auto data_settings = data.getSettings(); auto metadata_snapshot = data.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 744a38f5e52..1e7a5cb17ba 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -61,6 +61,12 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP auto out = disk->writeFile(path_prefix + file_name); *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; + if (!tid.isPrehistoric()) + { + *out << "tid: "; + TransactionID::write(tid, *out); + *out << "\n"; + } *out << "commands: "; commands.writeText(*out); *out << "\n"; @@ -112,6 +118,14 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); + assertNotEOF(*buf); + if (*(buf->position()) == 't') + { + *buf >> "tid: "; + tid = TransactionID::read(*buf); + *buf >> "\n"; + } + *buf >> "commands: "; commands.readText(*buf); *buf >> "\n"; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 190fc0d30a0..d0170dea404 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -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; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6861599a1ac..15f250bc9cd 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -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(UInt64, 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) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a467e474103..36d92962281 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -249,7 +249,7 @@ std::optional StorageMergeTree::totalRows(const Settings &) const std::optional StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const { - auto parts = getDataPartsVector({DataPartState::Active}); + auto parts = getVisibleDataPartsVector(local_context); return totalRowsByPartitionPredicateImpl(query_info, local_context, parts); } @@ -587,7 +587,7 @@ std::optional StorageMergeTree::getIncompleteMutationsS else if (txn) { /// Part is locked by concurrent transaction, most likely it will never be mutated - TIDHash part_locked = data_part->versions.maxtid_lock.load(); + TIDHash part_locked = data_part->version.removal_tid_lock.load(); if (part_locked && part_locked != mutation_entry.tid.getHash()) { result.latest_failed_part = data_part->name; @@ -712,6 +712,19 @@ void StorageMergeTree::loadMutations() MergeTreeMutationEntry entry(disk, relative_data_path, it->name()); UInt64 block_number = entry.block_number; LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); + + if (!entry.tid.isPrehistoric()) + { + if (!TransactionLog::instance().getCSN(entry.tid)) + { + LOG_DEBUG(log, "Mutation entry {} was created by transaction {}, but it was not committed. Removing mutation entry", + it->name(), entry.tid); + disk->removeFile(it->path()); + continue; + } + /// Transaction is committed => mutation is finished, but let's load it anyway (so it will be shown in system.mutations) + } + auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); @@ -756,9 +769,9 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( { /// Cannot merge parts if some of them is not visible in current snapshot /// TODO We can use simplified visibility rules (without CSN lookup) here - if (left && !left->versions.isVisible(tx->getSnapshot(), Tx::EmptyTID)) + if (left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; - if (right && !right->versions.isVisible(tx->getSnapshot(), Tx::EmptyTID)) + if (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; } @@ -939,7 +952,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( } auto mutations_end_it = current_mutations_by_version.end(); - for (const auto & part : getDataPartsVector()) + for (const auto & part : getDataPartsVectorForInternalUsage()) { if (currently_merging_mutating_parts.count(part)) continue; @@ -961,8 +974,6 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( TransactionID first_mutation_tid = mutations_begin_it->second.tid; MergeTreeTransactionPtr txn = tryGetTransactionForMutation(mutations_begin_it->second, log); - /// FIXME Transactions: we should kill mutations, but cannot do it here while holding currently_processing_in_background_mutex - /// TIDs are not persistent, so it cannot happen for now assert(txn || first_mutation_tid.isPrehistoric()); if (txn) @@ -970,7 +981,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( /// Mutate visible parts only /// NOTE Do not mutate visible parts in Outdated state, because it does not make sense: /// mutation will fail anyway due to serialization error. - if (!part->versions.isVisible(*txn)) + if (!part->version.isVisible(*txn)) continue; } @@ -1257,7 +1268,7 @@ std::vector StorageMergeTree::getSortedPa std::unique_lock & currently_processing_in_background_mutex_lock) const { std::vector part_versions_with_names; - auto data_parts = getDataPartsVector(); + auto data_parts = getDataPartsVectorForInternalUsage(); part_versions_with_names.reserve(data_parts.size()); for (const auto & part : data_parts) part_versions_with_names.emplace_back(PartVersionWithName{ @@ -1290,7 +1301,7 @@ bool StorageMergeTree::optimize( String disable_reason; if (!partition && final) { - DataPartsVector data_parts = getDataPartsVector(); + DataPartsVector data_parts = getVisibleDataPartsVector(local_context); std::unordered_set partition_ids; for (const DataPartPtr & part : data_parts) @@ -1681,7 +1692,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } else - data_parts = getDataPartsVector(); + data_parts = getVisibleDataPartsVector(local_context); for (auto & part : data_parts) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 898ec7d648a..2bd1598593f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3034,7 +3034,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) { /// Choose a part to mutate. - DataPartsVector data_parts = getDataPartsVector(); + DataPartsVector data_parts = getDataPartsVectorForInternalUsage(); for (const auto & part : data_parts) { if (part->getBytesOnDisk() > max_source_part_size_for_mutation) @@ -4418,7 +4418,7 @@ bool StorageReplicatedMergeTree::optimize( bool assigned = false; if (!partition && final) { - DataPartsVector data_parts = getDataPartsVector(); + DataPartsVector data_parts = getVisibleDataPartsVector(query_context); std::unordered_set partition_ids; for (const DataPartPtr & part : data_parts) @@ -7002,7 +7002,7 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } else - data_parts = getDataPartsVector(); + data_parts = getVisibleDataPartsVector(local_context); for (auto & part : data_parts) { diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 898f6ef070f..f34acb84a76 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -84,10 +84,10 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"projections", std::make_shared(std::make_shared())}, {"visible", std::make_shared()}, - {"mintid", getTransactionIDDataType()}, - {"maxtid", getTransactionIDDataType()}, - {"mincsn", std::make_shared()}, - {"maxcsn", std::make_shared()}, + {"creation_tid", getTransactionIDDataType()}, + {"removal_tid", getTransactionIDDataType()}, + {"creation_csn", std::make_shared()}, + {"removal_csn", std::make_shared()}, } ) { @@ -283,7 +283,7 @@ void StorageSystemParts::processNextStorage( { auto txn = context->getCurrentTransaction(); if (txn) - columns[res_index++]->insert(part->versions.isVisible(*txn)); + columns[res_index++]->insert(part->version.isVisible(*txn)); else columns[res_index++]->insert(part_state == State::Active); } @@ -294,13 +294,13 @@ void StorageSystemParts::processNextStorage( }; if (columns_mask[src_index++]) - columns[res_index++]->insert(get_tid_as_field(part->versions.mintid)); + columns[res_index++]->insert(get_tid_as_field(part->version.creation_tid)); if (columns_mask[src_index++]) - columns[res_index++]->insert(get_tid_as_field(part->versions.getMaxTID())); + columns[res_index++]->insert(get_tid_as_field(part->version.getRemovalTID())); if (columns_mask[src_index++]) - columns[res_index++]->insert(part->versions.mincsn.load(std::memory_order_relaxed)); + columns[res_index++]->insert(part->version.creation_csn.load(std::memory_order_relaxed)); if (columns_mask[src_index++]) - columns[res_index++]->insert(part->versions.maxcsn.load(std::memory_order_relaxed)); + columns[res_index++]->insert(part->version.removal_csn.load(std::memory_order_relaxed)); /// _state column should be the latest. /// Do not use part->getState*, it can be changed from different thread diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index cee20525f44..72f6329b0cf 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -57,12 +57,12 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat { /// If has_state_column is requested, return all states. if (!has_state_column) - return data->getDataPartsVector({State::Active, State::Outdated}, &state, require_projection_parts); + return data->getDataPartsVectorForInternalUsage({State::Active, State::Outdated}, &state, require_projection_parts); return data->getAllDataPartsVector(&state, require_projection_parts); } - return data->getDataPartsVector({State::Active}, &state, require_projection_parts); + return data->getDataPartsVectorForInternalUsage({State::Active}, &state, require_projection_parts); } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index b0d57ad651d..1087eb83f13 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,6 +1,9 @@ <_enable_experimental_mvcc_prototype_test_helper_dev>42 + + + system transactions_info_log
diff --git a/tests/integration/test_transactions/configs/transactions.xml b/tests/integration/test_transactions/configs/transactions.xml index 620515cd7bd..f48105fc98c 100644 --- a/tests/integration/test_transactions/configs/transactions.xml +++ b/tests/integration/test_transactions/configs/transactions.xml @@ -3,6 +3,7 @@ 100500 + 0 diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 97f1fa7b1f6..60806e91027 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -27,6 +27,8 @@ def test_rollback_unfinished_on_restart(start_cluster): # it will hold a snapshot and avoid parts cleanup tx(0, 'begin transaction') + tx(4, 'begin transaction') + tx(1, 'begin transaction') tid1 = tx(1, 'select transactionID()').strip() tx(1, "alter table mt drop partition id '1'") @@ -41,38 +43,51 @@ def test_rollback_unfinished_on_restart(start_cluster): csn1 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid1)).strip() csn2 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid2)).strip() + # insert a part before starting mutation and check that it will not be mutated + tx(4, 'insert into mt values (9, 90)') + + # check that uncommitted mutation will be rolled back on restart tx(1, 'begin transaction') tid3 = tx(1, 'select transactionID()').strip() tx(1, 'insert into mt values (5, 50)') tx(1, "alter table mt update m = m+n in partition id '1' where 1") + # check that uncommitted merge will be rolled back on restart tx(2, 'begin transaction') tid4 = tx(2, 'select transactionID()').strip() tx(2, "optimize table mt partition id '0' final settings optimize_throw_if_noop = 1") + # check that uncommitted insert will be rolled back on restart tx(3, 'begin transaction') tid5 = tx(3, 'select transactionID()').strip() tx(3, 'insert into mt values (6, 70)') + tid6 = tx(4, 'select transactionID()').strip() + tx(4, 'commit') + node.query('system flush logs') + csn6 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid6)).strip() + node.restart_clickhouse(kill=True) - assert node.query('select *, _part from mt order by n') == '2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n' - res = node.query("select name, active, mintid, 'csn' || toString(mincsn), maxtid, 'csn' || toString(maxcsn) from system.parts where table='mt' order by name") + assert node.query('select *, _part from mt order by n') == '2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n9\t90\t1_5_5_0\n' + res = node.query("select name, active, creation_tid, 'csn' || toString(creation_csn), removal_tid, 'csn' || toString(removal_csn) from system.parts where table='mt' order by name") res = res.replace(tid0, 'tid0') res = res.replace(tid1, 'tid1').replace('csn' + csn1, 'csn_1') res = res.replace(tid2, 'tid2').replace('csn' + csn2, 'csn_2') res = res.replace(tid3, 'tid3') res = res.replace(tid4, 'tid4') res = res.replace(tid5, 'tid5') + res = res.replace(tid6, 'tid6').replace('csn' + csn6, 'csn_6') assert res == "0_2_2_0\t1\ttid0\tcsn1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "0_2_4_1\t0\ttid4\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "0_7_7_0\t0\ttid5\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "0_8_8_0\t0\ttid5\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ "1_1_1_0\t0\ttid0\tcsn1\ttid1\tcsn_1\n" \ "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_3_3_0_6\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_5_5_0\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_5_5_0_6\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" + "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_6_6_0\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ + "1_6_6_0_7\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 89a3a08f57b..9754332a6a2 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,6 +1,6 @@ drop table if exists txn_counters; -create table txn_counters (n Int64, mintid DEFAULT transactionID()) engine=MergeTree order by n; +create table txn_counters (n Int64, creation_tid DEFAULT transactionID()) engine=MergeTree order by n; insert into txn_counters(n) values (1); select transactionID(); @@ -10,14 +10,14 @@ system stop merges txn_counters; begin transaction; insert into txn_counters(n) values (2); -select 1, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select 2, name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 1, system.parts.name, txn_counters.creation_tid = system.parts.creation_tid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 2, name, creation_csn, removal_tid, removal_csn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; rollback; begin transaction; insert into txn_counters(n) values (3); -select 3, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select 4, name, mincsn, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 3, system.parts.name, txn_counters.creation_tid = system.parts.creation_tid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 4, name, creation_csn, removal_tid, removal_csn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; select 5, transactionID().3 == serverUUID(); commit; @@ -26,8 +26,8 @@ attach table txn_counters; begin transaction; insert into txn_counters(n) values (4); -select 6, system.parts.name, txn_counters.mintid = system.parts.mintid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select 7, name, maxtid, maxcsn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 6, system.parts.name, txn_counters.creation_tid = system.parts.creation_tid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 7, name, removal_tid, removal_csn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; select 8, transactionID().3 == serverUUID(); commit; @@ -43,4 +43,4 @@ from system.transactions_info_log where tid in (select tid from system.transactions_info_log where database=currentDatabase() and table='txn_counters' and not (tid.1=1 and tid.2=1)) or (database=currentDatabase() and table='txn_counters') order by event_time; -drop table txn_counters; +--drop table txn_counters; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index f80aa0c2c49..8d1488c210d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -33,7 +33,7 @@ CREATE TABLE system.numbers\n(\n `number` UInt64\n)\nENGINE = SystemNumbers() CREATE TABLE system.numbers_mt\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `visible` UInt8,\n `mintid` Tuple(UInt64, UInt64, UUID),\n `maxtid` Tuple(UInt64, UInt64, UUID),\n `mincsn` UInt64,\n `maxcsn` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `visible` UInt8,\n `creation_tid` Tuple(UInt64, UInt64, UUID),\n `removal_tid` Tuple(UInt64, UInt64, UUID),\n `creation_csn` UInt64,\n `removal_csn` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `serialization_kind` String,\n `subcolumns.names` Array(String),\n `subcolumns.types` Array(String),\n `subcolumns.serializations` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'POSTGRES\' = -128, \'SQLITE\' = -127, \'ODBC\' = -126, \'JDBC\' = -125, \'HDFS\' = -124, \'S3\' = -123, \'SOURCES\' = -122, \'ALL\' = -121, \'NONE\' = -120, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToSymbol\' = 120, \'demangle\' = 121, \'INTROSPECTION\' = 122, \'FILE\' = 123, \'URL\' = 124, \'REMOTE\' = 125, \'MONGO\' = 126, \'MYSQL\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'POSTGRES\' = -128, \'SQLITE\' = -127, \'ODBC\' = -126, \'JDBC\' = -125, \'HDFS\' = -124, \'S3\' = -123, \'SOURCES\' = -122, \'ALL\' = -121, \'NONE\' = -120, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToSymbol\' = 120, \'demangle\' = 121, \'INTROSPECTION\' = 122, \'FILE\' = 123, \'URL\' = 124, \'REMOTE\' = 125, \'MONGO\' = 126, \'MYSQL\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From 5fad3fdffc76e8a82f1cd882f955358ce6bc171d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Feb 2022 01:27:55 +0300 Subject: [PATCH 041/372] throw exception on non-transactional queries --- src/Core/Settings.h | 1 + src/Databases/DatabaseReplicated.cpp | 4 +++ src/Interpreters/Context.cpp | 16 ++++++++--- src/Interpreters/Context.h | 1 + src/Interpreters/IInterpreter.cpp | 21 +++++++++++++++ src/Interpreters/IInterpreter.h | 8 ++++++ src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Interpreters/InterpreterAlterQuery.h | 2 ++ src/Interpreters/InterpreterInsertQuery.cpp | 2 ++ src/Interpreters/InterpreterInsertQuery.h | 2 ++ src/Interpreters/InterpreterOptimizeQuery.cpp | 1 + src/Interpreters/InterpreterOptimizeQuery.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 16 +++++++++++ src/Interpreters/InterpreterSelectQuery.h | 2 ++ .../InterpreterSelectWithUnionQuery.h | 2 ++ src/Interpreters/InterpreterSetQuery.h | 2 ++ .../InterpreterTransactionControlQuery.cpp | 1 + .../InterpreterTransactionControlQuery.h | 2 ++ src/Interpreters/MergeTreeTransaction.h | 1 - src/Interpreters/TransactionLog.cpp | 17 +++++++++--- src/Interpreters/TransactionLog.h | 8 +++--- .../TransactionVersionMetadata.cpp | 27 +++++++++++++++---- src/Interpreters/TransactionVersionMetadata.h | 5 +++- src/Interpreters/executeDDLQueryOnCluster.cpp | 3 +++ src/Interpreters/executeQuery.cpp | 7 +++++ src/Storages/IStorage.h | 4 +++ src/Storages/MergeTree/MergeTreeData.cpp | 8 +++--- src/Storages/StorageMaterializedView.h | 1 + src/Storages/StorageMergeTree.cpp | 7 +++-- src/Storages/StorageMergeTree.h | 2 ++ src/Storages/StorageValues.h | 4 +++ src/Storages/System/StorageSystemNumbers.h | 1 + src/Storages/System/StorageSystemOne.h | 2 ++ .../0_stateless/01167_isolation_hermitage.sh | 2 +- .../01169_alter_partition_isolation_stress.sh | 3 +++ .../01172_transaction_counters.sql | 2 ++ .../01173_transaction_control_queries.sql | 18 +++++++++++++ .../01174_select_insert_isolation.sh | 1 + .../01528_clickhouse_local_prepare_parts.sh | 1 - 39 files changed, 184 insertions(+), 26 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 54307b25efa..02fce31a963 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -555,6 +555,7 @@ class IColumn; M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 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. diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 84d2edd1bb1..5b226f63b0f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -444,6 +444,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"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d19e5cf91c4..2e2d805bc8d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2977,6 +2977,19 @@ void Context::resetZooKeeperMetadataTransaction() } +void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) const +{ + int enable_mvcc_test_helper = getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0); + if (enable_mvcc_test_helper == 42) + 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. Probably it's a bug."); +} + void Context::initCurrentTransaction(MergeTreeTransactionPtr txn) { merge_tree_transaction_holder = MergeTreeTransactionHolder(txn, false); @@ -2987,9 +3000,6 @@ void Context::setCurrentTransaction(MergeTreeTransactionPtr txn) { assert(!merge_tree_transaction || !txn); assert(this == session_context.lock().get() || this == query_context.lock().get()); - int enable_mvcc_test_helper = getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0); - if (enable_mvcc_test_helper != 42) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); merge_tree_transaction = std::move(txn); if (!merge_tree_transaction) merge_tree_transaction_holder = {}; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 295d35ce6cf..9634772f575 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -879,6 +879,7 @@ 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; diff --git a/src/Interpreters/IInterpreter.cpp b/src/Interpreters/IInterpreter.cpp index af0c06e7503..84fbfee7905 100644 --- a/src/Interpreters/IInterpreter.cpp +++ b/src/Interpreters/IInterpreter.cpp @@ -1,9 +1,16 @@ #include #include #include +#include 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()); +} + } diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 665a46190fd..74a568c5cba 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -3,6 +3,7 @@ #include #include #include +#include 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; }; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index d01f2b05567..df1d6b8c92c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -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); diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index 9494a400e7b..c6648ff9e7e 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -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; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index d340308122f..ef926f45fbe 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -263,6 +263,8 @@ BlockIO InterpreterInsertQuery::execute() QueryPipelineBuilder pipeline; StoragePtr table = getTable(query); + checkStorageSupportsTransactionsIfNeeded(table, getContext()); + StoragePtr inner_table; if (const auto * mv = dynamic_cast(table.get())) inner_table = mv->getTargetTable(); diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 93de92a0680..51a3f0384aa 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -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; diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index f9a701a0a77..abc3a8e97f7 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -31,6 +31,7 @@ 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(); // Empty list of names means we deduplicate by all columns, but user can explicitly state which columns to use. diff --git a/src/Interpreters/InterpreterOptimizeQuery.h b/src/Interpreters/InterpreterOptimizeQuery.h index 8491fe8df49..932700e51b5 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.h +++ b/src/Interpreters/InterpreterOptimizeQuery.h @@ -18,6 +18,8 @@ public: BlockIO execute() override; + bool supportsTransactions() const override { return true; } + private: AccessRightsElements getRequiredAccess() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e3cf6751cf7..d2cfe2fd956 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -328,6 +328,22 @@ InterpreterSelectQuery::InterpreterSelectQuery( 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) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 4298cbbb794..0f9555628ff 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -104,6 +104,8 @@ public: Names getRequiredColumns() { return required_columns; } + bool supportsTransactions() const override { return true; } + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 720632e7be5..adf8540d626 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -39,6 +39,8 @@ public: virtual void ignoreWithTotals() override; + bool supportsTransactions() const override { return true; } + private: std::vector> nested_interpreters; diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index 9bd49708421..39d331100d6 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -25,6 +25,8 @@ public: */ void executeForCurrentContext(); + bool supportsTransactions() const override { return true; } + private: ASTPtr query_ptr; }; diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index cdf91712a31..7faa74bb820 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -38,6 +38,7 @@ BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr sessi 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); diff --git a/src/Interpreters/InterpreterTransactionControlQuery.h b/src/Interpreters/InterpreterTransactionControlQuery.h index fc71939502e..996fb3247a6 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.h +++ b/src/Interpreters/InterpreterTransactionControlQuery.h @@ -18,6 +18,8 @@ public: 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); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 6733dba4d8f..969db105352 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -29,7 +29,6 @@ public: const TransactionID tid; - MergeTreeTransaction() = delete; MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id); void addNewPart(const StoragePtr & storage, const DataPartPtr & new_part); diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 105a62c87f9..016955ae2f8 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -43,6 +43,8 @@ TransactionLog::TransactionLog() : log(&Poco::Logger::get("TransactionLog")) { global_context = Context::getGlobalContextInstance(); + global_context->checkTransactionsAreAllowed(); + zookeeper_path = "/test/clickhouse/txn_log"; loadLogFromZooKeeper(); @@ -334,17 +336,24 @@ MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & return it->second; } -CSN TransactionLog::getCSN(const TransactionID & tid) const +CSN TransactionLog::getCSN(const TransactionID & tid) { return getCSN(tid.getHash()); } -CSN TransactionLog::getCSN(const TIDHash & tid) const +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) const { assert(tid); assert(tid != Tx::EmptyTID.getHash()); - if (tid == Tx::PrehistoricTID.getHash()) - return Tx::PrehistoricCSN; std::lock_guard lock{mutex}; auto it = tid_to_csn.find(tid); diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index f2599a4d104..97fc52294fb 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -53,7 +53,6 @@ using ZooKeeperPtr = std::shared_ptr; class TransactionLog final : public SingletonHelper { public: - //static TransactionLog & instance(); TransactionLog(); @@ -71,12 +70,11 @@ public: void rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept; - CSN getCSN(const TransactionID & tid) const; - CSN getCSN(const TIDHash & tid) const; + static CSN getCSN(const TransactionID & tid); + static CSN getCSN(const TIDHash & tid); MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); - private: void loadLogFromZooKeeper(); void runUpdatingThread(); @@ -90,6 +88,8 @@ private: ZooKeeperPtr getZooKeeper() const; + CSN getCSNImpl(const TIDHash & tid) const; + ContextPtr global_context; Poco::Logger * log; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 4b00a2e0625..b22683855ce 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -220,7 +220,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// so we can determine their visibility through fast path. /// But for long-running writing transactions we will always do /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. - min = TransactionLog::instance().getCSN(creation_tid); + min = TransactionLog::getCSN(creation_tid); if (!min) return false; /// Part creation is not committed yet @@ -230,7 +230,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current if (max_lock) { - max = TransactionLog::instance().getCSN(max_lock); + max = TransactionLog::getCSN(max_lock); if (max) removal_csn.store(max, std::memory_order_relaxed); } @@ -238,7 +238,24 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return min <= snapshot_version && (!max || snapshot_version < max); } -bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) +bool VersionMetadata::canBeRemoved() +{ + if (creation_tid == Tx::PrehistoricTID) + { + /// Avoid access to Transaction log if transactions are not involved + + TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); + if (!max_lock) + return false; + + if (max_lock == Tx::PrehistoricTID.getHash()) + return true; + } + + return canBeRemovedImpl(TransactionLog::instance().getOldestSnapshot()); +} + +bool VersionMetadata::canBeRemovedImpl(Snapshot oldest_snapshot_version) { CSN min = creation_csn.load(std::memory_order_relaxed); /// We can safely remove part if its creation was rolled back @@ -248,7 +265,7 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) if (!min) { /// Cannot remove part if its creation not committed yet - min = TransactionLog::instance().getCSN(creation_tid); + min = TransactionLog::getCSN(creation_tid); if (min) creation_csn.store(min, std::memory_order_relaxed); else @@ -268,7 +285,7 @@ bool VersionMetadata::canBeRemoved(Snapshot oldest_snapshot_version) if (!max) { /// Part removal is not committed yet - max = TransactionLog::instance().getCSN(max_lock); + max = TransactionLog::getCSN(max_lock); if (max) removal_csn.store(max, std::memory_order_relaxed); else diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 32607a521a2..de29d9580c2 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -29,6 +29,7 @@ struct VersionMetadata std::atomic creation_csn = Tx::UnknownCSN; std::atomic removal_csn = Tx::UnknownCSN; + /// Checks if an object is visible for transaction or not. bool isVisible(const MergeTreeTransaction & txn); bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); @@ -44,7 +45,9 @@ struct VersionMetadata /// It can be called only from MergeTreeTransaction or on server startup void setCreationTID(const TransactionID & tid, const TransactionInfoContext & context); - bool canBeRemoved(Snapshot oldest_snapshot_version); + /// Checks if it's safe to remove outdated version of an object + bool canBeRemoved(); + bool canBeRemovedImpl(Snapshot oldest_snapshot_version); void write(WriteBuffer & buf) const; void read(ReadBuffer & buf); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index d3a9bcbe1e4..0a9ecf882df 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -62,6 +62,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 and INTO OUTFILE if exists ASTPtr query_ptr = query_ptr_->clone(); ASTQueryWithOutput::resetOutputASTIfExist(*query_ptr); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 10b87927ea7..93c5d47ac1f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -644,11 +644,18 @@ static std::tuple 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(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d6578efa625..ac01aec3877 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -158,6 +158,10 @@ 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. + virtual bool supportsTransactions() const { return false; } + /// Requires squashing small blocks to large for optimal storage. /// This is true for most storages that store data on disk. virtual bool prefersLargeBlocks() const { return true; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ed2b6f62df8..e3da98045f2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1320,7 +1320,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) bool versions_updated = false; if (!versions.creation_tid.isEmpty() && !part->version.creation_csn) { - auto min = TransactionLog::instance().getCSN(versions.creation_tid); + auto min = TransactionLog::getCSN(versions.creation_tid); if (!min) { /// Transaction that created this part was not committed. Remove part. @@ -1333,7 +1333,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } if (!versions.removal_tid.isEmpty() && !part->version.removal_csn) { - auto max = TransactionLog::instance().getCSN(versions.removal_tid); + auto max = TransactionLog::getCSN(versions.removal_tid); if (max) { LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has removal_tid={}, setting removal_csn={}", @@ -1527,7 +1527,7 @@ 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(TransactionLog::instance().getOldestSnapshot())) + if (!part->version.canBeRemoved()) continue; auto part_remove_time = part->remove_time.load(std::memory_order_relaxed); @@ -1767,7 +1767,7 @@ size_t MergeTreeData::clearEmptyParts() continue; /// Do not try to drop uncommitted parts. - if (!part->version.isVisible(TransactionLog::instance().getLatestSnapshot())) + if (!part->version.getCreationTID().isPrehistoric() && !part->version.isVisible(TransactionLog::instance().getLatestSnapshot())) continue; LOG_TRACE(log, "Will drop empty part {}", part->name); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 395560c1ca7..cca6ee063b5 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -26,6 +26,7 @@ public: bool supportsIndexForIn() const override { return getTargetTable()->supportsIndexForIn(); } bool supportsParallelInsert() const override { return getTargetTable()->supportsParallelInsert(); } bool supportsSubcolumns() const override { return getTargetTable()->supportsSubcolumns(); } + bool supportsTransactions() const override { return getTargetTable()->supportsTransactions(); } bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /* metadata_snapshot */) const override { auto target_table = getTargetTable(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 36d92962281..d05e00ed40d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -304,6 +304,9 @@ void StorageMergeTree::alter( ContextPtr local_context, AlterLockHolder & table_lock_holder) { + if (local_context->getCurrentTransaction() && local_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER METADATA is not supported inside transactions"); + auto table_id = getStorageID(); auto old_storage_settings = getSettings(); @@ -715,7 +718,7 @@ void StorageMergeTree::loadMutations() if (!entry.tid.isPrehistoric()) { - if (!TransactionLog::instance().getCSN(entry.tid)) + if (!TransactionLog::getCSN(entry.tid)) { LOG_DEBUG(log, "Mutation entry {} was created by transaction {}, but it was not committed. Removing mutation entry", it->name(), entry.tid); @@ -1247,7 +1250,7 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) for (size_t i = 0; i < to_delete_count; ++i) { const auto & tid = it->second.tid; - if (!tid.isPrehistoric() && !TransactionLog::instance().getCSN(tid)) + if (!tid.isPrehistoric() && !TransactionLog::getCSN(tid)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove mutation {}, because transaction {} is not committed. It's a bug", it->first, tid); mutations_to_delete.push_back(std::move(it->second)); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f63e8330335..d1cd375ac8f 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -41,6 +41,8 @@ public: bool supportsIndexForIn() const override { return true; } + bool supportsTransactions() const override { return true; } + Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index 69b2f757046..c1d638053c6 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -31,6 +31,10 @@ public: { return virtuals; } + + /// FIXME probably it should return false, but StorageValues is used in ExecutingInnerQueryFromViewTransform (whatever it is) + bool supportsTransactions() const override { return true; } + private: Block res_block; NamesAndTypesList virtuals; diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 32105bb055d..c87b102019b 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -40,6 +40,7 @@ public: bool hasEvenlyDistributedRead() const override { return true; } bool isSystemStorage() const override { return true; } + bool supportsTransactions() const override { return true; } private: bool multithreaded; diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index cc1d5e05b75..53b0d348bba 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -32,6 +32,8 @@ public: bool isSystemStorage() const override { return true; } + bool supportsTransactions() const override { return true; } + protected: explicit StorageSystemOne(const StorageID & table_id_); }; diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.sh b/tests/queries/0_stateless/01167_isolation_hermitage.sh index 09b34694da6..7f495801dd0 100755 --- a/tests/queries/0_stateless/01167_isolation_hermitage.sh +++ b/tests/queries/0_stateless/01167_isolation_hermitage.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database +# Tags: long, no-fasttest, no-replicated-database # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 4506065484d..a31030ca132 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -50,6 +50,7 @@ function thread_partition_src_to_dst() INSERT INTO src VALUES ($i, 3); INSERT INTO dst SELECT * FROM src; ALTER TABLE src DROP PARTITION ID 'all'; + SET throw_on_unsupported_query_inside_transaction=0; SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; COMMIT;" 2>&1) ||: @@ -73,6 +74,7 @@ function thread_partition_dst_to_src() INSERT INTO dst VALUES ($i, 4); INSERT INTO src SELECT * FROM dst; ALTER TABLE dst DROP PARTITION ID 'all'; + SET throw_on_unsupported_query_inside_transaction=0; SYSTEM START MERGES dst; SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=4) != (toUInt8($i/2 + 1), (select sum(number) from numbers(1, $i) where number % 2 or number=$i))) FORMAT Null; $action;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" @@ -90,6 +92,7 @@ function thread_select() SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; SELECT type, throwIf(count(n) != countDistinct(n)) FROM dst GROUP BY type FORMAT Null; -- rows inserted by thread_insert moved together + SET throw_on_unsupported_query_inside_transaction=0; SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null; -- all rows are inserted in insert_thread SELECT type, throwIf(count(n) != max(n)), throwIf(sum(n) != max(n)*(max(n)+1)/2) FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type FORMAT Null; diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 9754332a6a2..9f09347f6be 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -8,6 +8,8 @@ select transactionID(); -- stop background cleanup system stop merges txn_counters; +set throw_on_unsupported_query_inside_transaction=0; + begin transaction; insert into txn_counters(n) values (2); select 1, system.parts.name, txn_counters.creation_tid = system.parts.creation_tid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index 841e99e276b..6877a11f1f1 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -67,5 +67,23 @@ begin transaction; select 'readonly', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); commit; +begin transaction; +create table m (n int) engine=Memory; -- { serverError 48 } +commit; -- { serverError 700 } +rollback; + +create table m (n int) engine=Memory; +begin transaction; +insert into m values (1); -- { serverError 48 } +select * from m; -- { serverError 700 } +commit; -- { serverError 700 } +rollback; + +begin transaction; +select * from m; -- { serverError 48 } +commit; -- { serverError 700 } +rollback; + +drop table m; drop table mt1; drop table mt2; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 663939eb269..ec1a989d42f 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long # shellcheck disable=SC2015 diff --git a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh index 48edf41edd7..95ecbf09cf5 100755 --- a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh +++ b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: disabled CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d93c9f0be4b0589d4bc8d00acdbc35bc1a1f4614 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Feb 2022 15:05:05 +0300 Subject: [PATCH 042/372] fix build after merge --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9feb98e92fe..757bca83c19 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5407,7 +5407,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) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 34197e0d25c..d505ed1240d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7046,7 +7046,7 @@ void StorageReplicatedMergeTree::checkBrokenDisks() LOG_INFO(log, "Scanning parts to recover on broken disk {} with path {}", disk_ptr->getName(), disk_ptr->getPath()); if (!parts) - parts = std::make_unique(getDataPartsVector()); + parts = std::make_unique(getDataPartsVectorForInternalUsage()); for (auto & part : *parts) { From fe30e0f1620706c851b0ed307b06d3e59e713fdc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 3 Feb 2022 21:57:09 +0300 Subject: [PATCH 043/372] fixes --- docker/test/base/Dockerfile | 2 +- docker/test/integration/base/Dockerfile | 2 +- docker/test/util/Dockerfile | 2 +- src/Interpreters/Context.cpp | 3 +- src/Interpreters/TransactionLog.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++ .../MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 64 ++++++++++++------- src/Storages/MergeTree/MergeTreeData.h | 3 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 ++ .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 15 +++-- 12 files changed, 74 insertions(+), 35 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 11881365318..6beab2e5bb7 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #24258 +# rebuild in #33610 # docker build -t clickhouse/test-base . ARG FROM_TAG=latest FROM clickhouse/test-util:$FROM_TAG diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 2ee8d369dc2..91b26735fe5 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #24258 +# rebuild in #33610 # docker build -t clickhouse/integration-test . ARG FROM_TAG=latest FROM clickhouse/test-base:$FROM_TAG diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index c83643a4069..d9827260acb 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -1,4 +1,4 @@ -# rebuild in #24258 +# rebuild in #33610 # docker build -t clickhouse/test-util . FROM ubuntu:20.04 diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e70036e4d68..4624ed976dc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3000,7 +3000,8 @@ void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) 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. Probably it's a bug."); + "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) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 016955ae2f8..2ff72d9c39c 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -206,7 +206,7 @@ void TransactionLog::runUpdatingThread() } catch (const Coordination::Exception & e) { - LOG_ERROR(log, getCurrentExceptionMessage(true)); + tryLogCurrentException(log); /// TODO better backoff std::this_thread::sleep_for(std::chrono::milliseconds(1000)); if (Coordination::isHardwareError(e.code)) @@ -218,7 +218,7 @@ void TransactionLog::runUpdatingThread() } catch (...) { - LOG_ERROR(log, getCurrentExceptionMessage(true)); + tryLogCurrentException(log); std::this_thread::sleep_for(std::chrono::milliseconds(1000)); log_updated_event->set(); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 445bd04c7e3..b61c94508d1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -711,10 +711,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; } diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 93a7ad9eae4..702f5b318e6 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -116,7 +116,7 @@ void MergePlainMergeTreeTask::prepare() void MergePlainMergeTreeTask::finish() { new_part = merge_task->getFuture().get(); - storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, txn, nullptr); //FIXME + storage.merger_mutator.renameMergedTemporaryPart(new_part, future_part->parts, txn, nullptr); write_part_log({}); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cfe3deb0c38..ea598da684c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1315,67 +1315,78 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) removePartContributionToDataVolume(*it); }; - /// All parts are in "Committed" state after loading + /// 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 & versions = part->version; + VersionMetadata & version = part->version; + if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) + { + ++iter; + continue; + } + else + { + have_parts_with_version_metadata = true; + } /// Check if CSNs were witten after committing transaction, update and write if needed. - bool versions_updated = false; - if (!versions.creation_tid.isEmpty() && !part->version.creation_csn) + bool version_updated = false; + assert(!version.creation_tid.isEmpty()); + if (!part->version.creation_csn) { - auto min = TransactionLog::getCSN(versions.creation_tid); + auto min = TransactionLog::getCSN(version.creation_tid); if (!min) { /// Transaction that created this part was not committed. Remove part. min = Tx::RolledBackCSN; } LOG_TRACE(log, "Will fix version metadata of {} after unclean restart: part has creation_tid={}, setting creation_csn={}", - part->name, versions.creation_tid, min); - versions.creation_csn = min; - versions_updated = true; + part->name, version.creation_tid, min); + version.creation_csn = min; + version_updated = true; } - if (!versions.removal_tid.isEmpty() && !part->version.removal_csn) + if (!version.removal_tid.isEmpty() && !part->version.removal_csn) { - auto max = TransactionLog::getCSN(versions.removal_tid); + 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, versions.removal_tid, max); - versions.removal_csn = max; + part->name, version.removal_tid, max); + version.removal_csn = max; } else { /// 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, versions.removal_tid); - versions.unlockMaxTID(versions.removal_tid, TransactionInfoContext{getStorageID(), part->name}); + part->name, version.removal_tid); + version.unlockMaxTID(version.removal_tid, TransactionInfoContext{getStorageID(), part->name}); } - versions_updated = true; + version_updated = true; } /// Sanity checks - bool csn_order = !versions.removal_csn || versions.creation_csn <= versions.removal_csn; - bool min_start_csn_order = versions.creation_tid.start_csn <= versions.creation_csn; - bool max_start_csn_order = versions.removal_tid.start_csn <= versions.removal_csn; - bool creation_csn_known = versions.creation_csn; + bool csn_order = !version.removal_csn || version.creation_csn <= version.removal_csn; + 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 versions metadata: {}", part->name, versions.toString()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} has invalid version metadata: {}", part->name, version.toString()); - if (versions_updated) + if (version_updated) part->storeVersionMetadata(); /// Deactivate part if creation was not committed or if removal was. - if (versions.creation_csn == Tx::RolledBackCSN || versions.removal_csn) + if (version.creation_csn == Tx::RolledBackCSN || version.removal_csn) { auto next_it = std::next(iter); deactivate_part(iter); @@ -1387,6 +1398,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } + 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. @@ -2587,6 +2601,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; @@ -2734,6 +2751,9 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( 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); for (const DataPartPtr & part : remove) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1ffb1f98029..438b4edaefa 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -961,6 +961,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{}; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e1e894c9a16..defb69da8dc 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -52,6 +52,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. @@ -516,6 +517,11 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart 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, txn.get(), nullptr, out_transaction); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 7388ba2790d..670a9850956 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -98,7 +98,7 @@ bool MutatePlainMergeTreeTask::executeStep() if (merge_mutate_entry->txn) merge_mutate_entry->txn->onException(); String exception_message = getCurrentExceptionMessage(false); - LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message); + LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); write_part_log(ExecutionStatus::fromCurrentException()); return false; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 617722bd948..cfdc0496f23 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -773,7 +773,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( if (tx) { /// Cannot merge parts if some of them is not visible in current snapshot - /// TODO We can use simplified visibility rules (without CSN lookup) here + /// TODO Transactions: We can use simplified visibility rules (without CSN lookup) here if (left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; if (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) @@ -1106,9 +1106,14 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - /// FIXME Transactions: do not begin transaction if we don't need it - auto txn = TransactionLog::instance().beginTransaction(); - MergeTreeTransactionHolder autocommit{txn, true}; + MergeTreeTransactionHolder transaction_for_merge; + MergeTreeTransactionPtr txn; + if (transactions_enabled.load(std::memory_order_relaxed)) + { + /// TODO Transactions: avoid beginning transaction if there is nothing to merge. + txn = TransactionLog::instance().beginTransaction(); + transaction_for_merge = MergeTreeTransactionHolder{txn, /* autocommit = */ true}; + } bool has_mutations = false; { @@ -1134,7 +1139,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (merge_entry) { auto task = std::make_shared(*this, metadata_snapshot, false, Names{}, merge_entry, share_lock, common_assignee_trigger); - task->setCurrentTransaction(std::move(autocommit), std::move(txn)); + task->setCurrentTransaction(std::move(transaction_for_merge), std::move(txn)); assignee.scheduleMergeMutateTask(task); return true; } From 6a646b413e890d12fa07be40e215d00573c29b80 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Feb 2022 00:40:17 +0300 Subject: [PATCH 044/372] fix --- src/Functions/FunctionsTransactionCounters.cpp | 14 ++++++++++++-- src/Interpreters/TransactionVersionMetadata.cpp | 4 ++-- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsTransactionCounters.cpp b/src/Functions/FunctionsTransactionCounters.cpp index f553d0a8be1..f2e9d3aa84b 100644 --- a/src/Functions/FunctionsTransactionCounters.cpp +++ b/src/Functions/FunctionsTransactionCounters.cpp @@ -35,18 +35,28 @@ public: class FunctionTransactionLatestSnapshot : public FunctionConstantBase { + 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(context); } - explicit FunctionTransactionLatestSnapshot(ContextPtr context) : FunctionConstantBase(TransactionLog::instance().getLatestSnapshot(), context->isDistributed()) {} + explicit FunctionTransactionLatestSnapshot(ContextPtr context) : FunctionConstantBase(getLatestSnapshot(context), context->isDistributed()) {} }; class FunctionTransactionOldestSnapshot : public FunctionConstantBase { + 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(context); } - explicit FunctionTransactionOldestSnapshot(ContextPtr context) : FunctionConstantBase(TransactionLog::instance().getOldestSnapshot(), context->isDistributed()) {} + explicit FunctionTransactionOldestSnapshot(ContextPtr context) : FunctionConstantBase(getOldestSnapshot(context), context->isDistributed()) {} }; } diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index b22683855ce..e80d08a1944 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -52,10 +52,10 @@ TransactionID VersionMetadata::getRemovalTID() const TIDHash max_lock = removal_tid_lock.load(); if (max_lock) { - if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) - return txn->tid; if (max_lock == Tx::PrehistoricTID.getHash()) return Tx::PrehistoricTID; + if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) + return txn->tid; } if (removal_csn.load(std::memory_order_relaxed)) From 3956941aaf9765c8a98953c51f367d80e5863c24 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Feb 2022 21:18:20 +0300 Subject: [PATCH 045/372] fixes --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 4 +++- src/Interpreters/TransactionVersionMetadata.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++++-- src/Storages/MergeTree/MergeTreeData.cpp | 8 +++++--- src/Storages/MergeTree/MergeTreePartsMover.cpp | 1 + 6 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4624ed976dc..ce218b27c1d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3000,7 +3000,7 @@ void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) 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." + "however, some query or background task tried to access TransactionLog. " "If you have not enabled this feature explicitly, then it's a bug."); } diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 2ff72d9c39c..f482471c5ac 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -312,7 +312,9 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) noexcept { - LOG_TRACE(log, "Rolling back transaction {}", txn->tid); + LOG_TRACE(log, "Rolling back transaction {}{}", txn->tid, + std::uncaught_exceptions() ? fmt::format(" due to uncaught exception (code: {})", getCurrentExceptionCode()) : ""); + if (!txn->rollback()) return; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index e80d08a1944..d6734f6cd8a 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -89,6 +89,7 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInf bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id) { assert(!tid.isEmpty()); + assert(!creation_tid.isEmpty()); TIDHash max_lock_value = tid.getHash(); TIDHash expected_max_lock_value = 0; bool locked = removal_tid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b61c94508d1..1b4bec9144e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1109,7 +1109,11 @@ void IMergeTreeDataPart::storeVersionMetadata() const String tmp_version_file_name = version_file_name + ".tmp"; DiskPtr disk = volume->getDisk(); { - auto out = volume->getDisk()->writeFile(tmp_version_file_name, 4096, WriteMode::Rewrite); + /// 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 the 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(); @@ -1117,7 +1121,7 @@ void IMergeTreeDataPart::storeVersionMetadata() const SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) - sync_guard = volume->getDisk()->getDirectorySyncGuard(getFullRelativePath()); + sync_guard = disk->getDirectorySyncGuard(getFullRelativePath()); disk->moveFile(tmp_version_file_name, version_file_name); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ea598da684c..9df39016c90 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2882,9 +2882,11 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( continue; /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - if (!part->version.isVisible(tid.start_csn, tid)) - continue; + if (txn) + { + if (!part->version.isVisible(*txn)) + continue; + } parts_to_remove.emplace_back(part); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 4fc268119fc..83b58960ad1 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -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; From cbd3b4564690f45b8d7472c6d9b13096ef1bb37d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Feb 2022 22:47:17 +0300 Subject: [PATCH 046/372] add EXPLAIN CURRENT TRANSACTION --- src/Interpreters/InterpreterExplainQuery.cpp | 18 ++++++++++++++++++ src/Interpreters/InterpreterExplainQuery.h | 2 ++ src/Interpreters/TransactionLog.cpp | 2 +- .../TransactionVersionMetadata.cpp | 6 ++---- src/Interpreters/executeQuery.cpp | 3 ++- src/Parsers/ASTExplainQuery.h | 2 ++ src/Parsers/ParserExplainQuery.cpp | 7 +++++++ src/Storages/IStorage.h | 1 + .../MergeTree/MergeFromLogEntryTask.cpp | 3 +-- .../MergeTree/MergeMutateSelectedEntry.h | 1 - .../MergeTree/MutateFromLogEntryTask.cpp | 3 +-- src/Storages/StorageGenerateRandom.h | 1 + src/Storages/System/StorageSystemParts.cpp | 8 ++++---- src/Storages/System/StorageSystemZeros.h | 1 + .../0_stateless/01172_transaction_counters.sql | 2 +- 15 files changed, 44 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 37b944d72d6..6d0d23d461b 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -387,6 +388,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("", buf); + } + + break; + } } if (insert_buf) { diff --git a/src/Interpreters/InterpreterExplainQuery.h b/src/Interpreters/InterpreterExplainQuery.h index a640b1c977c..ccfe8ec88a5 100644 --- a/src/Interpreters/InterpreterExplainQuery.h +++ b/src/Interpreters/InterpreterExplainQuery.h @@ -17,6 +17,8 @@ public: static Block getSampleBlock(ASTExplainQuery::ExplainKind kind); + bool supportsTransactions() const override { return true; } + private: ASTPtr query; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index f482471c5ac..103c011b96e 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -277,7 +277,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) } else { - LOG_TEST(log, "Committing transaction {}{}", txn->tid, txn->dumpDescription()); + LOG_TEST(log, "Committing transaction {}", txn->dumpDescription()); /// TODO handle connection loss /// TODO support batching auto current_zookeeper = getZooKeeper(); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index d6734f6cd8a..15f96141fd4 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -145,11 +145,9 @@ bool VersionMetadata::isMaxTIDLocked() const void VersionMetadata::setCreationTID(const TransactionID & tid, const TransactionInfoContext & context) { - /// TODO Transactions: initialize it in constructor on part creation and remove this method - /// FIXME ReplicatedMergeTreeBlockOutputStream may add one part multiple times + /// NOTE ReplicatedMergeTreeBlockOutputStream may add one part multiple times assert(creation_tid.isEmpty() || creation_tid == tid); - const_cast(creation_tid) = tid; - + creation_tid = tid; tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, context); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 93c5d47ac1f..740e26d3d6b 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -440,7 +441,7 @@ static std::tuple executeQueryImpl( if (auto txn = context->getCurrentTransaction()) { assert(txn->getState() != MergeTreeTransaction::COMMITTED); - if (txn->getState() == MergeTreeTransaction::ROLLED_BACK && !ast->as()) + if (txn->getState() == MergeTreeTransaction::ROLLED_BACK && !ast->as() && !ast->as()) throw Exception(ErrorCodes::INVALID_TRANSACTION, "Cannot execute query: transaction is rolled back"); } diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index abed9803a7b..3f169a93bad 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -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(); diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index e072f6a14d7..2361175af0c 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -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)) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ac01aec3877..bf862a815d7 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -160,6 +160,7 @@ public: /// 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; } /// Requires squashing small blocks to large for optimal storage. diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 93ae21c79e6..6731c6a354a 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -193,7 +193,7 @@ std::pair MergeFromLogEntryT settings.memory_profiler_sample_probability, settings.max_untracked_memory); - transaction_ptr = std::make_unique(storage, nullptr); //FIXME + transaction_ptr = std::make_unique(storage, nullptr); stopwatch_ptr = std::make_unique(); merge_task = storage.merger_mutator.mergePartsToTemporaryPart( @@ -230,7 +230,6 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite /// Task is not needed merge_task.reset(); - //FIXME storage.merger_mutator.renameMergedTemporaryPart(part, parts, nullptr, transaction_ptr.get()); try diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index 488dfba7618..e84102857f7 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -39,7 +39,6 @@ struct MergeMutateSelectedEntry FutureMergedMutatedPartPtr future_part; CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; - //TransactionID mutation_tid; MergeTreeTransactionPtr txn; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = nullptr) diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index e60cf04be39..405c58f9cd1 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -63,7 +63,7 @@ std::pair MutateFromLogEntry RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = storage.getInMemoryMetadataPtr(); - transaction_ptr = std::make_unique(storage, nullptr); //FIXME + transaction_ptr = std::make_unique(storage, nullptr); future_mutated_part = std::make_shared(); future_mutated_part->name = entry.new_part_name; @@ -108,7 +108,6 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit { new_part = mutate_task->getFuture().get(); - //FIXME storage.renameTempPartAndReplace(new_part, nullptr, nullptr, transaction_ptr.get()); try diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index d11a43b1dd6..96239001f25 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -24,6 +24,7 @@ public: size_t max_block_size, unsigned num_streams) override; + bool supportsTransactions() const override { return true; } private: UInt64 max_array_length = 10; UInt64 max_string_length = 10; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index f34acb84a76..6674de06c07 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -84,10 +84,10 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"projections", std::make_shared(std::make_shared())}, {"visible", std::make_shared()}, - {"creation_tid", getTransactionIDDataType()}, - {"removal_tid", getTransactionIDDataType()}, - {"creation_csn", std::make_shared()}, - {"removal_csn", std::make_shared()}, + {"creation_tid", getTransactionIDDataType()}, + {"removal_tid", getTransactionIDDataType()}, + {"creation_csn", std::make_shared()}, + {"removal_csn", std::make_shared()}, } ) { diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index f5b2bb43117..9ee2aa1141c 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -31,6 +31,7 @@ public: bool hasEvenlyDistributedRead() const override { return true; } bool isSystemStorage() const override { return true; } + bool supportsTransactions() const override { return true; } private: bool multithreaded; diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 9f09347f6be..e77d3e16a22 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -45,4 +45,4 @@ from system.transactions_info_log where tid in (select tid from system.transactions_info_log where database=currentDatabase() and table='txn_counters' and not (tid.1=1 and tid.2=1)) or (database=currentDatabase() and table='txn_counters') order by event_time; ---drop table txn_counters; +drop table txn_counters; From ae5aa8c12de2014f801503dd833d128cd05b1337 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Feb 2022 22:50:08 +0300 Subject: [PATCH 047/372] write part version before other files --- src/Interpreters/MergeTreeTransaction.cpp | 55 ++++++++++++--- .../TransactionVersionMetadata.cpp | 25 +------ src/Interpreters/TransactionVersionMetadata.h | 2 +- src/Interpreters/TransactionsInfoLog.cpp | 22 ++++++ src/Interpreters/TransactionsInfoLog.h | 4 ++ src/Storages/MergeTree/DataPartsExchange.cpp | 5 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 37 +++++++--- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/MergeFromLogEntryTask.cpp | 3 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 3 +- src/Storages/MergeTree/MergeTask.cpp | 2 + src/Storages/MergeTree/MergeTask.h | 4 ++ src/Storages/MergeTree/MergeTreeData.cpp | 67 +++++++++++++------ src/Storages/MergeTree/MergeTreeData.h | 3 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++ .../MergeTree/MergeTreeDataMergerMutator.h | 2 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 6 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 3 +- .../MergeTree/MergedBlockOutputStream.cpp | 12 ++++ .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergeTree/MutateFromLogEntryTask.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 12 +++- src/Storages/MergeTree/MutateTask.h | 1 + src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 +-- .../01172_transaction_counters.reference | 1 - 28 files changed, 212 insertions(+), 87 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 430749dedb9..b76d1618fc5 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -33,7 +34,10 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - new_part->version.setCreationTID(tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); + /// Now we know actual part name and can write it to system log table. + tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); + + new_part->assertHasVersionMetadata(txn); if (txn) txn->addNewPart(storage, new_part); } @@ -52,7 +56,9 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; TransactionInfoContext context{storage->getStorageID(), new_part->name}; - new_part->version.setCreationTID(tid, context); + tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, context); + new_part->assertHasVersionMetadata(txn); + if (txn) txn->addNewPart(storage, new_part); @@ -94,12 +100,14 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) { + storages.insert(table); mutations.emplace_back(table, mutation_id); } bool MergeTreeTransaction::isReadOnly() const { - return creating_parts.empty() && removing_parts.empty(); + assert((creating_parts.empty() && removing_parts.empty() && mutations.empty()) == storages.empty()); + return storages.empty(); } void MergeTreeTransaction::beforeCommit() @@ -169,20 +177,45 @@ void MergeTreeTransaction::onException() String MergeTreeTransaction::dumpDescription() const { - String res = "\ncreating parts:\n"; - for (const auto & part : creating_parts) + String res = fmt::format("{} state: {}, snapshot: {}", tid, getState(), snapshot); + + if (isReadOnly()) { - res += part->name; - res += "\n"; + res += ", readonly"; + return res; } - res += "removing parts:\n"; + res += fmt::format(", affects {} tables:", storages.size()); + + using ChangesInTable = std::tuple; + std::unordered_map 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) { - res += part->name; - res += fmt::format(" (created by {}, {})\n", part->version.getCreationTID(), part->version.creation_csn); + 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); - assert(!part->version.removal_csn); + } + + 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; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 15f96141fd4..5f0d7e08d02 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -20,26 +20,6 @@ namespace ErrorCodes extern const int CANNOT_PARSE_TEXT; } -static void tryWriteEventToSystemLog(Poco::Logger * log, - TransactionsInfoLogElement::Type type, const TransactionID & tid, - const TransactionInfoContext & context) -try -{ - auto system_log = Context::getGlobalContextInstance()->getTransactionsInfoLog(); - if (!system_log) - return; - - TransactionsInfoLogElement elem; - elem.type = type; - elem.tid = tid; - elem.fillCommonFields(&context); - system_log->add(elem); -} -catch (...) -{ - tryLogCurrentException(log); -} - VersionMetadata::VersionMetadata() { /// It would be better to make it static, but static loggers do not work for some reason (initialization order?) @@ -143,12 +123,13 @@ bool VersionMetadata::isMaxTIDLocked() const return removal_tid_lock.load() != 0; } -void VersionMetadata::setCreationTID(const TransactionID & tid, const TransactionInfoContext & context) +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; - tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, context); + if (context) + tryWriteEventToSystemLog(log, TransactionsInfoLogElement::ADD_PART, tid, *context); } bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index de29d9580c2..97ba754c298 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -43,7 +43,7 @@ struct VersionMetadata bool isMaxTIDLocked() const; /// It can be called only from MergeTreeTransaction or on server startup - void setCreationTID(const TransactionID & tid, const TransactionInfoContext & context); + void setCreationTID(const TransactionID & tid, TransactionInfoContext * context); /// Checks if it's safe to remove outdated version of an object bool canBeRemoved(); diff --git a/src/Interpreters/TransactionsInfoLog.cpp b/src/Interpreters/TransactionsInfoLog.cpp index c126e38b5b3..0498ee00e9e 100644 --- a/src/Interpreters/TransactionsInfoLog.cpp +++ b/src/Interpreters/TransactionsInfoLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -86,4 +87,25 @@ void TransactionsInfoLogElement::appendToBlock(MutableColumns & columns) const 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); +} + } diff --git a/src/Interpreters/TransactionsInfoLog.h b/src/Interpreters/TransactionsInfoLog.h index 8b48b54aa88..f595413a729 100644 --- a/src/Interpreters/TransactionsInfoLog.h +++ b/src/Interpreters/TransactionsInfoLog.h @@ -51,4 +51,8 @@ class TransactionsInfoLog : public SystemLog using SystemLog::SystemLog; }; + +void tryWriteEventToSystemLog(Poco::Logger * log, TransactionsInfoLogElement::Type type, + const TransactionID & tid, const TransactionInfoContext & context); + } diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index d2dcf8c1abd..7f8e0ebc4b9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -592,7 +592,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( metadata_snapshot->projections.get(projection_name).metadata, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {})); + CompressionCodecFactory::instance().get("NONE", {}), + nullptr); part_out.write(block); part_out.finalizePart(new_projection_part, false); @@ -616,7 +617,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( MergedBlockOutputStream part_out( new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {})); + CompressionCodecFactory::instance().get("NONE", {}), nullptr); part_out.write(block); part_out.finalizePart(new_data_part, false); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 9da34506817..c1e4eab22b0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -1103,13 +1104,31 @@ void IMergeTreeDataPart::loadColumns(bool require) setSerializationInfos(infos); } +void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const +{ + assert(storage.supportsTransactions()); + 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() : ""); + + assert(!txn || volume->getDisk()->exists(fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME)); +} void IMergeTreeDataPart::storeVersionMetadata() const { + assert(storage.supportsTransactions()); assert(!version.creation_tid.isEmpty()); if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) return; + LOG_TEST(storage.log, "Writing version for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid); + + 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(); @@ -1121,6 +1140,9 @@ void IMergeTreeDataPart::storeVersionMetadata() const auto out = disk->writeFile(tmp_version_file_name, 256, WriteMode::Rewrite); version.write(*out); out->finalize(); + /// TODO Small enough appends to file are usually atomic, + /// maybe we should append new metadata instead of rewriting file to reduce number of fsyncs. + /// (especially, it will allow to remove fsync after writing CSN after commit). out->sync(); } @@ -1160,21 +1182,18 @@ try /// 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 - /// We must remove part in this case, but we cannot distinguish it from the first case. - /// TODO Write something to some checksummed file if part was created with transaction, - /// so part will be ether broken or known to be created by transaction. + /// 3. Version metadata were written to *.tmp file, but hard restart happened before fsync. /// 4. Fsyncs in storeVersionMetadata() work incorrectly. - TransactionInfoContext txn_context{storage.getStorageID(), name}; - if (!disk->exists(tmp_version_file_name)) { - /// Case 1 (or 3). + /// 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. - version.setCreationTID(Tx::PrehistoricTID, txn_context); + /// 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; } @@ -1182,7 +1201,7 @@ try /// 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, txn_context); + version.setCreationTID(Tx::DummyTID, nullptr); version.creation_csn = Tx::RolledBackCSN; remove_tmp_file(); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 768c5cda973..3b48d5fe994 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -423,6 +423,7 @@ public: /// Required for distinguish different copies of the same part on remote FS. String getUniqueId() const; + void assertHasVersionMetadata(MergeTreeTransaction * txn) const; void storeVersionMetadata() const; void loadVersionMetadata() const; diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 6731c6a354a..fa0d50d6b4c 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -207,7 +207,8 @@ std::pair MergeFromLogEntryT reserved_space, entry.deduplicate, entry.deduplicate_by_columns, - storage.merging_params); + storage.merging_params, + nullptr); /// Adjust priority diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 702f5b318e6..df94bdf2620 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -109,7 +109,8 @@ void MergePlainMergeTreeTask::prepare() merge_mutate_entry->tagger->reserved_space, deduplicate, deduplicate_by_columns, - storage.merging_params); + storage.merging_params, + txn); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5d5035d62ca..61f646f0adb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -262,6 +262,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); @@ -599,6 +600,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c projection_merging_params, global_ctx->new_data_part.get(), ".proj", + nullptr, global_ctx->data, global_ctx->mutator, global_ctx->merges_blocker, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index aa64c4c2265..a0cce5976f2 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -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(); @@ -164,6 +166,8 @@ private: std::promise promise{}; IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; + + MergeTreeTransactionPtr txn; }; using GlobalRuntimeContextPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 327adcee33b..92a3b2f6975 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1310,7 +1310,18 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) 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.isMaxTIDLocked()) + { + /// 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); }; @@ -1405,40 +1416,45 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// 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); - (*prev_jt)->assertState({DataPartState::Active}); - - while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Active) + while (curr_it != data_parts_by_state_and_info.end() && (*curr_it)->getState() == DataPartState::Active) { + (*curr_it)->assertState({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; } } } @@ -5411,7 +5427,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; @@ -5429,6 +5446,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()); @@ -5455,6 +5473,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); + if (supportsTransactions()) + { + /// 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; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 274b24c734c..c92c8a1c8ff 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -728,7 +728,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 getMutationsStatus() const = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index defb69da8dc..023926f8621 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -422,6 +422,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) { @@ -438,6 +439,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( merging_params, parent_part, suffix, + txn, &data, this, &merges_blocker, @@ -452,6 +454,7 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart( MergeListEntry * merge_entry, time_t time_of_mutation, ContextPtr context, + const MergeTreeTransactionPtr & txn, ReservationSharedPtr space_reservation, TableLockHolder & holder) { @@ -464,6 +467,7 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart( context, space_reservation, holder, + txn, data, *this, merges_blocker diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index c64d05edfd4..98ff5b0cf06 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -113,6 +113,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 = ""); @@ -124,6 +125,7 @@ public: MergeListEntry * merge_entry, time_t time_of_mutation, ContextPtr context, + const MergeTreeTransactionPtr & txn, ReservationSharedPtr space_reservation, TableLockHolder & table_lock_holder); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index e4a174a7d29..257e99cab17 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -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, nullptr); 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, nullptr); projection_out.write(projection_part->block); projection_out.finalizePart(projection_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d16b5274a45..2f29ff39ef4 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -427,7 +428,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( const auto & index_factory = MergeTreeIndexFactory::instance(); auto out = std::make_unique(new_data_part, metadata_snapshot,columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, context->getCurrentTransaction()); out->writeWithPermutation(block, perm_ptr); @@ -545,7 +546,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( metadata_snapshot, columns, MergeTreeIndices{}, - compression_codec); + compression_codec, + nullptr); out->writeWithPermutation(block, perm_ptr); auto finalizer = out->finalizePartAsync(new_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 9a41204c7e5..99742a923bd 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -195,7 +195,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {})); + CompressionCodecFactory::instance().get("NONE", {}), + nullptr); part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e262217d17e..abb3c3be842 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -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) : IMergedBlockOutputStream(data_part, metadata_snapshot_, columns_list_, reset_columns_) @@ -34,6 +36,16 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (!part_path.empty()) volume->getDisk()->createDirectories(part_path); + if (storage.supportsTransactions()) + { + /// 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); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index b38395d56c2..0a4f9d23927 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -19,6 +19,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); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 405c58f9cd1..d805ea64be5 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -89,7 +89,7 @@ std::pair MutateFromLogEntry mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, merge_mutate_entry.get(), - entry.create_time, fake_query_context, reserved_space, table_lock_holder); + entry.create_time, fake_query_context, nullptr, reserved_space, table_lock_holder); /// Adjust priority for (auto & item : future_mutated_part->parts) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 670a9850956..858e26874eb 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -57,7 +57,7 @@ void MutatePlainMergeTreeTask::prepare() mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_part, metadata_snapshot, merge_mutate_entry->commands, merge_list_entry.get(), - time(nullptr), fake_query_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); + time(nullptr), fake_query_context, merge_mutate_entry->txn, merge_mutate_entry->tagger->reserved_space, table_lock_holder); } bool MutatePlainMergeTreeTask::executeStep() diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index cc690287ef6..b4c5a705157 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -536,6 +536,8 @@ struct MutationContext bool need_sync; ExecuteTTLType execute_ttl_type{ExecuteTTLType::NONE}; + + MergeTreeTransactionPtr txn; }; using MutationContextPtr = std::shared_ptr; @@ -646,6 +648,7 @@ public: false, // TODO Do we need deduplicate for projections {}, projection_merging_params, + nullptr, ctx->new_data_part.get(), ".tmp_proj"); @@ -967,7 +970,8 @@ private: ctx->metadata_snapshot, ctx->new_data_part->getColumns(), skip_part_indices, - ctx->compression_codec); + ctx->compression_codec, + ctx->txn); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); ctx->mutating_executor = std::make_unique(ctx->mutating_pipeline); @@ -1188,6 +1192,7 @@ MutateTask::MutateTask( ContextPtr context_, ReservationSharedPtr space_reservation_, TableLockHolder & table_lock_holder_, + const MergeTreeTransactionPtr & txn, MergeTreeData & data_, MergeTreeDataMergerMutator & mutator_, ActionBlocker & merges_blocker_) @@ -1205,6 +1210,7 @@ MutateTask::MutateTask( ctx->metadata_snapshot = metadata_snapshot_; ctx->space_reservation = space_reservation_; ctx->storage_columns = metadata_snapshot_->getColumns().getAllPhysical(); + ctx->txn = txn; } @@ -1264,7 +1270,7 @@ bool MutateTask::prepare() storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); - promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot)); + promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn)); return false; } else @@ -1358,7 +1364,7 @@ bool MutateTask::prepare() && ctx->files_to_rename.empty()) { LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {} (optimized)", ctx->source_part->name, ctx->future_part->part_info.mutation); - promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot)); + promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn)); return false; } diff --git a/src/Storages/MergeTree/MutateTask.h b/src/Storages/MergeTree/MutateTask.h index 79c3bff213a..aa38ee34b4a 100644 --- a/src/Storages/MergeTree/MutateTask.h +++ b/src/Storages/MergeTree/MutateTask.h @@ -32,6 +32,7 @@ public: ContextPtr context_, ReservationSharedPtr space_reservation_, TableLockHolder & table_lock_holder_, + const MergeTreeTransactionPtr & txn, MergeTreeData & data_, MergeTreeDataMergerMutator & mutator_, ActionBlocker & merges_blocker_); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cfdc0496f23..87ef26a8423 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1550,7 +1550,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - dst_parts.emplace_back(cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot)); + auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, local_context->getCurrentTransaction()); + dst_parts.emplace_back(std::move(dst_part)); } /// ATTACH empty part set @@ -1635,7 +1636,8 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot)); + auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, local_context->getCurrentTransaction()); + dst_parts.emplace_back(std::move(dst_part)); } /// empty part set diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fa167ed1d27..14b5798e829 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2107,7 +2107,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) throw Exception("Checksums of " + part_desc->src_table_part->name + " is suddenly changed", ErrorCodes::UNFINISHED); part_desc->res_part = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot); + part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, nullptr); } else if (!part_desc->replica.empty()) { @@ -3821,7 +3821,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora { get_part = [&, part_to_clone]() { - return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot); + return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, nullptr); }; } else @@ -6272,7 +6272,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot); + auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, nullptr); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -6483,7 +6483,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot); + auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, nullptr); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -7544,7 +7544,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, nullptr); bool sync_on_insert = settings->fsync_after_insert; diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 0ab65e5dc9a..3a167e76817 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -23,7 +23,6 @@ 3 1 Begin 1 1 1 1 3 1 AddPart 1 1 1 1 all_3_3_0 3 1 Commit 1 1 1 0 -1 1 AddPart 1 1 1 1 all_1_1_0 4 1 Begin 1 1 1 1 4 1 AddPart 1 1 1 1 all_4_4_0 4 1 Commit 1 1 1 0 From e37ef4560cd47c239eecad3342ff39ff32dc7307 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Feb 2022 18:00:45 +0300 Subject: [PATCH 048/372] fix --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 ++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 14 +++++--------- .../MergeTree/MergedBlockOutputStream.cpp | 15 ++++++--------- src/Storages/MergeTree/MutateTask.cpp | 8 ++++++++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 1 + src/Storages/StorageMergeTree.cpp | 6 ++++++ src/Storages/StorageReplicatedMergeTree.cpp | 1 + 8 files changed, 31 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7f8e0ebc4b9..3b7c373930a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -564,6 +564,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( auto volume = std::make_shared("volume_" + part_name, disk, 0); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared(data, part_name, volume); + new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); for (auto i = 0ul; i < projections; ++i) { @@ -745,6 +746,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( assertEOF(in); auto volume = std::make_shared("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); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c1e4eab22b0..743557bfada 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1106,24 +1106,24 @@ void IMergeTreeDataPart::loadColumns(bool require) void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const { - assert(storage.supportsTransactions()); 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() : ""); + assert(!txn || storage.supportsTransactions()); assert(!txn || volume->getDisk()->exists(fs::path(getFullRelativePath()) / TXN_VERSION_METADATA_FILE_NAME)); } void IMergeTreeDataPart::storeVersionMetadata() const { - assert(storage.supportsTransactions()); assert(!version.creation_tid.isEmpty()); if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) 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: {})", diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a4e4d46bab0..5dee7977a65 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -47,8 +47,6 @@ #include #include #include -#include -#include #include #include #include @@ -5472,13 +5470,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); - if (supportsTransactions()) - { - /// 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(); - } + + /// 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; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index abb3c3be842..8f303f37947 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -36,15 +36,12 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (!part_path.empty()) volume->getDisk()->createDirectories(part_path); - if (storage.supportsTransactions()) - { - /// 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(); - } + /// 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); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b4c5a705157..542ae8e0fbf 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -1058,6 +1059,13 @@ private: ctx->disk->createDirectories(ctx->new_part_tmp_path); + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = ctx->txn ? ctx->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. + ctx->new_data_part->version.setCreationTID(tid, nullptr); + ctx->new_data_part->storeVersionMetadata(); + /// Create hardlinks for unchanged files for (auto it = ctx->disk->iterateDirectory(ctx->source_part->getFullRelativePath()); it->isValid(); it->next()) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f5930bca08e..e1641894392 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -259,6 +259,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { + part->version.setCreationTID(Tx::PrehistoricTID, nullptr); commitPart(zookeeper, part, ""); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 87ef26a8423..20d78827bac 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1504,6 +1504,12 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( for (size_t i = 0; i < loaded_parts.size(); ++i) { LOG_INFO(log, "Attaching part {} from {}", loaded_parts[i]->name, renamed_parts.old_and_new_names[i].new_name); + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + auto txn = local_context->getCurrentTransaction(); + TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + loaded_parts[i]->version.setCreationTID(tid, nullptr); + loaded_parts[i]->storeVersionMetadata(); + String old_name = renamed_parts.old_and_new_names[i].old_name; renameTempPartAndAdd(loaded_parts[i], local_context->getCurrentTransaction().get(), &increment); renamed_parts.old_and_new_names[i].old_name.clear(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 14b5798e829..fbc3176a9d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1504,6 +1504,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) Transaction transaction(*this, nullptr); + part->version.setCreationTID(Tx::PrehistoricTID, nullptr); renameTempPartAndReplace(part, nullptr, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); From dec083ab4406f44bc5c189f74297052116049b4b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 18 Jan 2022 18:47:25 +0800 Subject: [PATCH 049/372] add minmax index for hivengine; remove libhdfspp dependency for hive engine --- src/Storages/Hive/HiveFile.cpp | 203 ++++++++++++++++++++++++++---- src/Storages/Hive/HiveFile.h | 28 ++--- src/Storages/Hive/StorageHive.cpp | 6 +- 3 files changed, 194 insertions(+), 43 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index b0cfa9809e1..a74a5e36575 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -1,22 +1,37 @@ +<<<<<<< HEAD #include +======= + +#include +>>>>>>> d9558cbca4... add minmax index for hivengine; remove libhdfspp dependency for hive engine #if USE_HIVE #include +<<<<<<< HEAD #include #include +======= +#include +#include +#include +#include +>>>>>>> d9558cbca4... add minmax index for hivengine; remove libhdfspp dependency for hive engine #include +#include #include -#include -#include #include #include #include +#include #include #include #include +#include +#include +#include #include #include @@ -28,10 +43,22 @@ namespace ErrorCodes namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +#define THROW_ARROW_NOT_OK(status) \ + do \ + { \ + if (::arrow::Status _s = (status); !_s.ok()) \ + throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ + } while (false) + + template Range createRangeFromOrcStatistics(const StatisticsType * stats) { - /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (stats->hasMinimum() && stats->hasMaximum()) { @@ -117,65 +144,193 @@ Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) void HiveOrcFile::prepareReader() { - // TODO To be implemented - throw Exception("Unimplemented HiveOrcFile::prepareReader", ErrorCodes::NOT_IMPLEMENTED); + in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); + auto format_settings = getFormatSettings(getContext()); + THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &reader)); } void HiveOrcFile::prepareColumnMapping() { - // TODO To be implemented - throw Exception("Unimplemented HiveOrcFile::prepareColumnMapping", ErrorCodes::NOT_IMPLEMENTED); + const orc::Type & type = reader->GetRawORCReader()->getType(); + size_t size = type.getSubtypeCount(); + for (size_t pos = 0; pos < size; pos++) + { + // hive中字符串不区分大小写。所以这里统一改成小写,方便匹配 + String column{type.getFieldName(pos)}; + boost::to_lower(column); + orc_column_positions[column] = pos; + } } bool HiveOrcFile::hasMinMaxIndex() const { - return false; + return !storage_settings->disable_orc_file_minmax_index; } -std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * /*statistics*/) +std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * statistics) { - // TODO To be implemented - throw Exception("Unimplemented HiveOrcFile::buildMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); + if (!statistics) + return nullptr; + + size_t size = index_names_and_types.size(); + auto idx = std::make_unique(); + idx->hyperrectangle.resize(size); + + size_t i = 0; + for (const auto & name_type : index_names_and_types) + { + String column{name_type.name}; + boost::to_lower(column); + auto it = orc_column_positions.find(column); + if (it == orc_column_positions.end()) + { + idx->hyperrectangle[i] = buildRange(nullptr); + // std::cerr << "statistics:nullptr" << std::endl; + } + else + { + size_t pos = it->second; + // 注意:column statistics从1开始. 0有特殊用途 + const orc::ColumnStatistics * col_stats = statistics->getColumnStatistics(pos + 1); + idx->hyperrectangle[i] = buildRange(col_stats); + // std::cerr << "statistics:" << col_stats->toString(); + // std::cerr << "name:" << column << ", pos" << pos << ", range:" << idx->hyperrectangle[i].toString() << std::endl; + } + ++i; + } + idx->initialized = true; + return idx; } void HiveOrcFile::loadMinMaxIndex() { - // TODO To be implemented - throw Exception("Unimplemented HiveOrcFile::loadMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); + if (!reader) + { + prepareReader(); + prepareColumnMapping(); + } + + auto statistics = reader->GetRawORCReader()->getStatistics(); + minmax_idx = buildMinMaxIndex(statistics.get()); } bool HiveOrcFile::hasSubMinMaxIndex() const { - // TODO To be implemented - return false; + return !storage_settings->disable_orc_stripe_minmax_index; } void HiveOrcFile::loadSubMinMaxIndex() { - // TODO To be implemented - throw Exception("Unimplemented HiveOrcFile::loadSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); + if (!reader) + { + prepareReader(); + prepareColumnMapping(); + } + + auto * raw_reader = reader->GetRawORCReader(); + auto stripe_num = raw_reader->getNumberOfStripes(); + auto stripe_stats_num = raw_reader->getNumberOfStripeStatistics(); + if (stripe_num != stripe_stats_num) + throw Exception( + fmt::format("orc file:{} has different strip num {} and strip statistics num {}", path, stripe_num, stripe_stats_num), + ErrorCodes::BAD_ARGUMENTS); + + sub_minmax_idxes.resize(stripe_num); + for (size_t i = 0; i < stripe_num; ++i) + { + auto stripe_stats = raw_reader->getStripeStatistics(i); + sub_minmax_idxes[i] = buildMinMaxIndex(stripe_stats.get()); + } } bool HiveParquetFile::hasSubMinMaxIndex() const { - // TODO To be implemented - return false; + return !storage_settings->disable_parquet_rowgroup_minmax_index; } void HiveParquetFile::prepareReader() { - // TODO To be implemented - throw Exception("Unimplemented HiveParquetFile::prepareReader", ErrorCodes::NOT_IMPLEMENTED); + in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); + auto format_settings = getFormatSettings(getContext()); + THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &reader)); } - void HiveParquetFile::loadSubMinMaxIndex() { - // TODO To be implemented - throw Exception("Unimplemented HiveParquetFile::loadSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); + if (!reader) + prepareReader(); + + auto meta = reader->parquet_reader()->metadata(); + size_t num_cols = meta->num_columns(); + size_t num_row_groups = meta->num_row_groups(); + const auto * schema = meta->schema(); + for (size_t pos = 0; pos < num_cols; ++pos) + { + String column{schema->Column(pos)->name()}; + boost::to_lower(column); + parquet_column_positions[column] = pos; + } + + + sub_minmax_idxes.resize(num_row_groups); + for (size_t i = 0; i < num_row_groups; ++i) + { + auto row_group_meta = meta->RowGroup(i); + sub_minmax_idxes[i] = std::make_shared(); + sub_minmax_idxes[i]->hyperrectangle.resize(num_cols); + + size_t j = 0; + auto it = index_names_and_types.begin(); + for (; it != index_names_and_types.end(); ++j, ++it) + { + // 如果parquet file中不存在该字段,使用空Range + String name{it->name}; + boost::to_lower(name); + auto mit = parquet_column_positions.find(name); + if (mit == parquet_column_positions.end()) + continue; + + size_t pos = mit->second; + auto col_chunk = row_group_meta->ColumnChunk(pos); + if (!col_chunk->is_stats_set()) + continue; + + auto stats = col_chunk->statistics(); + if (stats->HasNullCount() && stats->null_count() > 0) + continue; + + if (auto bool_stats = std::dynamic_pointer_cast(stats)) + { + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(bool_stats); + } + else if (auto int32_stats = std::dynamic_pointer_cast(stats)) + { + // Hive中没有unsigned interger, 这里不用考虑相关case + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int32_stats); + } + else if (auto int64_stats = std::dynamic_pointer_cast(stats)) + { + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int64_stats); + } + else if (auto float_stats = std::dynamic_pointer_cast(stats)) + { + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(float_stats); + } + else if (auto double_stats = std::dynamic_pointer_cast(stats)) + { + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(double_stats); + } + else if (auto string_stats = std::dynamic_pointer_cast(stats)) + { + sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(string_stats); + } + // 其他类型无法使用minmax index, 跳过 + } + sub_minmax_idxes[i]->initialized = true; + } } } diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 63cca2562eb..7a3fcac312f 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -17,26 +17,18 @@ namespace orc { -class Reader; +class Statistics; +class ColumnStatistics; } -namespace parquet +namespace parquet::arrow { -class ParquetFileReader; -namespace arrow -{ - class FileReader; -} +class FileReader; } -namespace arrow +namespace arrow::adapters::orc { -namespace io -{ - class RandomAccessFile; -} - -class Buffer; +class ORCFileReader; } namespace DB @@ -46,6 +38,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +class ReadBufferFromHDFS; class IHiveFile : public WithContext { public: @@ -230,7 +223,8 @@ protected: virtual void prepareReader(); virtual void prepareColumnMapping(); - std::shared_ptr reader; + std::unique_ptr in; + std::unique_ptr reader; std::map orc_column_positions; }; @@ -259,8 +253,8 @@ public: protected: virtual void prepareReader(); - std::shared_ptr fs; - std::shared_ptr reader; + std::unique_ptr in; + std::unique_ptr reader; std::map parquet_column_positions; }; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3040ad23283..ed9da822fb0 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -2,12 +2,14 @@ #if USE_HIVE -#include #include #include +#include #include - #include +#include +#include + #include #include #include From f19f0d847f1752ed1263253e898c04a51f17966e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Feb 2022 12:23:06 +0800 Subject: [PATCH 050/372] fix code style --- src/Storages/Hive/HiveFile.cpp | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index a74a5e36575..85f0a5dfee0 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -1,23 +1,12 @@ -<<<<<<< HEAD #include -======= - -#include ->>>>>>> d9558cbca4... add minmax index for hivengine; remove libhdfspp dependency for hive engine #if USE_HIVE #include -<<<<<<< HEAD -#include - -#include -======= #include #include #include #include ->>>>>>> d9558cbca4... add minmax index for hivengine; remove libhdfspp dependency for hive engine #include #include #include @@ -35,11 +24,6 @@ #include #include -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - namespace DB { From afcb295273c14708cf12d6f0102f56e7bced0278 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Feb 2022 14:51:56 +0800 Subject: [PATCH 051/372] fix compile error --- src/Storages/Hive/HiveFile.cpp | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 85f0a5dfee0..bbcb1d3e82e 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -43,7 +43,7 @@ namespace ErrorCodes template Range createRangeFromOrcStatistics(const StatisticsType * stats) { - /// null values or NaN/Inf values of double type. + /// Null values or NaN/Inf values of double type. if (stats->hasMinimum() && stats->hasMaximum()) { return Range(FieldType(stats->getMinimum()), true, FieldType(stats->getMaximum()), true); @@ -130,7 +130,8 @@ void HiveOrcFile::prepareReader() { in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); - THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &reader)); + std::atomic is_stopped{0}; + THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); } void HiveOrcFile::prepareColumnMapping() @@ -139,7 +140,7 @@ void HiveOrcFile::prepareColumnMapping() size_t size = type.getSubtypeCount(); for (size_t pos = 0; pos < size; pos++) { - // hive中字符串不区分大小写。所以这里统一改成小写,方便匹配 + /// Column names in hive is case-insensitive. String column{type.getFieldName(pos)}; boost::to_lower(column); orc_column_positions[column] = pos; @@ -148,7 +149,7 @@ void HiveOrcFile::prepareColumnMapping() bool HiveOrcFile::hasMinMaxIndex() const { - return !storage_settings->disable_orc_file_minmax_index; + return !storage_settings->enable_orc_file_minmax_index; } @@ -170,16 +171,13 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c if (it == orc_column_positions.end()) { idx->hyperrectangle[i] = buildRange(nullptr); - // std::cerr << "statistics:nullptr" << std::endl; } else { size_t pos = it->second; - // 注意:column statistics从1开始. 0有特殊用途 + /// Attention: column statistics start from 1. 0 has special purpose. const orc::ColumnStatistics * col_stats = statistics->getColumnStatistics(pos + 1); idx->hyperrectangle[i] = buildRange(col_stats); - // std::cerr << "statistics:" << col_stats->toString(); - // std::cerr << "name:" << column << ", pos" << pos << ", range:" << idx->hyperrectangle[i].toString() << std::endl; } ++i; } @@ -202,7 +200,7 @@ void HiveOrcFile::loadMinMaxIndex() bool HiveOrcFile::hasSubMinMaxIndex() const { - return !storage_settings->disable_orc_stripe_minmax_index; + return !storage_settings->enable_orc_stripe_minmax_index; } @@ -232,14 +230,15 @@ void HiveOrcFile::loadSubMinMaxIndex() bool HiveParquetFile::hasSubMinMaxIndex() const { - return !storage_settings->disable_parquet_rowgroup_minmax_index; + return !storage_settings->enable_parquet_rowgroup_minmax_index; } void HiveParquetFile::prepareReader() { in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); - THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &reader)); + std::atomic is_stopped{0}; + THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); } void HiveParquetFile::loadSubMinMaxIndex() @@ -270,7 +269,6 @@ void HiveParquetFile::loadSubMinMaxIndex() auto it = index_names_and_types.begin(); for (; it != index_names_and_types.end(); ++j, ++it) { - // 如果parquet file中不存在该字段,使用空Range String name{it->name}; boost::to_lower(name); auto mit = parquet_column_positions.find(name); @@ -292,7 +290,6 @@ void HiveParquetFile::loadSubMinMaxIndex() } else if (auto int32_stats = std::dynamic_pointer_cast(stats)) { - // Hive中没有unsigned interger, 这里不用考虑相关case sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int32_stats); } else if (auto int64_stats = std::dynamic_pointer_cast(stats)) @@ -311,7 +308,7 @@ void HiveParquetFile::loadSubMinMaxIndex() { sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(string_stats); } - // 其他类型无法使用minmax index, 跳过 + /// Other types are not supported for minmax index, skip } sub_minmax_idxes[i]->initialized = true; } From a4baec6d267f3aa5283f26d4034914cdfdfc2b3b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Feb 2022 15:12:43 +0800 Subject: [PATCH 052/372] fix building --- src/Storages/Hive/HiveFile.cpp | 7 ++++--- src/Storages/Hive/HiveFile.h | 2 -- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index bbcb1d3e82e..896c464c80f 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -3,8 +3,6 @@ #if USE_HIVE #include -#include -#include #include #include #include @@ -13,6 +11,7 @@ #include #include #include +#include #include #include @@ -131,7 +130,9 @@ void HiveOrcFile::prepareReader() in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; - THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); + auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool()); + THROW_ARROW_NOT_OK(result.status()); + reader = std::move(result).ValueOrDie(); } void HiveOrcFile::prepareColumnMapping() diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 7a3fcac312f..dfecd79f932 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -7,8 +7,6 @@ #include #include -#include -#include #include #include From f4a46a13fbeb6398a683ee88ec35d5cee2191c19 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Feb 2022 00:26:37 +0300 Subject: [PATCH 053/372] fixes --- src/Interpreters/MergeTreeTransaction.cpp | 6 +- .../TransactionVersionMetadata.cpp | 99 +++++++++++-------- src/Interpreters/TransactionVersionMetadata.h | 5 +- src/Storages/MergeTree/DataPartsExchange.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 81 +++++++++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 14 +++ src/Storages/MergeTree/MergeTreeData.cpp | 17 ++-- src/Storages/MergeTree/localBackup.cpp | 11 ++- src/Storages/MergeTree/localBackup.h | 2 +- 9 files changed, 172 insertions(+), 64 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index b76d1618fc5..f63f4535b44 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -82,7 +82,7 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart storages.insert(storage); creating_parts.push_back(new_part); - new_part->storeVersionMetadata(); + //new_part->storeVersionMetadata(); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) @@ -132,13 +132,13 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept for (const auto & part : creating_parts) { part->version.creation_csn.store(csn); - part->storeVersionMetadata(); + part->appendCSNToVersionMetadata(VersionMetadata::WhichCSN::CREATION); } for (const auto & part : removing_parts) { part->version.removal_csn.store(csn); - part->storeVersionMetadata(); + part->appendCSNToVersionMetadata(VersionMetadata::WhichCSN::REMOVAL); } } diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 5f0d7e08d02..fbccbc6ac32 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -118,7 +118,7 @@ void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionI tryWriteEventToSystemLog(log, TransactionsInfoLogElement::UNLOCK_PART, tid, context); } -bool VersionMetadata::isMaxTIDLocked() const +bool VersionMetadata::isRemovalTIDLocked() const { return removal_tid_lock.load() != 0; } @@ -281,16 +281,37 @@ bool VersionMetadata::canBeRemovedImpl(Snapshot oldest_snapshot_version) #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 min = creation_csn.load()) + { + writeCString("\n" CREATION_CSN_STR, buf); + writeText(min, 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 max = removal_csn.load()) + { + writeCString("\n" REMOVAL_CSN_STR, buf); + writeText(max, buf); + } + else if (!internal) + throw Exception(ErrorCodes::LOGICAL_ERROR, "writeCSN called for removal_csn = 0, it's a bug"); + } +} + void VersionMetadata::write(WriteBuffer & buf) const { writeCString("version: 1", buf); writeCString("\n" CREATION_TID_STR, buf); TransactionID::write(creation_tid, buf); - if (CSN min = creation_csn.load()) - { - writeCString("\n" CREATION_CSN_STR, buf); - writeText(min, buf); - } + writeCSN(buf, CREATION, /* internal */ true); if (removal_tid_lock) { @@ -298,11 +319,7 @@ void VersionMetadata::write(WriteBuffer & buf) const assert(removal_tid.getHash() == removal_tid_lock); writeCString("\n" REMOVAL_TID_STR, buf); TransactionID::write(removal_tid, buf); - if (CSN max = removal_csn.load()) - { - writeCString("\n" REMOVAL_CSN_STR, buf); - writeText(max, buf); - } + writeCSN(buf, REMOVAL, /* internal */ true); } } @@ -322,41 +339,41 @@ void VersionMetadata::read(ReadBuffer & buf) String name; name.resize(size); - assertChar('\n', buf); - buf.readStrict(name.data(), size); - if (name == CREATION_CSN_STR) + auto read_csn = [&]() { - UInt64 min; - readText(min, buf); - creation_csn = min; - if (buf.eof()) - return; + 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) + { + assert(removal_tid.isEmpty() && removal_tid_lock == 0); + removal_tid = TransactionID::read(buf); + 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); + } } - - if (name == REMOVAL_TID_STR) - { - removal_tid = TransactionID::read(buf); - removal_tid_lock = removal_tid.getHash(); - if (buf.eof()) - return; - - assertChar('\n', buf); - buf.readStrict(name.data(), size); - } - - 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); - UInt64 max; - readText(max, buf); - removal_csn = max; - } - - assertEOF(buf); } String VersionMetadata::toString(bool one_line) const diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 97ba754c298..3958a3f3e86 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -40,7 +40,7 @@ struct VersionMetadata void lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); void unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); - bool isMaxTIDLocked() const; + bool isRemovalTIDLocked() const; /// It can be called only from MergeTreeTransaction or on server startup void setCreationTID(const TransactionID & tid, TransactionInfoContext * context); @@ -52,6 +52,9 @@ struct VersionMetadata void write(WriteBuffer & buf) const; void read(ReadBuffer & buf); + enum WhichCSN { CREATION, REMOVAL }; + void writeCSN(WriteBuffer & buf, WhichCSN which_csn, bool internal = false) const; + String toString(bool one_line = true) const; Poco::Logger * log; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index a07133a68b0..6ff985aac37 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -845,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); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 753ce87a787..64cd4b940a1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -26,6 +25,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -450,6 +450,7 @@ SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & co void IMergeTreeDataPart::removeIfNeeded() { + assert(assertHasValidVersionMetadata()); if (!is_temp && state != State::DeleteOnDestroy) return; @@ -1118,8 +1119,7 @@ void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) co void IMergeTreeDataPart::storeVersionMetadata() const { - assert(!version.creation_tid.isEmpty()); - if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) + if (!wasInvolvedInTransaction()) return; LOG_TEST(storage.log, "Writing version for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid); @@ -1140,9 +1140,6 @@ void IMergeTreeDataPart::storeVersionMetadata() const auto out = disk->writeFile(tmp_version_file_name, 256, WriteMode::Rewrite); version.write(*out); out->finalize(); - /// TODO Small enough appends to file are usually atomic, - /// maybe we should append new metadata instead of rewriting file to reduce number of fsyncs. - /// (especially, it will allow to remove fsync after writing CSN after commit). out->sync(); } @@ -1152,6 +1149,27 @@ void IMergeTreeDataPart::storeVersionMetadata() const disk->moveFile(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::loadVersionMetadata() const try { @@ -1211,6 +1229,56 @@ catch (Exception & e) throw; } +bool IMergeTreeDataPart::wasInvolvedInTransaction() const +{ + assert(!version.creation_tid.isEmpty()); + 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. + + if (!wasInvolvedInTransaction()) + return true; + + if (!isStoredOnDisk()) + return false; + + 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); + //FIXME + bool valid_creation_tid = version.creation_tid == file.creation_tid; + bool valid_removal_tid = version.removal_tid == file.removal_tid || (version.removal_tid.isEmpty() && TransactionLog::getCSN(file.removal_tid) == Tx::UnknownCSN) || 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::RolledBackCSN || 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; + } +} + bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const { @@ -1305,6 +1373,7 @@ std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const void IMergeTreeDataPart::remove() const { + assert(assertHasValidVersionMetadata()); std::optional keep_shared_data = keepSharedDataInDecoupledStorage(); if (!keep_shared_data.has_value()) return; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 6125a76cf2f..1617ded464e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -435,10 +435,24 @@ 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; + + /// 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; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bcd4265bd23..bd95d3c4ed0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1320,7 +1320,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) (*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.isMaxTIDLocked()) + 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), @@ -1347,14 +1347,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) const DataPartPtr & part = *iter; part->loadVersionMetadata(); VersionMetadata & version = part->version; - if (version.creation_tid.isPrehistoric() && (version.removal_tid.isEmpty() || version.removal_tid.isPrehistoric())) + if (part->wasInvolvedInTransaction()) { - ++iter; - continue; + have_parts_with_version_metadata = true; } else { - have_parts_with_version_metadata = true; + ++iter; + continue; } /// Check if CSNs were witten after committing transaction, update and write if needed. @@ -3891,8 +3891,11 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto part = 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); - renameTempPartAndAdd(part, nullptr, increment); //FIXME + renameTempPartAndAdd(part, nullptr, increment); }; restore_tasks.emplace_back(std::move(restore_task)); @@ -5487,7 +5490,7 @@ 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); diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 1a04aa4b678..8bb3e4cf78a 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes } -static void localBackupImpl(const DiskPtr & disk, const String & source_path, const String & destination_path, size_t level, +static void localBackupImpl(const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly, size_t level, std::optional max_level) { if (max_level && level > *max_level) @@ -32,12 +32,13 @@ static void localBackupImpl(const DiskPtr & disk, const String & source_path, co if (!disk->isDirectory(source)) { - disk->setReadOnly(source); + if (make_source_readonly) + disk->setReadOnly(source); disk->createHardLink(source, destination); } else { - localBackupImpl(disk, source, destination, level + 1, max_level); + localBackupImpl(disk, source, destination, make_source_readonly, level + 1, max_level); } } } @@ -80,7 +81,7 @@ private: }; } -void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, std::optional max_level) +void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly, std::optional max_level) { if (disk->exists(destination_path) && !disk->isDirectoryEmpty(destination_path)) { @@ -100,7 +101,7 @@ void localBackup(const DiskPtr & disk, const String & source_path, const String { try { - localBackupImpl(disk, source_path, destination_path, 0, max_level); + localBackupImpl(disk, source_path, destination_path, make_source_readonly, 0, max_level); } catch (const DB::ErrnoException & e) { diff --git a/src/Storages/MergeTree/localBackup.h b/src/Storages/MergeTree/localBackup.h index 066ba8f7489..c6a46620447 100644 --- a/src/Storages/MergeTree/localBackup.h +++ b/src/Storages/MergeTree/localBackup.h @@ -20,6 +20,6 @@ namespace DB * If max_level is specified, than only files which depth relative source_path less or equal max_level will be copied. * So, if max_level=0 than only direct file child are copied. */ -void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, std::optional max_level = {}); +void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly = true, std::optional max_level = {}); } From 711aad695325ede3ba2583845847164c750d2638 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Feb 2022 01:31:21 +0300 Subject: [PATCH 054/372] fix --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 64cd4b940a1..98c67bed867 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1231,7 +1231,7 @@ catch (Exception & e) bool IMergeTreeDataPart::wasInvolvedInTransaction() const { - assert(!version.creation_tid.isEmpty()); + 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; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 49ae08a7e40..b6f8c3e62f7 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1297,6 +1297,8 @@ bool MutateTask::prepare() } ctx->single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); + /// FIXME new_data_part is not used in the case when we clone part with cloneAndLoadDataPartOnSameDisk and return false + /// Is it possible to handle this case earlier? ctx->new_data_part = ctx->data->createPart( ctx->future_part->name, ctx->future_part->type, ctx->future_part->part_info, ctx->single_disk_volume, "tmp_mut_" + ctx->future_part->name); @@ -1363,7 +1365,7 @@ bool MutateTask::prepare() && ctx->files_to_rename.empty()) { LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {} (optimized)", ctx->source_part->name, ctx->future_part->part_info.mutation); - promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn)); + promise.set_value(ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_mut_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn)); return false; } From 11ae0d144bd7dc9c46bf8879002fdafff82e6100 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 25 Feb 2022 00:51:21 +0300 Subject: [PATCH 055/372] fix --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 98c67bed867..b0e3c2291c5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1242,6 +1242,9 @@ 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. + if (isProjectionPart()) + return true; + if (!wasInvolvedInTransaction()) return true; From ea2f65fef6fd1f4ae66e72f5a5100c3b769629a1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Mar 2022 17:35:47 +0100 Subject: [PATCH 056/372] fix tests with DiskS3 --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b0e3c2291c5..7ea135c2905 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1146,7 +1146,7 @@ void IMergeTreeDataPart::storeVersionMetadata() const SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) sync_guard = disk->getDirectorySyncGuard(getFullRelativePath()); - disk->moveFile(tmp_version_file_name, version_file_name); + disk->replaceFile(tmp_version_file_name, version_file_name); } void IMergeTreeDataPart::appendCSNToVersionMetadata(VersionMetadata::WhichCSN which_csn) const From b85ada8ff6c29a9aefcec5c09616b4326995e8a3 Mon Sep 17 00:00:00 2001 From: Meena Renganathan Date: Tue, 8 Mar 2022 09:35:56 -0800 Subject: [PATCH 057/372] Upgrdaed the curl package to fix vulnerabilities reported by WhiteSource --- contrib/curl | 2 +- contrib/curl-cmake/CMakeLists.txt | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/contrib/curl b/contrib/curl index 3b8bbbbd160..801bd5138ce 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit 3b8bbbbd1609c638a3d3d0acb148a33dedb67be3 +Subproject commit 801bd5138ce31aa0d906fa4e2eabfc599d74e793 diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 589f40384e3..b1e1a0ded8a 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -32,7 +32,6 @@ set (SRCS "${LIBRARY_DIR}/lib/transfer.c" "${LIBRARY_DIR}/lib/strcase.c" "${LIBRARY_DIR}/lib/easy.c" - "${LIBRARY_DIR}/lib/security.c" "${LIBRARY_DIR}/lib/curl_fnmatch.c" "${LIBRARY_DIR}/lib/fileinfo.c" "${LIBRARY_DIR}/lib/wildcard.c" @@ -115,6 +114,12 @@ set (SRCS "${LIBRARY_DIR}/lib/curl_get_line.c" "${LIBRARY_DIR}/lib/altsvc.c" "${LIBRARY_DIR}/lib/socketpair.c" + "${LIBRARY_DIR}/lib/bufref.c" + "${LIBRARY_DIR}/lib/dynbuf.c" + "${LIBRARY_DIR}/lib/hsts.c" + "${LIBRARY_DIR}/lib/http_aws_sigv4.c" + "${LIBRARY_DIR}/lib/mqtt.c" + "${LIBRARY_DIR}/lib/rename.c" "${LIBRARY_DIR}/lib/vauth/vauth.c" "${LIBRARY_DIR}/lib/vauth/cleartext.c" "${LIBRARY_DIR}/lib/vauth/cram.c" @@ -131,8 +136,6 @@ set (SRCS "${LIBRARY_DIR}/lib/vtls/gtls.c" "${LIBRARY_DIR}/lib/vtls/vtls.c" "${LIBRARY_DIR}/lib/vtls/nss.c" - "${LIBRARY_DIR}/lib/vtls/polarssl.c" - "${LIBRARY_DIR}/lib/vtls/polarssl_threadlock.c" "${LIBRARY_DIR}/lib/vtls/wolfssl.c" "${LIBRARY_DIR}/lib/vtls/schannel.c" "${LIBRARY_DIR}/lib/vtls/schannel_verify.c" @@ -141,6 +144,7 @@ set (SRCS "${LIBRARY_DIR}/lib/vtls/mbedtls.c" "${LIBRARY_DIR}/lib/vtls/mesalink.c" "${LIBRARY_DIR}/lib/vtls/bearssl.c" + "${LIBRARY_DIR}/lib/vtls/keylog.c" "${LIBRARY_DIR}/lib/vquic/ngtcp2.c" "${LIBRARY_DIR}/lib/vquic/quiche.c" "${LIBRARY_DIR}/lib/vssh/libssh2.c" From d2f838bd91315c9357f6a0e092f179421a5af45f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Mar 2022 20:11:47 +0100 Subject: [PATCH 058/372] fix another race condition --- src/Interpreters/MergeTreeTransaction.cpp | 8 +++-- .../TransactionVersionMetadata.cpp | 14 ++++++-- src/Interpreters/TransactionVersionMetadata.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 35 ++++++++++++++++--- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 ++ .../01169_alter_partition_isolation_stress.sh | 1 + 6 files changed, 52 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index f63f4535b44..7c6f334d33f 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -82,7 +82,6 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart storages.insert(storage); creating_parts.push_back(new_part); - //new_part->storeVersionMetadata(); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) @@ -95,7 +94,7 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP storages.insert(storage); removing_parts.push_back(part_to_remove); - part_to_remove->storeVersionMetadata(); + part_to_remove->appendRemovalTIDToVersionMetadata(); } void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) @@ -156,8 +155,11 @@ bool MergeTreeTransaction::rollback() noexcept for (const auto & part : creating_parts) part->version.creation_csn.store(Tx::RolledBackCSN); - for (const auto & part : removing_parts) /// TODO update metadata file + for (const auto & part : removing_parts) + { + part->appendRemovalTIDToVersionMetadata(/* clear */ true); part->version.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); + } /// FIXME const_cast for (const auto & part : creating_parts) diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index fbccbc6ac32..4f5fb1ed38f 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -306,6 +306,15 @@ void VersionMetadata::writeCSN(WriteBuffer & buf, WhichCSN which_csn, bool inter } } +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); @@ -317,8 +326,7 @@ void VersionMetadata::write(WriteBuffer & buf) const { assert(!removal_tid.isEmpty()); assert(removal_tid.getHash() == removal_tid_lock); - writeCString("\n" REMOVAL_TID_STR, buf); - TransactionID::write(removal_tid, buf); + writeRemovalTID(buf); writeCSN(buf, REMOVAL, /* internal */ true); } } @@ -358,7 +366,7 @@ void VersionMetadata::read(ReadBuffer & buf) } else if (name == REMOVAL_TID_STR) { - assert(removal_tid.isEmpty() && removal_tid_lock == 0); + /// NOTE Metadata file may actually contain multiple creation TIDs, we need the last one. removal_tid = TransactionID::read(buf); removal_tid_lock = removal_tid.getHash(); } diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 3958a3f3e86..e691d45dd1d 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -54,6 +54,7 @@ struct VersionMetadata 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; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7ea135c2905..835371e2a21 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1135,7 +1135,7 @@ void IMergeTreeDataPart::storeVersionMetadata() const { /// 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 the overwrite it. + /// and then overwrite it. disk->createFile(tmp_version_file_name); auto out = disk->writeFile(tmp_version_file_name, 256, WriteMode::Rewrite); version.write(*out); @@ -1170,6 +1170,34 @@ void IMergeTreeDataPart::appendCSNToVersionMetadata(VersionMetadata::WhichCSN wh 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; + } + + 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 metedata + if (!clear) + out->sync(); +} + void IMergeTreeDataPart::loadVersionMetadata() const try { @@ -1264,11 +1292,10 @@ bool IMergeTreeDataPart::assertHasValidVersionMetadata() const ReadBufferFromString str_buf{content}; VersionMetadata file; file.read(str_buf); - //FIXME bool valid_creation_tid = version.creation_tid == file.creation_tid; - bool valid_removal_tid = version.removal_tid == file.removal_tid || (version.removal_tid.isEmpty() && TransactionLog::getCSN(file.removal_tid) == Tx::UnknownCSN) || version.removal_tid == Tx::PrehistoricTID; + 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::RolledBackCSN || version.removal_csn == Tx::PrehistoricCSN; + 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; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1617ded464e..a80aab19118 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -444,6 +444,9 @@ public: /// 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; diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index a31030ca132..ab582aeae37 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -70,6 +70,7 @@ function thread_partition_dst_to_src() fi $CLICKHOUSE_CLIENT --multiquery --query " SYSTEM STOP MERGES dst; + ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick BEGIN TRANSACTION; INSERT INTO dst VALUES ($i, 4); INSERT INTO src SELECT * FROM dst; From 0906b59fba16afeb30b4e0f1eaf7d4c3e65f7575 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Mar 2022 21:38:18 +0100 Subject: [PATCH 059/372] fixes --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/MergeTreeTransaction.cpp | 44 +++++++++++-------- src/Interpreters/MergeTreeTransaction.h | 3 +- .../MergeTreeTransactionHolder.cpp | 31 ++++++++----- src/Interpreters/MergeTreeTransactionHolder.h | 5 ++- src/Interpreters/TransactionLog.cpp | 2 +- .../TransactionVersionMetadata.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 ++ .../MergeTree/MergeTreeMutationEntry.cpp | 3 -- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- 11 files changed, 62 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 51125c2dcc6..fa17e2a6a31 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3035,7 +3035,7 @@ void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) void Context::initCurrentTransaction(MergeTreeTransactionPtr txn) { - merge_tree_transaction_holder = MergeTreeTransactionHolder(txn, false); + merge_tree_transaction_holder = MergeTreeTransactionHolder(txn, false, this); setCurrentTransaction(std::move(txn)); } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 7c6f334d33f..c5bcd5a1573 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -36,7 +36,6 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart /// Now we know actual part name and can write it to system log table. tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); - new_part->assertHasVersionMetadata(txn); if (txn) txn->addNewPart(storage, new_part); @@ -44,31 +43,37 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) { - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; TransactionInfoContext context{storage->getStorageID(), part_to_remove->name}; - part_to_remove->version.lockMaxTID(tid, context); if (txn) - txn->removeOldPart(storage, part_to_remove); + txn->removeOldPart(storage, part_to_remove, context); + else + part_to_remove->version.lockMaxTID(Tx::PrehistoricTID, context); } void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - TransactionInfoContext context{storage->getStorageID(), new_part->name}; tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, context); + context.covering_part = std::move(context.part_name); new_part->assertHasVersionMetadata(txn); if (txn) - txn->addNewPart(storage, new_part); - - context.covering_part = std::move(context.part_name); - for (const auto & covered : covered_parts) { - context.part_name = covered->name; - covered->version.lockMaxTID(tid, context); - if (txn) - txn->removeOldPart(storage, covered); + txn->addNewPart(storage, new_part); + for (const auto & covered : covered_parts) + { + context.part_name = covered->name; + txn->removeOldPart(storage, covered, context); + } + } + else + { + for (const auto & covered : covered_parts) + { + context.part_name = covered->name; + covered->version.lockMaxTID(tid, context); + } } } @@ -84,16 +89,20 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart creating_parts.push_back(new_part); } -void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove) +void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context) { CSN c = csn.load(); if (c == Tx::RolledBackCSN) - throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled");//FIXME + throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); else if (c != Tx::UnknownCSN) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", c); - storages.insert(storage); - removing_parts.push_back(part_to_remove); + { + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); + part_to_remove->version.lockMaxTID(tid, context); + storages.insert(storage); + removing_parts.push_back(part_to_remove); + } part_to_remove->appendRemovalTIDToVersionMetadata(); } @@ -161,7 +170,6 @@ bool MergeTreeTransaction::rollback() noexcept part->version.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); } - /// FIXME const_cast for (const auto & part : creating_parts) const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 969db105352..baaf1ec360a 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -32,7 +32,7 @@ public: MergeTreeTransaction(Snapshot 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); + void removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context); void addMutation(const StoragePtr & table, const String & mutation_id); @@ -59,7 +59,6 @@ private: std::atomic csn; - /// FIXME it's ugly std::list::iterator snapshot_in_use_it; std::vector> mutations; diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index b1c50ff6d55..ab2558b0d49 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -1,30 +1,34 @@ #include #include #include +#include namespace DB { -MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_ = false) +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 - : txn(std::move(rhs.txn)) - , autocommit(rhs.autocommit) { - rhs.txn = {}; + *this = std::move(rhs); } MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(MergeTreeTransactionHolder && rhs) noexcept { onDestroy(); - txn = std::move(rhs.txn); - rhs.txn = {}; - autocommit = rhs.autocommit; + txn = nullptr; + 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; } @@ -56,14 +60,19 @@ void MergeTreeTransactionHolder::onDestroy() noexcept TransactionLog::instance().rollbackTransaction(txn); } -MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionHolder &) +MergeTreeTransactionHolder::MergeTreeTransactionHolder(const MergeTreeTransactionHolder & rhs) { - txn = nullptr; + *this = rhs; } -MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder &) // NOLINT +MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(const MergeTreeTransactionHolder & rhs) // NOLINT { - assert(txn == nullptr); + 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; } diff --git a/src/Interpreters/MergeTreeTransactionHolder.h b/src/Interpreters/MergeTreeTransactionHolder.h index 11ab0627d00..88f226e18a1 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.h +++ b/src/Interpreters/MergeTreeTransactionHolder.h @@ -4,6 +4,8 @@ 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; @@ -12,7 +14,7 @@ class MergeTreeTransactionHolder { public: MergeTreeTransactionHolder() = default; - MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_); + MergeTreeTransactionHolder(const MergeTreeTransactionPtr & txn_, bool autocommit_, const Context * owned_by_session_context_ = nullptr); MergeTreeTransactionHolder(MergeTreeTransactionHolder && rhs) noexcept; MergeTreeTransactionHolder & operator=(MergeTreeTransactionHolder && rhs) noexcept; ~MergeTreeTransactionHolder(); @@ -30,6 +32,7 @@ private: MergeTreeTransactionPtr txn; bool autocommit = false; + const Context * owned_by_session_context = nullptr; }; } diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 103c011b96e..170dfaa749d 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -140,6 +140,7 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite tid_to_csn.emplace(entry.first, entry.second); last_loaded_entry = last_entry; latest_snapshot = loaded.back().second; + local_tid_counter = Tx::MaxReservedLocalTID; }; LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); @@ -268,7 +269,6 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) txn->beforeCommit(); CSN new_csn; - /// TODO Transactions: reset local_tid_counter if (txn->isReadOnly()) { LOG_TEST(log, "Closing readonly transaction {}", txn->tid); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 4f5fb1ed38f..0536fe7dd21 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -78,7 +78,6 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction if (tid == Tx::PrehistoricTID && expected_max_lock_value == Tx::PrehistoricTID.getHash()) { /// Don't need to lock part for queries without transaction - //FIXME Transactions: why is it possible? LOG_TEST(log, "Assuming removal_tid is locked by {}, table: {}, part: {}", tid, context.table.getNameForLogs(), context.part_name); return true; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 835371e2a21..d61a1d9f8fc 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1279,6 +1279,9 @@ bool IMergeTreeDataPart::assertHasValidVersionMetadata() const if (!isStoredOnDisk()) return false; + if (part_is_probably_removed_from_disk) + return true; + DiskPtr disk = volume->getDisk(); if (!disk->exists(getFullRelativePath())) return true; @@ -1404,6 +1407,8 @@ std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const void IMergeTreeDataPart::remove() const { assert(assertHasValidVersionMetadata()); + part_is_probably_removed_from_disk = true; + std::optional keep_shared_data = keepSharedDataInDecoupledStorage(); if (!keep_shared_data.has_value()) return; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a80aab19118..6c4edaa0065 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -544,6 +544,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; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 1e7a5cb17ba..21b68cf33cf 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -53,11 +53,9 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) - , csn(Tx::UnknownCSN) { try { - /// TODO Transactions: write (and read) tid auto out = disk->writeFile(path_prefix + file_name); *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; @@ -131,7 +129,6 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat *buf >> "\n"; assertEOF(*buf); - csn = Tx::PrehistoricCSN; } MergeTreeMutationEntry::~MergeTreeMutationEntry() diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index fda7aabaa0e..80a33bfe0e3 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -83,7 +83,7 @@ bool MutatePlainMergeTreeTask::executeStep() new_part = mutate_task->getFuture().get(); - /// FIXME Transaction: it's too optimistic, better to lock parts before starting transaction + /// FIXME Transactions: it's too optimistic, better to lock parts before starting transaction storage.renameTempPartAndReplace(new_part, merge_mutate_entry->txn.get()); storage.updateMutationEntriesErrors(future_part, true, ""); write_part_log({}); From 7f47f20aba3299e3016ba000f3237ae45ee3aca3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Mar 2022 22:29:58 +0100 Subject: [PATCH 060/372] add kill transaction query --- src/Access/Common/AccessType.h | 1 + .../InterpreterKillQueryQuery.cpp | 44 +++++++++++++++ src/Interpreters/MergeTreeTransaction.cpp | 54 ++++++++++++------- src/Interpreters/MergeTreeTransaction.h | 10 +++- .../MergeTreeTransactionHolder.cpp | 5 ++ src/Interpreters/TransactionLog.cpp | 6 +++ src/Interpreters/TransactionLog.h | 5 +- src/Parsers/ASTKillQueryQuery.cpp | 3 ++ src/Parsers/ASTKillQueryQuery.h | 1 + src/Parsers/ParserKillQueryQuery.cpp | 3 ++ .../System/StorageSystemTransactions.cpp | 49 +++++++++++++++++ .../System/StorageSystemTransactions.h | 27 ++++++++++ src/Storages/System/attachSystemTables.cpp | 4 ++ .../01168_mutations_isolation.reference | 9 ++-- .../0_stateless/01168_mutations_isolation.sh | 11 +++- 15 files changed, 204 insertions(+), 28 deletions(-) create mode 100644 src/Storages/System/StorageSystemTransactions.cpp create mode 100644 src/Storages/System/StorageSystemTransactions.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 0b69bd5fd0e..f9fc5bb7ea2 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -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 */\ diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 5ec6abb08a7..481355878aa 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -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(*transactions_block.getByName("tid_hash").column); + + auto header = transactions_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "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(header.cloneWithColumns(std::move(res_columns))))); + break; + } } return res_io; diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index c5bcd5a1573..32b78e9b5ae 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -30,6 +30,15 @@ MergeTreeTransaction::State MergeTreeTransaction::getState() const 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) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; @@ -79,58 +88,61 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part) { - 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); - + std::lock_guard lock{mutex}; + checkIsNotCancelled(); storages.insert(storage); creating_parts.push_back(new_part); } void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, const TransactionInfoContext & context) { - 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); - { + std::lock_guard lock{mutex}; + checkIsNotCancelled(); + LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); part_to_remove->version.lockMaxTID(tid, context); storages.insert(storage); removing_parts.push_back(part_to_remove); } + part_to_remove->appendRemovalTIDToVersionMetadata(); } void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & mutation_id) { + std::lock_guard lock{mutex}; + checkIsNotCancelled(); storages.insert(table); 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(); } void MergeTreeTransaction::beforeCommit() { - for (const auto & table_and_mutation : mutations) - table_and_mutation.first->waitForMutation(table_and_mutation.second); - CSN expected = Tx::UnknownCSN; bool can_commit = csn.compare_exchange_strong(expected, Tx::CommittingCSN); - if (can_commit) - return; + if (!can_commit) + { + if (expected == Tx::RolledBackCSN) + throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", expected); + } - if (expected == Tx::RolledBackCSN) - throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", expected); + RunningMutationsList mutations_to_wait; + { + std::lock_guard lock{mutex}; + mutations_to_wait = mutations; + } + + for (const auto & table_and_mutation : mutations_to_wait) + table_and_mutation.first->waitForMutation(table_and_mutation.second); } void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept @@ -195,6 +207,8 @@ String MergeTreeTransaction::dumpDescription() const return res; } + std::lock_guard lock{mutex}; + res += fmt::format(", affects {} tables:", storages.size()); using ChangesInTable = std::tuple; diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index baaf1ec360a..c36a7b8ab05 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -46,10 +47,16 @@ public: String dumpDescription() const; + Float64 elapsedSeconds() const { return elapsed.elapsedSeconds(); } + private: void beforeCommit(); void afterCommit(CSN assigned_csn) noexcept; bool rollback() noexcept; + void checkIsNotCancelled() const; + + mutable std::mutex mutex; + Stopwatch elapsed; Snapshot snapshot; @@ -61,7 +68,8 @@ private: std::list::iterator snapshot_in_use_it; - std::vector> mutations; + using RunningMutationsList = std::vector>; + RunningMutationsList mutations; }; using MergeTreeTransactionPtr = std::shared_ptr; diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index ab2558b0d49..9077dff754b 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -6,6 +6,11 @@ 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_) diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 170dfaa749d..de5301a0535 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -372,4 +372,10 @@ Snapshot TransactionLog::getOldestSnapshot() const return snapshots_in_use.front(); } +TransactionLog::TransactionsList TransactionLog::getTransactionsList() const +{ + std::lock_guard lock{running_list_mutex}; + return running_list; +} + } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 97fc52294fb..2cae3fe69e8 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -75,6 +75,9 @@ public: MergeTreeTransactionPtr tryGetRunningTransaction(const TIDHash & tid); + using TransactionsList = std::unordered_map; + TransactionsList getTransactionsList() const; + private: void loadLogFromZooKeeper(); void runUpdatingThread(); @@ -100,7 +103,7 @@ private: std::unordered_map tid_to_csn; mutable std::mutex running_list_mutex; - std::unordered_map running_list; + TransactionsList running_list; std::list snapshots_in_use; String zookeeper_path; diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 71c3011dd2c..8bf99312544 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -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); diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index 6ff12bcba93..95be3ec6309 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -14,6 +14,7 @@ public: Query, /// KILL QUERY Mutation, /// KILL MUTATION PartMoveToShard, /// KILL PART_MOVE_TO_SHARD + Transaction, /// KILL TRANSACTION }; Type type = Type::Query; diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index bc895406c9f..0b1b37e61bf 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -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; diff --git a/src/Storages/System/StorageSystemTransactions.cpp b/src/Storages/System/StorageSystemTransactions.cpp new file mode 100644 index 00000000000..396fc875f74 --- /dev/null +++ b/src/Storages/System/StorageSystemTransactions.cpp @@ -0,0 +1,49 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +static DataTypePtr getStateEnumType() +{ + return std::make_shared( + DataTypeEnum8::Values + { + {"RUNNING", static_cast(MergeTreeTransaction::State::RUNNING)}, + {"COMMITTED", static_cast(MergeTreeTransaction::State::COMMITTED)}, + {"ROLLED_BACK", static_cast(MergeTreeTransaction::State::ROLLED_BACK)}, + }); +} + +NamesAndTypesList StorageSystemTransactions::getNamesAndTypes() +{ + return { + {"tid", getTransactionIDDataType()}, + {"tid_hash", std::make_shared()}, + {"elapsed", std::make_shared()}, + {"is_readonly", std::make_shared()}, + {"state", getStateEnumType()}, + }; +} + +void StorageSystemTransactions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + auto list = TransactionLog::instance().getTransactionsList(); + for (const auto & elem : list) + { + auto txn = elem.second; + size_t i = 0; + res_columns[i++]->insert(Tuple{txn->tid.start_csn, txn->tid.local_tid, txn->tid.host_id}); + res_columns[i++]->insert(txn->tid.getHash()); + res_columns[i++]->insert(txn->elapsedSeconds()); + res_columns[i++]->insert(txn->isReadOnly()); + res_columns[i++]->insert(txn->getState()); + } +} + +} diff --git a/src/Storages/System/StorageSystemTransactions.h b/src/Storages/System/StorageSystemTransactions.h new file mode 100644 index 00000000000..38244815549 --- /dev/null +++ b/src/Storages/System/StorageSystemTransactions.h @@ -0,0 +1,27 @@ +#pragma once +#include +#include + + +namespace DB +{ + +class Context; + +class StorageSystemTransactions final : public shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct shared_ptr_helper; +public: + String getName() const override { return "SystemTransactions"; } + + static NamesAndTypesList getNamesAndTypes(); + + static NamesAndAliases getNamesAndAliases() { return {}; } + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 023ced35a6b..2fe9b307e7b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -162,6 +163,9 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b if (has_zookeeper) attach(context, system_database, "zookeeper"); + + if (context->getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0) == 42) + attach(context, system_database, "transactions"); } void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics) diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index f7a1a707cfe..1b3e3f145b1 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -31,7 +31,8 @@ tx11 9 21 all_1_14_1_17 tx11 9 41 all_1_14_1_17 tx11 9 61 all_1_14_1_17 tx11 9 81 all_1_14_1_17 -tx13 10 22 all_1_14_1_18 -tx13 10 42 all_1_14_1_18 -tx13 10 62 all_1_14_1_18 -tx13 10 82 all_1_14_1_18 +1 1 RUNNING +tx14 10 22 all_1_14_1_18 +tx14 10 42 all_1_14_1_18 +tx14 10 62 all_1_14_1_18 +tx14 10 82 all_1_14_1_18 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 6b0fb15d7f7..5d91557dc6d 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -80,7 +80,14 @@ tx_async 12 "commit" >/dev/null tx_wait 11 tx_wait 12 -tx 13 "begin transaction" -tx 13 "select 10, n, _part from mt order by n" +tx 13 "begin transaction" +tid_to_kill=$(tx 13 "select transactionID()" | grep -Po "\(.*") +$CLICKHOUSE_CLIENT -q "select count(), any(is_readonly), any(state) from system.transactions where tid=$tid_to_kill" +tx_async 13 "alter table mt update n = 0 where 1" >/dev/null +$CLICKHOUSE_CLIENT -q "kill transaction where tid=$tid_to_kill format Null" +tx_sync 13 "rollback" + +tx 14 "begin transaction" +tx 14 "select 10, n, _part from mt order by n" $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" From c10a58e4bceceaf777a44c59b273d8bad0b2a8aa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Mar 2022 12:15:21 +0100 Subject: [PATCH 061/372] fix tests --- .../0_stateless/02117_show_create_table_system.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 1141b9357ad..f5aa6475794 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -20,7 +20,7 @@ CREATE TABLE system.errors\n(\n `name` String,\n `code` Int32,\n `value CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String\n)\nENGINE = SystemEvents()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToLineWithInlines\' = 120, \'addressToSymbol\' = 121, \'demangle\' = 122, \'INTROSPECTION\' = 123, \'FILE\' = 124, \'URL\' = 125, \'REMOTE\' = 126, \'MONGO\' = 127, \'MYSQL\' = 128, \'POSTGRES\' = 129, \'SQLITE\' = 130, \'ODBC\' = 131, \'JDBC\' = 132, \'HDFS\' = 133, \'S3\' = 134, \'SOURCES\' = 135, \'ALL\' = 136, \'NONE\' = 137),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'SOURCES\' = 136, \'ALL\' = 137, \'NONE\' = 138),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `rule_type` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' @@ -35,7 +35,7 @@ CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `visible` UInt8,\n `creation_tid` Tuple(UInt64, UInt64, UUID),\n `removal_tid` Tuple(UInt64, UInt64, UUID),\n `creation_csn` UInt64,\n `removal_csn` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `serialization_kind` String,\n `subcolumns.names` Array(String),\n `subcolumns.types` Array(String),\n `subcolumns.serializations` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.privileges\n(\n `privilege` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToLineWithInlines\' = 120, \'addressToSymbol\' = 121, \'demangle\' = 122, \'INTROSPECTION\' = 123, \'FILE\' = 124, \'URL\' = 125, \'REMOTE\' = 126, \'MONGO\' = 127, \'MYSQL\' = 128, \'POSTGRES\' = 129, \'SQLITE\' = 130, \'ODBC\' = 131, \'JDBC\' = 132, \'HDFS\' = 133, \'S3\' = 134, \'SOURCES\' = 135, \'ALL\' = 136, \'NONE\' = 137),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToLineWithInlines\' = 120, \'addressToSymbol\' = 121, \'demangle\' = 122, \'INTROSPECTION\' = 123, \'FILE\' = 124, \'URL\' = 125, \'REMOTE\' = 126, \'MONGO\' = 127, \'MYSQL\' = 128, \'POSTGRES\' = 129, \'SQLITE\' = 130, \'ODBC\' = 131, \'JDBC\' = 132, \'HDFS\' = 133, \'S3\' = 134, \'SOURCES\' = 135, \'ALL\' = 136, \'NONE\' = 137))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.privileges\n(\n `privilege` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'SOURCES\' = 136, \'ALL\' = 137, \'NONE\' = 138),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'SOURCES\' = 136, \'ALL\' = 137, \'NONE\' = 138))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.projection_parts_columns\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From 645629a01b2357aff620d2beedfb8076e7e92334 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Mar 2022 13:06:44 +0100 Subject: [PATCH 062/372] fix tests --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 06bd6ab04e4..831e65d021d 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -60,6 +60,7 @@ DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL KILL QUERY [] GLOBAL ALL +KILL TRANSACTION [] GLOBAL ALL MOVE PARTITION BETWEEN SHARDS [] GLOBAL ALL CREATE USER [] GLOBAL ACCESS MANAGEMENT ALTER USER [] GLOBAL ACCESS MANAGEMENT From 278d779a013bf8933ca9b7a7ec612f9fcb5d6b2e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Mar 2022 21:43:34 +0100 Subject: [PATCH 063/372] log cleanup, more comments --- programs/server/Server.cpp | 2 + src/Common/TransactionID.h | 14 +- src/Common/ZooKeeper/ZooKeeper.cpp | 10 + src/Common/ZooKeeper/ZooKeeper.h | 2 + src/Interpreters/Context.cpp | 18 ++ src/Interpreters/Context.h | 3 + src/Interpreters/DDLTask.cpp | 7 +- src/Interpreters/MergeTreeTransaction.cpp | 67 +++++-- src/Interpreters/MergeTreeTransaction.h | 8 +- src/Interpreters/TransactionLog.cpp | 176 ++++++++++++++---- src/Interpreters/TransactionLog.h | 79 +++++++- .../TransactionVersionMetadata.cpp | 146 ++++++++------- src/Interpreters/TransactionVersionMetadata.h | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/StorageMergeTree.cpp | 1 + tests/clickhouse-test | 12 ++ 17 files changed, 417 insertions(+), 152 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b856131d821..0f46faf8670 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1455,6 +1455,8 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Listening for {}", server.getDescription()); } + + global_context->setServerCompletelyStarted(); LOG_INFO(log, "Ready for connections."); } diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 2652667ddd1..902b119111b 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -11,7 +11,6 @@ class IDataType; using DataTypePtr = std::shared_ptr; class MergeTreeTransaction; -/// FIXME Transactions: Sequential node numbers in ZooKeeper are Int32, but 31 bit is not enough using CSN = UInt64; using Snapshot = CSN; using LocalTID = UInt64; @@ -21,8 +20,8 @@ namespace Tx { const CSN UnknownCSN = 0; const CSN PrehistoricCSN = 1; - const CSN CommittingCSN = 2; /// TODO do we really need it? - const CSN MaxReservedCSN = 2; + const CSN CommittingCSN = 2; + const CSN MaxReservedCSN = 16; const LocalTID PrehistoricLocalTID = 1; const LocalTID DummyLocalTID = 1; @@ -31,10 +30,19 @@ namespace Tx 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; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b1574341c40..394938194e7 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1184,4 +1184,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::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; +} + } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 371f93f6df3..956d5d417cb 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -386,4 +386,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); + } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fa17e2a6a31..d6b328a0380 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -277,6 +277,8 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; + bool is_server_completely_started = false; + ContextSharedPart() : access_control(std::make_unique()) , global_overcommit_tracker(&process_list) @@ -3053,6 +3055,22 @@ 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(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 44db66b0d50..dffd00954fd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -887,6 +887,9 @@ public: void setCurrentTransaction(MergeTreeTransactionPtr txn); MergeTreeTransactionPtr getCurrentTransaction() const; + bool isServerCompletelyStarted() const; + void setServerCompletelyStarted(); + PartUUIDsPtr getPartUUIDs() const; PartUUIDsPtr getIgnoredPartUUIDs() const; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index a490d7bed43..fb51394af9f 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -390,12 +390,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::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) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 32b78e9b5ae..1ca22d634cc 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -MergeTreeTransaction::MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id) +MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, UUID host_id) : tid({snapshot_, local_tid_, host_id}) , snapshot(snapshot_) , csn(Tx::UnknownCSN) @@ -41,22 +41,33 @@ void MergeTreeTransaction::checkIsNotCancelled() const void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPartPtr & new_part, MergeTreeTransaction * txn) { - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - - /// Now we know actual part name and can write it to system log table. - tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, TransactionInfoContext{storage->getStorageID(), new_part->name}); + /// 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 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, context); + } else - part_to_remove->version.lockMaxTID(Tx::PrehistoricTID, context); + { + /// Lock part for removal with special TID, so transactions will no try to remove it concurrently. + /// We lock it only in memory. + part_to_remove->version.lockRemovalTID(Tx::PrehistoricTID, context); + } } void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) @@ -81,7 +92,7 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage for (const auto & covered : covered_parts) { context.part_name = covered->name; - covered->version.lockMaxTID(tid, context); + covered->version.lockRemovalTID(tid, context); } } } @@ -101,7 +112,7 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP checkIsNotCancelled(); LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - part_to_remove->version.lockMaxTID(tid, context); + part_to_remove->version.lockRemovalTID(tid, context); storages.insert(storage); removing_parts.push_back(part_to_remove); } @@ -126,27 +137,37 @@ bool MergeTreeTransaction::isReadOnly() const void MergeTreeTransaction::beforeCommit() { - CSN expected = Tx::UnknownCSN; - bool can_commit = csn.compare_exchange_strong(expected, Tx::CommittingCSN); - if (!can_commit) - { - if (expected == Tx::RolledBackCSN) - throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction was cancelled"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CSN state: {}", expected); - } - 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); + } } 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) @@ -167,21 +188,31 @@ 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. + + /// Forcefully stop related mutations if any for (const auto & table_and_mutation : mutations) table_and_mutation.first->killMutation(table_and_mutation.second); + /// Kind of optimization: cleanup thread can remove these parts immediately for (const auto & part : creating_parts) part->version.creation_csn.store(Tx::RolledBackCSN); for (const auto & part : removing_parts) { + /// 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.unlockMaxTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); + part->version.unlockRemovalTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); } + /// 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 : creating_parts) const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index c36a7b8ab05..a9d338c87bd 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -25,12 +25,12 @@ public: ROLLED_BACK, }; - Snapshot getSnapshot() const { return snapshot; } + CSN getSnapshot() const { return snapshot; } State getState() const; const TransactionID tid; - MergeTreeTransaction(Snapshot snapshot_, LocalTID local_tid_, UUID host_id); + 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); @@ -58,7 +58,7 @@ private: mutable std::mutex mutex; Stopwatch elapsed; - Snapshot snapshot; + CSN snapshot; std::unordered_set storages; DataPartsVector creating_parts; @@ -66,7 +66,7 @@ private: std::atomic csn; - std::list::iterator snapshot_in_use_it; + std::list::iterator snapshot_in_use_it; using RunningMutationsList = std::vector>; RunningMutationsList mutations; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index de5301a0535..ae453bf40c4 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -10,6 +10,11 @@ #include #include + +/// 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 { @@ -45,7 +50,8 @@ TransactionLog::TransactionLog() global_context = Context::getGlobalContextInstance(); global_context->checkTransactionsAreAllowed(); - zookeeper_path = "/test/clickhouse/txn_log"; + zookeeper_path = global_context->getConfigRef().getString("transaction_log.zookeeper_path", "/test/clickhouse/txn"); + zookeeper_path_log = zookeeper_path + "/log"; loadLogFromZooKeeper(); @@ -86,6 +92,11 @@ UInt64 TransactionLog::parseCSN(const String & csn_node_name) return res; } +String TransactionLog::writeCSN(CSN csn) +{ + return zkutil::getSequentialNodeName("csn-", csn); +} + TransactionID TransactionLog::parseTID(const String & csn_node_content) { TransactionID tid = Tx::EmptyTID; @@ -114,12 +125,12 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite return; String last_entry = *std::prev(end); - LOG_TRACE(log, "Loading {} entries from {}: {}..{}", entries_count, zookeeper_path, *beg, last_entry); + 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) / *it)); + futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path_log) / *it)); - std::vector> loaded; + std::vector> loaded; loaded.reserve(entries_count); auto it = beg; for (size_t i = 0; i < entries_count; ++i, ++it) @@ -127,25 +138,24 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite auto res = futures[i].get(); CSN csn = parseCSN(*it); TransactionID tid = parseTID(res.data); - loaded.emplace_back(tid.getHash(), csn); + loaded.emplace_back(tid.getHash(), CSNEntry{csn, tid}); LOG_TEST(log, "Got entry {} -> {}", tid, csn); } futures.clear(); - /// Use noexcept here to exit on unexpected exceptions (SIGABRT is better that broken state in memory) - auto insert = [&]() noexcept - { - for (const auto & entry : loaded) - if (entry.first != Tx::EmptyTID.getHash()) - tid_to_csn.emplace(entry.first, entry.second); - last_loaded_entry = last_entry; - latest_snapshot = loaded.back().second; - local_tid_counter = Tx::MaxReservedLocalTID; - }; - + NOEXCEPT_SCOPE; LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); std::lock_guard lock{mutex}; - insert(); + 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() @@ -158,15 +168,19 @@ void TransactionLog::loadLogFromZooKeeper() /// 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 + "/csn-", "", zkutil::CreateMode::PersistentSequential); + 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); + zookeeper->createAncestors(zookeeper_path_log); Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/tail_ptr", writeCSN(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 + "/csn-", "", zkutil::CreateMode::PersistentSequential)); + 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) @@ -177,13 +191,15 @@ void TransactionLog::loadLogFromZooKeeper() /// 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, nullptr, log_updated_event); + 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 == parseCSN(last_loaded_entry)); local_tid_counter = Tx::MaxReservedLocalTID; + + tail_ptr = parseCSN(zookeeper->get(zookeeper_path + "/tail_ptr")); } void TransactionLog::runUpdatingThread() @@ -204,6 +220,7 @@ void TransactionLog::runUpdatingThread() } loadNewEntries(); + removeOldEntries(); } catch (const Coordination::Exception & e) { @@ -228,7 +245,7 @@ void TransactionLog::runUpdatingThread() void TransactionLog::loadNewEntries() { - Strings entries_list = zookeeper->getChildren(zookeeper_path, nullptr, log_updated_event); + 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); @@ -238,8 +255,64 @@ void TransactionLog::loadNewEntries() 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; -Snapshot TransactionLog::getLatestSnapshot() const + /// 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 = parseCSN(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); + + /// (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", writeCSN(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 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 + "/" + writeCSN(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(); } @@ -249,7 +322,7 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() MergeTreeTransactionPtr txn; { std::lock_guard lock{running_list_mutex}; - Snapshot snapshot = latest_snapshot.load(); + CSN snapshot = latest_snapshot.load(); LocalTID ltid = 1 + local_tid_counter.fetch_add(1); txn = std::make_shared(snapshot, ltid, ServerUUID::get()); bool inserted = running_list.try_emplace(txn->tid.getHash(), txn).second; @@ -266,11 +339,13 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) { + /// Some precommit checks, may throw 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); @@ -281,13 +356,17 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// TODO handle connection loss /// TODO support batching auto current_zookeeper = getZooKeeper(); - String path_created = current_zookeeper->create(zookeeper_path + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point - new_csn = parseCSN(path_created.substr(zookeeper_path.size() + 1)); + String path_created = current_zookeeper->create(zookeeper_path_log + "/csn-", writeTID(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 = parseCSN(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 will see changes + /// 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) @@ -297,9 +376,11 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) } } + /// 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) @@ -316,7 +397,10 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no std::uncaught_exceptions() ? fmt::format(" due to uncaught exception (code: {})", getCurrentExceptionCode()) : ""); if (!txn->rollback()) + { + /// Transaction was cancelled concurrently, it's already rolled back. return; + } { std::lock_guard lock{running_list_mutex}; @@ -340,7 +424,10 @@ MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & CSN TransactionLog::getCSN(const TransactionID & tid) { - return getCSN(tid.getHash()); + /// 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) @@ -348,23 +435,36 @@ 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) const +CSN TransactionLog::getCSNImpl(const TIDHash & tid_hash) const { - assert(tid); - assert(tid != Tx::EmptyTID.getHash()); + assert(tid_hash); + assert(tid_hash != Tx::EmptyTID.getHash()); std::lock_guard lock{mutex}; - auto it = tid_to_csn.find(tid); - if (it == tid_to_csn.end()) - return Tx::UnknownCSN; - return it->second; + auto it = tid_to_csn.find(tid_hash); + if (it != tid_to_csn.end()) + return it->second.csn; + + return Tx::UnknownCSN; } -Snapshot TransactionLog::getOldestSnapshot() const +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 + 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()) diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 2cae3fe69e8..67d8e709900 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -50,6 +50,35 @@ class TransactionsInfoLog; using TransactionsInfoLogPtr = std::shared_ptr; using ZooKeeperPtr = std::shared_ptr; +/// 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 { public: @@ -60,22 +89,37 @@ public: void shutdown(); - Snapshot getLatestSnapshot() const; - Snapshot getOldestSnapshot() const; + /// Returns the newest snapshot available for reading + CSN getLatestSnapshot() const; + /// Returns the oldest snapshot that is visible for some running transaction + CSN getOldestSnapshot() const; - /// Allocated TID, returns transaction object + /// 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; + /// Returns copy of list of running transactions. TransactionsList getTransactionsList() const; private: @@ -84,31 +128,52 @@ private: void loadEntries(Strings::const_iterator beg, Strings::const_iterator end); void loadNewEntries(); + void removeOldEntries(); static UInt64 parseCSN(const String & csn_node_name); + static String writeCSN(CSN csn); static TransactionID parseTID(const String & csn_node_content); static String writeTID(const TransactionID & tid); ZooKeeperPtr getZooKeeper() const; - CSN getCSNImpl(const TIDHash & tid) const; + CSN getCSNImpl(const TIDHash & tid_hash) const; ContextPtr global_context; Poco::Logger * log; + /// The newest snapshot available for reading std::atomic latest_snapshot; + + /// Local part of TransactionID number. We reset this counter for each new snapshot. std::atomic local_tid_counter; mutable std::mutex mutex; - std::unordered_map tid_to_csn; + /// 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; + TIDMap tid_to_csn; mutable std::mutex running_list_mutex; + /// Transactions that are currently processed TransactionsList running_list; - std::list snapshots_in_use; + /// Ordered list of snapshots that are currently used by some transactions. Needed for background cleanup. + std::list snapshots_in_use; - String zookeeper_path; 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 tail_ptr = Tx::UnknownCSN; + zkutil::EventPtr log_updated_event = std::make_shared(); std::atomic_bool stop_flag = false; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 0536fe7dd21..20ebc1170ed 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -20,6 +20,22 @@ namespace ErrorCodes extern const int CANNOT_PARSE_TEXT; } +inline static CSN getCSNAndAssert(TIDHash tid_hash, std::atomic & csn, const TransactionID * tid = nullptr) +{ + CSN maybe_csn = TransactionLog::getCSN(tid_hash); + if (maybe_csn) + return maybe_csn; + + 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?) @@ -29,12 +45,12 @@ VersionMetadata::VersionMetadata() /// It can be used for introspection purposes only TransactionID VersionMetadata::getRemovalTID() const { - TIDHash max_lock = removal_tid_lock.load(); - if (max_lock) + TIDHash removal_lock = removal_tid_lock.load(); + if (removal_lock) { - if (max_lock == Tx::PrehistoricTID.getHash()) + if (removal_lock == Tx::PrehistoricTID.getHash()) return Tx::PrehistoricTID; - if (auto txn = TransactionLog::instance().tryGetRunningTransaction(max_lock)) + if (auto txn = TransactionLog::instance().tryGetRunningTransaction(removal_lock)) return txn->tid; } @@ -47,11 +63,11 @@ TransactionID VersionMetadata::getRemovalTID() const return Tx::EmptyTID; } -void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) +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 (tryLockMaxTID(tid, context, &locked_by)) + if (tryLockRemovalTID(tid, context, &locked_by)) return; String part_desc; @@ -66,16 +82,16 @@ void VersionMetadata::lockMaxTID(const TransactionID & tid, const TransactionInf tid, part_desc, context.table.getNameForLogs(), getRemovalTID(), locked_by); } -bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id) +bool VersionMetadata::tryLockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id) { assert(!tid.isEmpty()); assert(!creation_tid.isEmpty()); - TIDHash max_lock_value = tid.getHash(); - TIDHash expected_max_lock_value = 0; - bool locked = removal_tid_lock.compare_exchange_strong(expected_max_lock_value, max_lock_value); + 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_max_lock_value == Tx::PrehistoricTID.getHash()) + 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); @@ -83,7 +99,7 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction } if (locked_by_id) - *locked_by_id = expected_max_lock_value; + *locked_by_id = expected_removal_lock_value; return false; } @@ -92,21 +108,21 @@ bool VersionMetadata::tryLockMaxTID(const TransactionID & tid, const Transaction return true; } -void VersionMetadata::unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context) +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 max_lock_value = tid.getHash(); + 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: {} {}", - max_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID); + removal_lock_value, tid, locked_by, locked_by_txn ? locked_by_txn->tid : Tx::EmptyTID); }; - if (locked_by != max_lock_value) + if (locked_by != removal_lock_value) throw_cannot_unlock(); removal_tid = Tx::EmptyTID; @@ -136,22 +152,20 @@ bool VersionMetadata::isVisible(const MergeTreeTransaction & txn) return isVisible(txn.getSnapshot(), txn.tid); } -bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current_tid) +bool VersionMetadata::isVisible(CSN snapshot_version, TransactionID current_tid) { assert(!creation_tid.isEmpty()); - CSN min = creation_csn.load(std::memory_order_relaxed); - TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); - CSN max = removal_csn.load(std::memory_order_relaxed); + 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); - //LOG_TEST(log, "Checking if creation_tid {} creation_csn {} removal_tidhash {} removal_csn {} visible for {} {}", creation_tid, min, max_lock, max, snapshot_version, current_tid); - - [[maybe_unused]] bool had_creation_csn = min; - [[maybe_unused]] bool had_removal_tid = max_lock; - [[maybe_unused]] bool had_removal_csn = max; + [[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(min == Tx::UnknownCSN || min == Tx::PrehistoricCSN || Tx::MaxReservedCSN < min); - assert(max == Tx::UnknownCSN || max == Tx::PrehistoricCSN || Tx::MaxReservedCSN < max); + assert(creation == Tx::UnknownCSN || creation == Tx::PrehistoricCSN || Tx::MaxReservedCSN < creation); + assert(removal == Tx::UnknownCSN || removal == Tx::PrehistoricCSN || Tx::MaxReservedCSN < removal); /// Fast path: @@ -159,20 +173,20 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// - creation was committed after we took the snapshot /// - removal was committed before we took the snapshot /// - current transaction is removing it - if (min && snapshot_version < min) + if (creation && snapshot_version < creation) return false; - if (max && max <= snapshot_version) + if (removal && removal <= snapshot_version) return false; - if (!current_tid.isEmpty() && max_lock && max_lock == current_tid.getHash()) + 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 (min && min <= snapshot_version && !max_lock) + if (creation && creation <= snapshot_version && !removal_lock) return true; - if (min && min <= snapshot_version && max && snapshot_version < max) + if (creation && creation <= snapshot_version && removal && snapshot_version < removal) return true; if (!current_tid.isEmpty() && creation_tid == current_tid) return true; @@ -183,7 +197,7 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current /// 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 && max_lock != current_tid.getHash())); + 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 @@ -193,28 +207,30 @@ bool VersionMetadata::isVisible(Snapshot snapshot_version, TransactionID current return false; /// Check if creation_tid/removal_tid transactions are committed and write CSNs - /// TODO Transactions: we probably need some optimizations here + /// 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 committer/rolled back. - min = TransactionLog::getCSN(creation_tid); - if (!min) + 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(min, std::memory_order_relaxed); + creation_csn.store(creation, std::memory_order_relaxed); - if (max_lock) + if (removal_lock) { - max = TransactionLog::getCSN(max_lock); - if (max) - removal_csn.store(max, std::memory_order_relaxed); + removal = getCSNAndAssert(removal_lock, removal_csn); + if (removal) + removal_csn.store(removal, std::memory_order_relaxed); } - return min <= snapshot_version && (!max || snapshot_version < max); + return creation <= snapshot_version && (!removal || snapshot_version < removal); } bool VersionMetadata::canBeRemoved() @@ -223,56 +239,56 @@ bool VersionMetadata::canBeRemoved() { /// Avoid access to Transaction log if transactions are not involved - TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); - if (!max_lock) + TIDHash removal_lock = removal_tid_lock.load(std::memory_order_relaxed); + if (!removal_lock) return false; - if (max_lock == Tx::PrehistoricTID.getHash()) + if (removal_lock == Tx::PrehistoricTID.getHash()) return true; } return canBeRemovedImpl(TransactionLog::instance().getOldestSnapshot()); } -bool VersionMetadata::canBeRemovedImpl(Snapshot oldest_snapshot_version) +bool VersionMetadata::canBeRemovedImpl(CSN oldest_snapshot_version) { - CSN min = creation_csn.load(std::memory_order_relaxed); + CSN creation = creation_csn.load(std::memory_order_relaxed); /// We can safely remove part if its creation was rolled back - if (min == Tx::RolledBackCSN) + if (creation == Tx::RolledBackCSN) return true; - if (!min) + if (!creation) { /// Cannot remove part if its creation not committed yet - min = TransactionLog::getCSN(creation_tid); - if (min) - creation_csn.store(min, std::memory_order_relaxed); + 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 < min) + if (oldest_snapshot_version < creation) return false; - TIDHash max_lock = removal_tid_lock.load(std::memory_order_relaxed); + TIDHash removal_lock = removal_tid_lock.load(std::memory_order_relaxed); /// Part is active - if (!max_lock) + if (!removal_lock) return false; - CSN max = removal_csn.load(std::memory_order_relaxed); - if (!max) + CSN removal = removal_csn.load(std::memory_order_relaxed); + if (!removal) { /// Part removal is not committed yet - max = TransactionLog::getCSN(max_lock); - if (max) - removal_csn.store(max, std::memory_order_relaxed); + 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 max <= oldest_snapshot_version; + return removal <= oldest_snapshot_version; } #define CREATION_TID_STR "creation_tid: " @@ -285,20 +301,20 @@ void VersionMetadata::writeCSN(WriteBuffer & buf, WhichCSN which_csn, bool inter { if (which_csn == CREATION) { - if (CSN min = creation_csn.load()) + if (CSN creation = creation_csn.load()) { writeCString("\n" CREATION_CSN_STR, buf); - writeText(min, 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 max = removal_csn.load()) + if (CSN removal = removal_csn.load()) { writeCString("\n" REMOVAL_CSN_STR, buf); - writeText(max, buf); + writeText(removal, buf); } else if (!internal) throw Exception(ErrorCodes::LOGICAL_ERROR, "writeCSN called for removal_csn = 0, it's a bug"); diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index e691d45dd1d..08af2e15fad 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -31,14 +31,14 @@ struct VersionMetadata /// Checks if an object is visible for transaction or not. bool isVisible(const MergeTreeTransaction & txn); - bool isVisible(Snapshot snapshot_version, TransactionID current_tid = Tx::EmptyTID); + bool isVisible(CSN snapshot_version, TransactionID current_tid = Tx::EmptyTID); TransactionID getCreationTID() const { return creation_tid; } TransactionID getRemovalTID() const; - bool tryLockMaxTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr); - void lockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); - void unlockMaxTID(const TransactionID & tid, const TransactionInfoContext & context); + bool tryLockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context, TIDHash * locked_by_id = nullptr); + void lockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context); + void unlockRemovalTID(const TransactionID & tid, const TransactionInfoContext & context); bool isRemovalTIDLocked() const; @@ -47,7 +47,7 @@ struct VersionMetadata /// Checks if it's safe to remove outdated version of an object bool canBeRemoved(); - bool canBeRemovedImpl(Snapshot oldest_snapshot_version); + bool canBeRemovedImpl(CSN oldest_snapshot_version); void write(WriteBuffer & buf) const; void read(ReadBuffer & buf); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee9120dbceb..1f598be6896 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1366,6 +1366,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) 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={}", @@ -1384,10 +1385,11 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } 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.unlockMaxTID(version.removal_tid, TransactionInfoContext{getStorageID(), part->name}); + version.unlockRemovalTID(version.removal_tid, TransactionInfoContext{getStorageID(), part->name}); } version_updated = true; } @@ -4030,14 +4032,14 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(const Me return res; } -MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(Snapshot snapshot_version, TransactionID current_tid) const +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, Snapshot snapshot_version, TransactionID current_tid) const +void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts, CSN snapshot_version, TransactionID current_tid) const { if (maybe_visible_parts.empty()) return; @@ -4559,7 +4561,7 @@ void MergeTreeData::Transaction::rollback() DataPartPtr covering_part; DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, lock); for (auto & covered : covered_parts) - covered->version.unlockMaxTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name}); + covered->version.unlockRemovalTID(Tx::PrehistoricTID, TransactionInfoContext{data.getStorageID(), covered->name}); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e45c8fab54b..0040d1c903f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -446,12 +446,12 @@ public: DataParts getDataPartsForInternalUsage() const; DataPartsVector getDataPartsVectorForInternalUsage() const; - void filterVisibleDataParts(DataPartsVector & maybe_visible_parts, Snapshot snapshot_version, TransactionID current_tid) const; + void filterVisibleDataParts(DataPartsVector & maybe_visible_parts, CSN snapshot_version, TransactionID current_tid) const; /// Returns parts that visible with current snapshot DataPartsVector getVisibleDataPartsVector(ContextPtr local_context) const; DataPartsVector getVisibleDataPartsVector(const MergeTreeTransactionPtr & txn) const; - DataPartsVector getVisibleDataPartsVector(Snapshot snapshot_version, TransactionID current_tid) const; + DataPartsVector getVisibleDataPartsVector(CSN snapshot_version, TransactionID current_tid) const; /// 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; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fc14d055d63..70b88012a65 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -722,6 +722,7 @@ void StorageMergeTree::loadMutations() { if (!TransactionLog::getCSN(entry.tid)) { + TransactionLog::assertTIDIsNotOutdated(entry.tid); LOG_DEBUG(log, "Mutation entry {} was created by transaction {}, but it was not committed. Removing mutation entry", it->name(), entry.tid); disk->removeFile(it->path()); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 121a283d0e4..d57056333e4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -201,6 +201,17 @@ def get_processlist(args): else: return clickhouse_execute_json(args, 'SHOW PROCESSLIST') +def get_transactions_list(args): + try: + if args.replicated_database: + return clickhouse_execute_json(args, """ + SELECT materialize((hostName(), tcpPort())) as host, * + FROM clusterAllReplicas('test_cluster_database_replicated', system.transactions) + """) + else: + return clickhouse_execute_json(args, 'select * from system.transactions') + except Exception as e: + return f"Cannot get list of transactions: {e}" # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): @@ -1320,6 +1331,7 @@ def main(args): if processlist: print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) print(json.dumps(processlist, indent=4)) + print(get_transactions_list(args)) print_stacktraces() exit_code.value = 1 From c8d6c13c2dec8fecb4f9ec85dccbcd0f563c248b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Mar 2022 00:08:25 +0100 Subject: [PATCH 064/372] fix --- src/Interpreters/MergeTreeTransaction.cpp | 3 ++- tests/integration/test_transactions/test.py | 28 ++++++++++----------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 1ca22d634cc..263282f36d5 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -147,7 +147,8 @@ void MergeTreeTransaction::beforeCommit() for (const auto & table_and_mutation : mutations_to_wait) table_and_mutation.first->waitForMutation(table_and_mutation.second); - assert([&]() { + assert([&]() + { std::lock_guard lock{mutex}; return mutations == mutations_to_wait; }()); diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 60806e91027..66b5f13213a 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -70,24 +70,24 @@ def test_rollback_unfinished_on_restart(start_cluster): node.restart_clickhouse(kill=True) assert node.query('select *, _part from mt order by n') == '2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n9\t90\t1_5_5_0\n' - res = node.query("select name, active, creation_tid, 'csn' || toString(creation_csn), removal_tid, 'csn' || toString(removal_csn) from system.parts where table='mt' order by name") + res = node.query("select name, active, creation_tid, 'csn' || toString(creation_csn) || '_', removal_tid, 'csn' || toString(removal_csn) || '_' from system.parts where table='mt' order by name") res = res.replace(tid0, 'tid0') - res = res.replace(tid1, 'tid1').replace('csn' + csn1, 'csn_1') - res = res.replace(tid2, 'tid2').replace('csn' + csn2, 'csn_2') + res = res.replace(tid1, 'tid1').replace('csn' + csn1 + '_', 'csn_1') + res = res.replace(tid2, 'tid2').replace('csn' + csn2 + '_', 'csn_2') res = res.replace(tid3, 'tid3') res = res.replace(tid4, 'tid4') res = res.replace(tid5, 'tid5') - res = res.replace(tid6, 'tid6').replace('csn' + csn6, 'csn_6') - assert res == "0_2_2_0\t1\ttid0\tcsn1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "0_2_4_1\t0\ttid4\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "0_8_8_0\t0\ttid5\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_1_1_0\t0\ttid0\tcsn1\ttid1\tcsn_1\n" \ - "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_6_6_0\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" \ - "1_6_6_0_7\t0\ttid3\tcsn18446744073709551615\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0\n" + res = res.replace(tid6, 'tid6').replace('csn' + csn6 + '_', 'csn_6') + assert res == "0_2_2_0\t1\ttid0\tcsn1_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "0_2_4_1\t0\ttid4\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "0_8_8_0\t0\ttid5\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n" \ + "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "1_6_6_0\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ + "1_6_6_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" From 9c3e4cdc6e33a2976febed0277aa138128e2e111 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Mar 2022 14:35:48 +0100 Subject: [PATCH 065/372] fix --- src/Interpreters/MergeTreeTransaction.cpp | 22 +++++++++++++++++++--- src/Interpreters/MergeTreeTransaction.h | 3 ++- src/Interpreters/TransactionLog.cpp | 3 ++- 3 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 263282f36d5..c03ce2f530c 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -135,7 +135,7 @@ bool MergeTreeTransaction::isReadOnly() const return storages.empty(); } -void MergeTreeTransaction::beforeCommit() +scope_guard MergeTreeTransaction::beforeCommit() { RunningMutationsList mutations_to_wait; { @@ -162,6 +162,13 @@ void MergeTreeTransaction::beforeCommit() 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 @@ -196,10 +203,19 @@ bool MergeTreeTransaction::rollback() noexcept /// 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. - /// Forcefully stop related mutations if any - for (const auto & table_and_mutation : mutations) + /// Forcefully stop related mutations if any (call killMutation with unlocked mutex) + RunningMutationsList mutations_to_kill; + { + std::lock_guard lock{mutex}; + mutations_to_kill = mutations; + } + + for (const auto & table_and_mutation : mutations_to_kill) table_and_mutation.first->killMutation(table_and_mutation.second); + std::lock_guard lock{mutex}; + assert(mutations == mutations_to_kill); + /// Kind of optimization: cleanup thread can remove these parts immediately for (const auto & part : creating_parts) part->version.creation_csn.store(Tx::RolledBackCSN); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index a9d338c87bd..8d28c25fc51 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -50,7 +51,7 @@ public: Float64 elapsedSeconds() const { return elapsed.elapsedSeconds(); } private: - void beforeCommit(); + scope_guard beforeCommit(); void afterCommit(CSN assigned_csn) noexcept; bool rollback() noexcept; void checkIsNotCancelled() const; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index ae453bf40c4..3315387edc5 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -340,7 +340,7 @@ MergeTreeTransactionPtr TransactionLog::beginTransaction() CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) { /// Some precommit checks, may throw - txn->beforeCommit(); + auto committing_lock = txn->beforeCommit(); CSN new_csn; if (txn->isReadOnly()) @@ -399,6 +399,7 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no if (!txn->rollback()) { /// Transaction was cancelled concurrently, it's already rolled back. + assert(txn->csn == Tx::RolledBackCSN); return; } From 009e5d01302547cbff0760d2d1ebbe69ff7b0c68 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Mar 2022 14:52:34 +0100 Subject: [PATCH 066/372] disable test with s3 --- tests/queries/0_stateless/01172_transaction_counters.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index e77d3e16a22..5431673fd62 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,3 +1,5 @@ +-- Tags: no-s3-storage +-- FIXME this test fails with S3 due to a bug in DiskCacheWrapper drop table if exists txn_counters; create table txn_counters (n Int64, creation_tid DEFAULT transactionID()) engine=MergeTree order by n; From b5b3d09b4a3ce11c9d4d34c01ea6f1904bef4844 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Mar 2022 17:51:53 +0100 Subject: [PATCH 067/372] fix --- src/Interpreters/MergeTreeTransaction.cpp | 27 ++++++++++++++++------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index c03ce2f530c..6225bb9dc0e 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -203,24 +203,26 @@ bool MergeTreeTransaction::rollback() noexcept /// 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. - /// Forcefully stop related mutations if any (call killMutation with unlocked mutex) 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); - std::lock_guard lock{mutex}; - assert(mutations == mutations_to_kill); - /// Kind of optimization: cleanup thread can remove these parts immediately - for (const auto & part : creating_parts) + for (const auto & part : parts_to_remove) part->version.creation_csn.store(Tx::RolledBackCSN); - for (const auto & part : removing_parts) + 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. @@ -230,13 +232,22 @@ bool MergeTreeTransaction::rollback() noexcept /// 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 : creating_parts) + for (const auto & part : parts_to_remove) const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); - for (const auto & part : removing_parts) + for (const auto & part : parts_to_activate) if (part->version.getCreationTID() != tid) const_cast(part->storage).restoreAndActivatePart(part); + 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; } From c2ac8d4a5cc9056f4a827a9c765ac16324e1ec6a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Mar 2022 20:16:26 +0100 Subject: [PATCH 068/372] review fixes --- src/Common/ErrorCodes.cpp | 4 +- src/Common/TransactionID.h | 8 ++++ src/IO/ReadHelpers.cpp | 2 +- src/Interpreters/Context.cpp | 3 +- .../InterpreterTransactionControlQuery.cpp | 4 +- src/Interpreters/MergeTreeTransaction.cpp | 24 ++++++------ .../MergeTreeTransactionHolder.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 34 ++++++++--------- src/Interpreters/TransactionLog.h | 8 ++-- src/Interpreters/TransactionVersionMetadata.h | 5 +++ src/Storages/MergeTree/DataPartsExchange.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++- .../MergeTree/MergeFromLogEntryTask.cpp | 6 +-- .../MergeTree/MergeMutateSelectedEntry.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 17 ++++----- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MutateFromLogEntryTask.cpp | 6 +-- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 38 +++++++++---------- src/Storages/System/attachSystemTables.cpp | 2 +- tests/config/config.d/transactions.xml | 2 +- .../configs/transactions.xml | 2 +- .../0_stateless/01168_mutations_isolation.sh | 2 +- .../01169_alter_partition_isolation_stress.sh | 8 ++-- .../01171_mv_select_insert_isolation_long.sh | 8 ++-- .../01174_select_insert_isolation.sh | 6 +-- 33 files changed, 118 insertions(+), 104 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 40dbbb5bc29..1d44ddbb46e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -613,9 +613,9 @@ M(642, CANNOT_PACK_ARCHIVE) \ M(643, CANNOT_UNPACK_ARCHIVE) \ M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \ + M(645, INVALID_TRANSACTION) \ + M(646, SERIALIZATION_ERROR) \ \ - M(700, INVALID_TRANSACTION) \ - M(701, SERIALIZATION_ERROR) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 902b119111b..8cbb8b84217 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -11,6 +11,14 @@ class IDataType; using DataTypePtr = std::shared_ptr; 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(nullptr) +#define NO_TRANSACTION_RAW static_cast(nullptr) +#endif + using CSN = UInt64; using Snapshot = CSN; using LocalTID = UInt64; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 05d4c67514e..35e966f9283 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -141,7 +141,7 @@ void assertEOF(ReadBuffer & buf) void assertNotEOF(ReadBuffer & buf) { if (buf.eof()) - throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); + throw Exception("Attempt to read after EOF", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a043ec24082..a3e790e16fe 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3035,8 +3035,7 @@ void Context::resetZooKeeperMetadataTransaction() void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) const { - int enable_mvcc_test_helper = getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0); - if (enable_mvcc_test_helper == 42) + if (getConfigRef().getInt("allow_experimental_transactions", 0)) return; if (explicit_tcl_query) diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index 7faa74bb820..962784f7671 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -54,7 +54,7 @@ BlockIO InterpreterTransactionControlQuery::executeCommit(ContextMutablePtr sess throw Exception(ErrorCodes::INVALID_TRANSACTION, "Transaction is not in RUNNING state"); TransactionLog::instance().commitTransaction(txn); - session_context->setCurrentTransaction(nullptr); + session_context->setCurrentTransaction(NO_TRANSACTION_PTR); return {}; } @@ -68,7 +68,7 @@ BlockIO InterpreterTransactionControlQuery::executeRollback(ContextMutablePtr se if (txn->getState() == MergeTreeTransaction::RUNNING) TransactionLog::instance().rollbackTransaction(txn); - session_context->setCurrentTransaction(nullptr); + session_context->setCurrentTransaction(NO_TRANSACTION_PTR); return {}; } diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 6225bb9dc0e..2442c16772a 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -54,28 +54,28 @@ void MergeTreeTransaction::addNewPart(const StoragePtr & storage, const DataPart void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataPartPtr & part_to_remove, MergeTreeTransaction * txn) { - TransactionInfoContext context{storage->getStorageID(), part_to_remove->name}; + 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, context); + txn->removeOldPart(storage, part_to_remove, transaction_context); } else { - /// Lock part for removal with special TID, so transactions will no try to remove it concurrently. + /// Lock part for removal with special TID, so transactions will not try to remove it concurrently. /// We lock it only in memory. - part_to_remove->version.lockRemovalTID(Tx::PrehistoricTID, context); + part_to_remove->version.lockRemovalTID(Tx::PrehistoricTID, transaction_context); } } void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage, const DataPartPtr & new_part, const DataPartsVector & covered_parts, MergeTreeTransaction * txn) { TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; - TransactionInfoContext context{storage->getStorageID(), new_part->name}; - tryWriteEventToSystemLog(new_part->version.log, TransactionsInfoLogElement::ADD_PART, tid, context); - context.covering_part = std::move(context.part_name); + 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) @@ -83,16 +83,16 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage txn->addNewPart(storage, new_part); for (const auto & covered : covered_parts) { - context.part_name = covered->name; - txn->removeOldPart(storage, covered, context); + transaction_context.part_name = covered->name; + txn->removeOldPart(storage, covered, transaction_context); } } else { for (const auto & covered : covered_parts) { - context.part_name = covered->name; - covered->version.lockRemovalTID(tid, context); + transaction_context.part_name = covered->name; + covered->version.lockRemovalTID(tid, transaction_context); } } } @@ -233,7 +233,7 @@ bool MergeTreeTransaction::rollback() noexcept /// 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) - const_cast(part->storage).removePartsFromWorkingSet(nullptr, {part}, true); + const_cast(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true); for (const auto & part : parts_to_activate) if (part->version.getCreationTID() != tid) diff --git a/src/Interpreters/MergeTreeTransactionHolder.cpp b/src/Interpreters/MergeTreeTransactionHolder.cpp index 9077dff754b..bf63a471282 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.cpp +++ b/src/Interpreters/MergeTreeTransactionHolder.cpp @@ -28,7 +28,7 @@ MergeTreeTransactionHolder::MergeTreeTransactionHolder(MergeTreeTransactionHolde MergeTreeTransactionHolder & MergeTreeTransactionHolder::operator=(MergeTreeTransactionHolder && rhs) noexcept { onDestroy(); - txn = nullptr; + txn = NO_TRANSACTION_PTR; autocommit = false; owned_by_session_context = nullptr; std::swap(txn, rhs.txn); diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 3315387edc5..7b8a85ea9ab 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -50,7 +50,7 @@ TransactionLog::TransactionLog() global_context = Context::getGlobalContextInstance(); global_context->checkTransactionsAreAllowed(); - zookeeper_path = global_context->getConfigRef().getString("transaction_log.zookeeper_path", "/test/clickhouse/txn"); + zookeeper_path = global_context->getConfigRef().getString("transaction_log.zookeeper_path", "/clickhouse/txn"); zookeeper_path_log = zookeeper_path + "/log"; loadLogFromZooKeeper(); @@ -82,7 +82,7 @@ ZooKeeperPtr TransactionLog::getZooKeeper() const return zookeeper; } -UInt64 TransactionLog::parseCSN(const String & csn_node_name) +UInt64 TransactionLog::deserializeCSN(const String & csn_node_name) { ReadBufferFromString buf{csn_node_name}; assertString("csn-", buf); @@ -92,12 +92,12 @@ UInt64 TransactionLog::parseCSN(const String & csn_node_name) return res; } -String TransactionLog::writeCSN(CSN csn) +String TransactionLog::serializeCSN(CSN csn) { return zkutil::getSequentialNodeName("csn-", csn); } -TransactionID TransactionLog::parseTID(const String & csn_node_content) +TransactionID TransactionLog::deserializeTID(const String & csn_node_content) { TransactionID tid = Tx::EmptyTID; if (csn_node_content.empty()) @@ -109,7 +109,7 @@ TransactionID TransactionLog::parseTID(const String & csn_node_content) return tid; } -String TransactionLog::writeTID(const TransactionID & tid) +String TransactionLog::serializeTID(const TransactionID & tid) { WriteBufferFromOwnString buf; TransactionID::write(tid, buf); @@ -136,8 +136,8 @@ void TransactionLog::loadEntries(Strings::const_iterator beg, Strings::const_ite for (size_t i = 0; i < entries_count; ++i, ++it) { auto res = futures[i].get(); - CSN csn = parseCSN(*it); - TransactionID tid = parseTID(res.data); + CSN csn = deserializeCSN(*it); + TransactionID tid = deserializeTID(res.data); loaded.emplace_back(tid.getHash(), CSNEntry{csn, tid}); LOG_TEST(log, "Got entry {} -> {}", tid, csn); } @@ -175,7 +175,7 @@ void TransactionLog::loadLogFromZooKeeper() assert(code == Coordination::Error::ZNONODE); zookeeper->createAncestors(zookeeper_path_log); Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/tail_ptr", writeCSN(Tx::MaxReservedCSN), zkutil::CreateMode::Persistent)); + 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 @@ -196,10 +196,10 @@ void TransactionLog::loadLogFromZooKeeper() std::sort(entries_list.begin(), entries_list.end()); loadEntries(entries_list.begin(), entries_list.end()); assert(!last_loaded_entry.empty()); - assert(latest_snapshot == parseCSN(last_loaded_entry)); + assert(latest_snapshot == deserializeCSN(last_loaded_entry)); local_tid_counter = Tx::MaxReservedLocalTID; - tail_ptr = parseCSN(zookeeper->get(zookeeper_path + "/tail_ptr")); + tail_ptr = deserializeCSN(zookeeper->get(zookeeper_path + "/tail_ptr")); } void TransactionLog::runUpdatingThread() @@ -251,7 +251,7 @@ void TransactionLog::loadNewEntries() 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 == parseCSN(last_loaded_entry)); + assert(latest_snapshot == deserializeCSN(last_loaded_entry)); latest_snapshot.notify_all(); } @@ -271,14 +271,14 @@ void TransactionLog::removeOldEntries() /// TODO we will need a bit more complex logic for multiple hosts Coordination::Stat stat; - CSN old_tail_ptr = parseCSN(zookeeper->get(zookeeper_path + "/tail_ptr", &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); /// (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", writeCSN(new_tail_ptr), stat.version); + 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 @@ -302,7 +302,7 @@ void TransactionLog::removeOldEntries() continue; LOG_TEST(log, "Removing entry {} -> {}", elem.second.tid, elem.second.csn); - auto code = zookeeper->tryRemove(zookeeper_path_log + "/" + writeCSN(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); } @@ -356,12 +356,12 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn) /// TODO handle connection loss /// TODO support batching auto current_zookeeper = getZooKeeper(); - String path_created = current_zookeeper->create(zookeeper_path_log + "/csn-", writeTID(txn->tid), zkutil::CreateMode::PersistentSequential); /// Commit point + 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 = parseCSN(path_created.substr(zookeeper_path_log.size() + 1)); + 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); @@ -419,7 +419,7 @@ MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & std::lock_guard lock{running_list_mutex}; auto it = running_list.find(tid); if (it == running_list.end()) - return nullptr; + return NO_TRANSACTION_PTR; return it->second; } diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 67d8e709900..af741f7e76e 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -130,10 +130,10 @@ private: void loadNewEntries(); void removeOldEntries(); - static UInt64 parseCSN(const String & csn_node_name); - static String writeCSN(CSN csn); - static TransactionID parseTID(const String & csn_node_content); - static String writeTID(const TransactionID & tid); + 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; diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index 08af2e15fad..a397a4983b3 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -10,10 +10,15 @@ 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)) {} diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index a78d9050c94..0dcccc33266 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -603,7 +603,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}), - nullptr); + NO_TRANSACTION_PTR); part_out.write(block); part_out.finalizePart(new_projection_part, false); @@ -627,7 +627,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( MergedBlockOutputStream part_out( new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}), nullptr); + CompressionCodecFactory::instance().get("NONE", {}), NO_TRANSACTION_PTR); part_out.write(block); part_out.finalizePart(new_data_part, false); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d61a1d9f8fc..1f466214a71 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1185,7 +1185,10 @@ void IMergeTreeDataPart::appendRemovalTIDToVersionMetadata(bool clear) const return; } - LOG_TEST(storage.log, "Appending removal TID for {} (creation: {}, removal {})", name, version.creation_tid, version.removal_tid); + 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(); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 3e14b446381..25c451dc661 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -189,7 +189,7 @@ std::pair MergeFromLogEntryT future_merged_part, settings); - transaction_ptr = std::make_unique(storage, nullptr); + transaction_ptr = std::make_unique(storage, NO_TRANSACTION_RAW); stopwatch_ptr = std::make_unique(); merge_task = storage.merger_mutator.mergePartsToTemporaryPart( @@ -204,7 +204,7 @@ std::pair MergeFromLogEntryT entry.deduplicate, entry.deduplicate_by_columns, storage.merging_params, - nullptr); + NO_TRANSACTION_PTR); /// Adjust priority @@ -227,7 +227,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite /// Task is not needed merge_task.reset(); - storage.merger_mutator.renameMergedTemporaryPart(part, parts, nullptr, transaction_ptr.get()); + storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, transaction_ptr.get()); try { diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index e84102857f7..c420cbca12b 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -41,7 +41,7 @@ struct MergeMutateSelectedEntry MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, - MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = nullptr) + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 508f510ea26..f59f6200503 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -588,7 +588,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c projection_merging_params, global_ctx->new_data_part.get(), ".proj", - nullptr, + NO_TRANSACTION_PTR, global_ctx->data, global_ctx->mutator, global_ctx->merges_blocker, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e76c2888315..9f9af97b0ae 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3897,7 +3897,7 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac disk->removeFileIfExists(fs::path(temp_part_dir) / IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); part->loadColumnsChecksumsIndexes(false, true); - renameTempPartAndAdd(part, nullptr, increment); + renameTempPartAndAdd(part, NO_TRANSACTION_RAW, increment); }; restore_tasks.emplace_back(std::move(restore_task)); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 257e99cab17..a66f040b3ab 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -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, nullptr); + projection_compression_codec, NO_TRANSACTION_PTR); projection_out.write(projection_part->block); projection_out.finalizePart(projection_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2f29ff39ef4..75162cfd668 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -547,7 +547,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( columns, MergeTreeIndices{}, compression_codec, - nullptr); + NO_TRANSACTION_PTR); out->writeWithPermutation(block, perm_ptr); auto finalizer = out->finalizePartAsync(new_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 21b68cf33cf..6c29714a474 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -59,15 +59,15 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP auto out = disk->writeFile(path_prefix + file_name); *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; + *out << "commands: "; + commands.writeText(*out); + *out << "\n"; if (!tid.isPrehistoric()) { *out << "tid: "; TransactionID::write(tid, *out); *out << "\n"; } - *out << "commands: "; - commands.writeText(*out); - *out << "\n"; out->sync(); } catch (...) @@ -116,18 +116,17 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); - assertNotEOF(*buf); - if (*(buf->position()) == 't') + *buf >> "commands: "; + commands.readText(*buf); + *buf >> "\n"; + + if (!buf->eof()) { *buf >> "tid: "; tid = TransactionID::read(*buf); *buf >> "\n"; } - *buf >> "commands: "; - commands.readText(*buf); - *buf >> "\n"; - assertEOF(*buf); } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f6931def490..3ba11c6661e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -61,7 +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(UInt64, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 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) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index bc3e39e0f86..bab0947a8ff 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -198,7 +198,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}), - nullptr); + NO_TRANSACTION_PTR); part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 7089d3d4c61..888ebc0b809 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -80,7 +80,7 @@ std::pair MutateFromLogEntry RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = storage.getInMemoryMetadataPtr(); - transaction_ptr = std::make_unique(storage, nullptr); + transaction_ptr = std::make_unique(storage, NO_TRANSACTION_RAW); future_mutated_part = std::make_shared(); future_mutated_part->name = entry.new_part_name; @@ -126,7 +126,7 @@ std::pair MutateFromLogEntry mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, merge_mutate_entry.get(), - entry.create_time, fake_query_context, nullptr, reserved_space, table_lock_holder); + entry.create_time, fake_query_context, NO_TRANSACTION_PTR, reserved_space, table_lock_holder); /// Adjust priority for (auto & item : future_mutated_part->parts) @@ -145,7 +145,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit { new_part = mutate_task->getFuture().get(); - storage.renameTempPartAndReplace(new_part, nullptr, nullptr, transaction_ptr.get()); + storage.renameTempPartAndReplace(new_part, NO_TRANSACTION_RAW, nullptr, transaction_ptr.get()); try { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b6f8c3e62f7..cd3d92471a5 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -643,7 +643,7 @@ public: false, // TODO Do we need deduplicate for projections {}, projection_merging_params, - nullptr, + NO_TRANSACTION_PTR, ctx->new_data_part.get(), ".tmp_proj"); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index e1641894392..a5968a4ab54 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -444,12 +444,12 @@ void ReplicatedMergeTreeSink::commitPart( /// Information about the part. storage.getCommitPartOps(ops, part, block_id_path); - MergeTreeData::Transaction transaction(storage, nullptr); /// If you can not add a part to ZK, we'll remove it back from the working set. + MergeTreeData::Transaction transaction(storage, NO_TRANSACTION_RAW); /// If you can not add a part to ZK, we'll remove it back from the working set. bool renamed = false; try { - renamed = storage.renameTempPartAndAdd(part, nullptr, nullptr, &transaction); + renamed = storage.renameTempPartAndAdd(part, NO_TRANSACTION_RAW, nullptr, &transaction); } catch (const Exception & e) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 70b88012a65..bf310a433bf 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -773,7 +773,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( { if (tx) { - /// Cannot merge parts if some of them is not visible in current snapshot + /// Cannot merge parts if some of them are not visible in current snapshot /// TODO Transactions: We can use simplified visibility rules (without CSN lookup) here if (left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 332abd83095..9db452c7974 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1503,10 +1503,10 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { LOG_TRACE(log, "Found valid local part for {}, preparing the transaction", part->name); - Transaction transaction(*this, nullptr); + Transaction transaction(*this, NO_TRANSACTION_RAW); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - renameTempPartAndReplace(part, nullptr, nullptr, &transaction); + renameTempPartAndReplace(part, NO_TRANSACTION_RAW, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, @@ -1796,7 +1796,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) DataPartsVector parts_to_remove; { auto data_parts_lock = lockParts(); - parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range_info, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range_info, true, data_parts_lock); if (parts_to_remove.empty()) { if (!drop_range_info.isFakeDropRangePart()) @@ -1929,7 +1929,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (parts_to_add.empty() && replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -2109,7 +2109,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) throw Exception("Checksums of " + part_desc->src_table_part->name + " is suddenly changed", ErrorCodes::UNFINISHED); part_desc->res_part = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, nullptr); + part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, NO_TRANSACTION_PTR); } else if (!part_desc->replica.empty()) { @@ -2149,12 +2149,12 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { /// Commit parts auto zookeeper = getZooKeeper(); - Transaction transaction(*this, nullptr); + Transaction transaction(*this, NO_TRANSACTION_RAW); Coordination::Requests ops; for (PartDescriptionPtr & part_desc : final_parts) { - renameTempPartAndReplace(part_desc->res_part, nullptr, nullptr, &transaction); + renameTempPartAndReplace(part_desc->res_part, NO_TRANSACTION_RAW, nullptr, &transaction); getCommitPartOps(ops, part_desc->res_part); } @@ -2167,7 +2167,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) transaction.commit(&data_parts_lock); if (replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -3030,7 +3030,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() future_merged_part->uuid = UUIDHelpers::generateV4(); if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr, nullptr) == SelectPartsDecision::SELECTED) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts( zookeeper, @@ -3884,8 +3884,8 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora if (!to_detached) { - Transaction transaction(*this, nullptr); - renameTempPartAndReplace(part, nullptr, nullptr, &transaction); + Transaction transaction(*this, NO_TRANSACTION_RAW); + renameTempPartAndReplace(part, NO_TRANSACTION_RAW, nullptr, &transaction); replaced_parts = checkPartChecksumsAndCommit(transaction, part); @@ -4397,12 +4397,12 @@ bool StorageReplicatedMergeTree::optimize( { select_decision = merger_mutator.selectPartsToMerge( future_merged_part, /* aggressive */ true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, - can_merge, /* merge_with_ttl_allowed */ false, nullptr, &disable_reason); + can_merge, /* merge_with_ttl_allowed */ false, NO_TRANSACTION_PTR, &disable_reason); } else { select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, can_merge, partition_id, final, metadata_snapshot, nullptr, + future_merged_part, can_merge, partition_id, final, metadata_snapshot, NO_TRANSACTION_PTR, &disable_reason, query_context->getSettingsRef().optimize_skip_merged_partitions); } @@ -6291,7 +6291,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, nullptr); + auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, NO_TRANSACTION_PTR); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -6374,7 +6374,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto data_parts_lock = lockParts(); transaction.commit(&data_parts_lock); if (replace) - parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); } PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); @@ -6502,7 +6502,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, nullptr); + auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, NO_TRANSACTION_PTR); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -6581,7 +6581,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta else zkutil::KeeperMultiException::check(code, ops, op_results); - parts_to_remove = removePartsInRangeFromWorkingSet(nullptr, drop_range, true, lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, lock); transaction.commit(&lock); } @@ -7630,8 +7630,8 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP try { - MergeTreeData::Transaction transaction(*this, nullptr); - auto replaced_parts = renameTempPartAndReplace(new_data_part, nullptr, nullptr, &transaction); + MergeTreeData::Transaction transaction(*this, NO_TRANSACTION_RAW); + auto replaced_parts = renameTempPartAndReplace(new_data_part, NO_TRANSACTION_RAW, nullptr, &transaction); if (!replaced_parts.empty()) { diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 2fe9b307e7b..da6657f64dc 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -164,7 +164,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b if (has_zookeeper) attach(context, system_database, "zookeeper"); - if (context->getConfigRef().getInt("_enable_experimental_mvcc_prototype_test_helper_dev", 0) == 42) + if (context->getConfigRef().getInt("allow_experimental_transactions", 0)) attach(context, system_database, "transactions"); } diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index 1087eb83f13..19810986ea1 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,5 +1,5 @@ - <_enable_experimental_mvcc_prototype_test_helper_dev>42 + 42 diff --git a/tests/integration/test_transactions/configs/transactions.xml b/tests/integration/test_transactions/configs/transactions.xml index f48105fc98c..a8d3e8fbf6d 100644 --- a/tests/integration/test_transactions/configs/transactions.xml +++ b/tests/integration/test_transactions/configs/transactions.xml @@ -1,5 +1,5 @@ - <_enable_experimental_mvcc_prototype_test_helper_dev>42 + 42 100500 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 5d91557dc6d..47ab89e5646 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -73,7 +73,7 @@ tx 7 "commit" tx_async 11 "begin transaction" tx_async 11 "select 9, n, _part from mt order by n" tx_async 12 "begin transaction" -tx_async 11 "alter table mt update n=n+1 where 1" >/dev/null & +tx_async 11 "alter table mt update n=n+1 where 1" >/dev/null tx_async 12 "alter table mt update n=n+1 where 1" >/dev/null tx_async 11 "commit" >/dev/null tx_async 12 "commit" >/dev/null diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index ab582aeae37..99ae9741aab 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -26,8 +26,8 @@ function thread_insert() fi $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO src VALUES ($val, 1); - INSERT INTO src VALUES ($val, 2); + INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); + INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); COMMIT;" val=$((val+1)) sleep 0.$RANDOM; @@ -47,7 +47,7 @@ function thread_partition_src_to_dst() out=$( $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO src VALUES ($i, 3); + INSERT INTO src VALUES /* ($i, 3) */ ($i, 3); INSERT INTO dst SELECT * FROM src; ALTER TABLE src DROP PARTITION ID 'all'; SET throw_on_unsupported_query_inside_transaction=0; @@ -72,7 +72,7 @@ function thread_partition_dst_to_src() SYSTEM STOP MERGES dst; ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick BEGIN TRANSACTION; - INSERT INTO dst VALUES ($i, 4); + INSERT INTO dst VALUES /* ($i, 4) */ ($i, 4); INSERT INTO src SELECT * FROM dst; ALTER TABLE dst DROP PARTITION ID 'all'; SET throw_on_unsupported_query_inside_transaction=0; diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 672a49df5fc..755c47011e5 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -26,9 +26,9 @@ function thread_insert_commit() for i in {1..100}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO src VALUES ($i, $1); + INSERT INTO src VALUES /* ($i, $1) */ ($i, $1); SELECT throwIf((SELECT sum(nm) FROM mv) != $(($i * $1))) FORMAT Null; - INSERT INTO src VALUES (-$i, $1); + INSERT INTO src VALUES /* (-$i, $1) */ (-$i, $1); COMMIT;" 2>&1| grep -Fv "is violated at row" | grep -Fv "Transaction is not in RUNNING state" | grep -F "Received from " ||: done } @@ -39,7 +39,7 @@ function thread_insert_rollback() for _ in {1..100}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO src VALUES (42, $1); + INSERT INTO src VALUES /* (42, $1) */ (42, $1); SELECT throwIf((SELECT count() FROM src WHERE n=42 AND m=$1) != 1) FORMAT Null; ROLLBACK;" done @@ -89,7 +89,7 @@ function thread_select() SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; - COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT 'src', arraySort(groupArray(n*m)) FROM src UNION ALL SELECT 'mv', arraySort(groupArray(nm)) FROM mv" + COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT 'src', arraySort(groupArray(n*m)), arraySort(groupArray((n*m, _part))) FROM src UNION ALL SELECT 'mv', arraySort(groupArray(nm)), arraySort(groupArray((nm, _part))) FROM mv" done } diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index ec1a989d42f..8872ab82c03 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -17,8 +17,8 @@ function thread_insert_commit() for i in {1..50}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO mt VALUES ($i, $1); - INSERT INTO mt VALUES (-$i, $1); + INSERT INTO mt VALUES /* ($i, $1) */ ($i, $1); + INSERT INTO mt VALUES /* (-$i, $1) */ (-$i, $1); COMMIT;"; done } @@ -28,7 +28,7 @@ function thread_insert_rollback() for _ in {1..50}; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; - INSERT INTO mt VALUES (42, $1); + INSERT INTO mt VALUES /* (42, $1) */ (42, $1); ROLLBACK;"; done } From 00c39a46d520e39e38f8d8422a877ef31645cfa9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Mar 2022 21:30:24 +0100 Subject: [PATCH 069/372] fix --- .../01173_transaction_control_queries.sql | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index 6877a11f1f1..3e08268903a 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -4,8 +4,8 @@ drop table if exists mt2; create table mt1 (n Int64) engine=MergeTree order by n; create table mt2 (n Int64) engine=MergeTree order by n; -commit; -- { serverError 700 } -rollback; -- { serverError 700 } +commit; -- { serverError 645 } +rollback; -- { serverError 645 } begin transaction; insert into mt1 values (1); @@ -21,7 +21,7 @@ rollback; begin transaction; select 'no nested', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); -begin transaction; -- { serverError 700 } +begin transaction; -- { serverError 645 } rollback; begin transaction; @@ -31,8 +31,8 @@ select 'on exception before start', arraySort(groupArray(n)) from (select n from -- rollback on exception before start select functionThatDoesNotExist(); -- { serverError 46 } -- cannot commit after exception -commit; -- { serverError 700 } -begin transaction; -- { serverError 700 } +commit; -- { serverError 645 } +begin transaction; -- { serverError 645 } rollback; begin transaction; @@ -42,10 +42,10 @@ select 'on exception while processing', arraySort(groupArray(n)) from (select n -- rollback on exception while processing select throwIf(100 < number) from numbers(1000); -- { serverError 395 } -- cannot commit after exception -commit; -- { serverError 700 } -insert into mt1 values (5); -- { serverError 700 } -insert into mt2 values (50); -- { serverError 700 } -select 1; -- { serverError 700 } +commit; -- { serverError 645 } +insert into mt1 values (5); -- { serverError 645 } +insert into mt2 values (50); -- { serverError 645 } +select 1; -- { serverError 645 } rollback; begin transaction; @@ -54,8 +54,8 @@ insert into mt2 values (60); select 'on session close', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); -- trigger reconnection by error on client, check rollback on session close insert into mt1 values ([1]); -- { clientError 43 } -commit; -- { serverError 700 } -rollback; -- { serverError 700 } +commit; -- { serverError 645 } +rollback; -- { serverError 645 } begin transaction; insert into mt1 values (7); @@ -69,19 +69,19 @@ commit; begin transaction; create table m (n int) engine=Memory; -- { serverError 48 } -commit; -- { serverError 700 } +commit; -- { serverError 645 } rollback; create table m (n int) engine=Memory; begin transaction; insert into m values (1); -- { serverError 48 } -select * from m; -- { serverError 700 } -commit; -- { serverError 700 } +select * from m; -- { serverError 645 } +commit; -- { serverError 645 } rollback; begin transaction; select * from m; -- { serverError 48 } -commit; -- { serverError 700 } +commit; -- { serverError 645 } rollback; drop table m; From 7bb66e6702ae1d12b0a4cd58b033ff830e9631a6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 17 Mar 2022 01:51:35 -0400 Subject: [PATCH 070/372] added INTERPOLATE extension for ORDER BY WITH FILL --- src/Core/InterpolateDescription.cpp | 61 ++++++++++++++ src/Core/InterpolateDescription.h | 79 +++++++++++++++++++ src/Interpreters/FillingRow.cpp | 57 +++++++------ src/Interpreters/FillingRow.h | 27 ++++++- src/Interpreters/InterpreterSelectQuery.cpp | 21 ++++- src/Parsers/ASTInterpolateElement.cpp | 15 ++++ src/Parsers/ASTInterpolateElement.h | 29 +++++++ src/Parsers/ASTOrderByElement.h | 1 + src/Parsers/ASTSelectQuery.h | 8 +- src/Parsers/ExpressionElementParsers.cpp | 27 +++++++ src/Parsers/ExpressionElementParsers.h | 9 +++ src/Parsers/ExpressionListParsers.cpp | 7 ++ src/Parsers/ExpressionListParsers.h | 6 ++ src/Parsers/ParserSelectQuery.cpp | 20 +++++ src/Processors/QueryPlan/FillingStep.cpp | 11 ++- src/Processors/QueryPlan/FillingStep.h | 4 +- .../Transforms/FillingTransform.cpp | 56 ++++++++++--- src/Processors/Transforms/FillingTransform.h | 8 +- 18 files changed, 398 insertions(+), 48 deletions(-) create mode 100644 src/Core/InterpolateDescription.cpp create mode 100644 src/Core/InterpolateDescription.h create mode 100644 src/Parsers/ASTInterpolateElement.cpp create mode 100644 src/Parsers/ASTInterpolateElement.h diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp new file mode 100644 index 00000000000..1f5c343fddd --- /dev/null +++ b/src/Core/InterpolateDescription.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include + +namespace DB +{ + +void dumpInterpolateDescription(const InterpolateDescription & description, const Block & /*header*/, WriteBuffer & out) +{ + bool first = true; + + for (const auto & desc : description) + { + if (!first) + out << ", "; + first = false; + + if (desc.column.name.empty()) + out << "?"; + else + out << desc.column.name; + } +} + +void InterpolateColumnDescription::interpolate(Field & field) const +{ + if(field.isNull()) + return; + Block expr_columns; + expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); + actions->execute(expr_columns); + expr_columns.getByPosition(0).column->get(0, field); +} + +void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const +{ + map.add("Column", column.name); +} + +std::string dumpInterpolateDescription(const InterpolateDescription & description) +{ + WriteBufferFromOwnString wb; + dumpInterpolateDescription(description, Block{}, wb); + return wb.str(); +} + +JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header) +{ + auto json_array = std::make_unique(); + for (const auto & descr : description) + { + auto json_map = std::make_unique(); + descr.explain(*json_map, header); + json_array->add(std::move(json_map)); + } + + return json_array; +} + +} diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h new file mode 100644 index 00000000000..7cd9f86890e --- /dev/null +++ b/src/Core/InterpolateDescription.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +class Collator; + +namespace DB +{ + +namespace JSONBuilder +{ + class JSONMap; + class IItem; + using ItemPtr = std::unique_ptr; +} + +class Block; + + +/// Interpolate description +struct InterpolateColumnDescription +{ + using Signature = ExecutableFunctionExpression::Signature; + + ColumnWithTypeAndName column; + ExpressionActionsPtr actions; + // ExecutableFunctionExpression interpolate_func; + // std::string column_name; /// The name of the column. + // size_t column_number; /// Column number (used if no name is given). + + + + explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : + column(column_), actions(actions_) {} + + // explicit InterpolateColumnDescription(size_t column_number_, const ASTInterpolateElement & /*ast*/) : column_number(column_number_) {} + + // explicit InterpolateColumnDescription(const std::string & column_name_, const ASTInterpolateElement & /*ast*/) : column_name(column_name_), column_number(0) {} + + bool operator == (const InterpolateColumnDescription & other) const + { + return column == other.column;// && column_number == other.column_number; + } + + bool operator != (const InterpolateColumnDescription & other) const + { + return !(*this == other); + } + + void interpolate(Field & field) const; + + std::string dump() const + { + return fmt::format("{}", column.name); + } + + void explain(JSONBuilder::JSONMap & map, const Block & header) const; +}; + +/// Description of interpolation for several columns. +using InterpolateDescription = std::vector; + +/// Outputs user-readable description into `out`. +void dumpInterpolateDescription(const InterpolateDescription & description, const Block & header, WriteBuffer & out); + +std::string dumpInterpolateDescription(const InterpolateDescription & description); + +JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header); + +} diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 94f185a44cc..8cc918d2c72 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,26 +19,30 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description) : description(sort_description) +FillingRow::FillingRow(const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_) + : sort{*this} + , interpolate{*this} + , sort_description(sort_description_) + , interpolate_description(interpolate_description_) { - row.resize(description.size()); + row.resize(sort_description.size() + interpolate_description.size()); } bool FillingRow::operator<(const FillingRow & other) const { - for (size_t i = 0; i < size(); ++i) + for (size_t i = 0; i < sort.size(); ++i) { - if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i])) + if (sort[i].isNull() || other.sort[i].isNull() || equals(sort[i], other.sort[i])) continue; - return less(row[i], other[i], getDirection(i)); + return less(sort[i], other.sort[i], getDirection(i)); } return false; } bool FillingRow::operator==(const FillingRow & other) const { - for (size_t i = 0; i < size(); ++i) - if (!equals(row[i], other[i])) + for (size_t i = 0; i < sort.size(); ++i) + if (!equals(sort[i], other.sort[i])) return false; return true; } @@ -47,49 +51,54 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; + for(size_t i = 0; i < to_row.interpolate.size(); ++i) { + std::cout << to_row.interpolate[i] <<" : "; + interpolate[i] = to_row.interpolate[i]; + } + /// Find position we need to increment for generating next row. - for (; pos < row.size(); ++pos) - if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) + for (; pos < sort.size(); ++pos) + if (!sort[pos].isNull() && !to_row.sort[pos].isNull() && !equals(sort[pos], to_row.sort[pos])) break; - if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) + if (pos == sort.size() || less(to_row.sort[pos], sort[pos], getDirection(pos))) return false; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. - for (size_t i = row.size() - 1; i > pos; --i) + for (size_t i = sort.size() - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + if (getFillDescription(i).fill_to.isNull() || sort[i].isNull()) continue; - auto next_value = row[i]; + auto next_value = sort[i]; getFillDescription(i).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { - row[i] = next_value; + sort[i] = next_value; initFromDefaults(i + 1); return true; } } - auto next_value = row[pos]; + auto next_value = sort[pos]; getFillDescription(pos).step_func(next_value); - if (less(to_row[pos], next_value, getDirection(pos))) + if (less(to_row.sort[pos], next_value, getDirection(pos))) return false; - row[pos] = next_value; - if (equals(row[pos], to_row[pos])) + sort[pos] = next_value; + if (equals(sort[pos], to_row.sort[pos])) { bool is_less = false; - for (size_t i = pos + 1; i < size(); ++i) + for (size_t i = pos + 1; i < sort.size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) - row[i] = fill_from; + sort[i] = fill_from; else - row[i] = to_row[i]; - is_less |= less(row[i], to_row[i], getDirection(i)); + sort[i] = to_row.sort[i]; + is_less |= less(sort[i], to_row.sort[i], getDirection(i)); } return is_less; @@ -101,8 +110,8 @@ bool FillingRow::next(const FillingRow & to_row) void FillingRow::initFromDefaults(size_t from_pos) { - for (size_t i = from_pos; i < row.size(); ++i) - row[i] = getFillDescription(i).fill_from; + for (size_t i = from_pos; i < sort.size(); ++i) + sort[i] = getFillDescription(i).fill_from; } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 604f4b1ee74..6081effe703 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include @@ -17,7 +18,23 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - FillingRow(const SortDescription & sort_description); + struct { + FillingRow & filling_row; + + Field & operator[](size_t index) { return filling_row.row[index]; } + const Field & operator[](size_t index) const { return filling_row.row[index]; } + size_t size() const { return filling_row.sort_description.size(); } + } sort; + + struct { + FillingRow & filling_row; + + Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } + const Field & operator[](size_t index) const { return filling_row.row[filling_row.sort_description.size() + index]; } + size_t size() const { return filling_row.interpolate_description.size(); } + } interpolate; +public: + FillingRow(const SortDescription & sort_description, const InterpolateDescription & interpolate_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -30,12 +47,14 @@ public: bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; - int getDirection(size_t index) const { return description[index].direction; } - FillColumnDescription & getFillDescription(size_t index) { return description[index].fill_description; } + int getDirection(size_t index) const { return sort_description[index].direction; } + FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } + InterpolateColumnDescription & getInterpolateDescription(size_t index) { return interpolate_description[index]; } private: Row row; - SortDescription description; + SortDescription sort_description; + InterpolateDescription interpolate_description; }; void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f2fc17fbf9a..6a4456b1caf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -827,6 +828,23 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } +static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +{ + InterpolateDescription interpolate_descr; + interpolate_descr.reserve(query.interpolate()->children.size()); + + for (const auto & elem : query.interpolate()->children) + { + auto interpolate = elem->as(); + auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + interpolate_descr.emplace_back(block.findByName(interpolate.column->getColumnName())->cloneEmpty(), actions); + } + + return interpolate_descr; +} + static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) { SortDescription order_descr; @@ -2498,7 +2516,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr)); + InterpolateDescription interpolate_descr = getInterpolateDescription(query, source_header, context); + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), std::move(interpolate_descr)); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp new file mode 100644 index 00000000000..14aa5f7e672 --- /dev/null +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -0,0 +1,15 @@ +#include +#include +#include +#include + + +namespace DB +{ + +void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const +{ + +} + +} diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h new file mode 100644 index 00000000000..e23b10cd434 --- /dev/null +++ b/src/Parsers/ASTInterpolateElement.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class ASTInterpolateElement : public IAST +{ +public: + ASTPtr column; + ASTPtr expr; + + String getID(char) const override { return "InterpolateElement"; } + + ASTPtr clone() const override + { + auto clone = std::make_shared(*this); + clone->cloneChildren(); + return clone; + } + + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 8c868312834..4d07405c17a 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -37,4 +37,5 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; + } diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 1c631783fdb..4a30d6afee3 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -32,7 +32,8 @@ public: LIMIT_BY, LIMIT_OFFSET, LIMIT_LENGTH, - SETTINGS + SETTINGS, + INTERPOLATE }; static String expressionToString(Expression expr) @@ -69,6 +70,8 @@ public: return "LIMIT LENGTH"; case Expression::SETTINGS: return "SETTINGS"; + case Expression::INTERPOLATE: + return "INTERPOLATE"; } return ""; } @@ -98,7 +101,7 @@ public: const ASTPtr where() const { return getExpression(Expression::WHERE); } const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } const ASTPtr having() const { return getExpression(Expression::HAVING); } - const ASTPtr window() const { return getExpression(Expression::WINDOW); } + const ASTPtr window() const { return getExpression(Expression::WINDOW); } const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } const ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); } const ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); } @@ -106,6 +109,7 @@ public: const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); } const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); } const ASTPtr settings() const { return getExpression(Expression::SETTINGS); } + const ASTPtr interpolate() const { return getExpression(Expression::INTERPOLATE); } bool hasFiltration() const { return where() || prewhere() || having(); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index c51201750c5..b0cd2f5e4db 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -2314,6 +2315,32 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect return true; } +bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword as("AS"); + ParserExpression element_p; + ParserIdentifier ident_p; + + ASTPtr ident; + if (!ident_p.parse(pos, ident, expected)) + return false; + + if (!as.ignore(pos, expected)) + return false; + + ASTPtr expr; + if (!element_p.parse(pos, expr, expected)) + return false; + + auto elem = std::make_shared(); + elem->column = ident; + elem->expr = expr; + + node = elem; + + return true; +} + bool ParserFunctionWithKeyValueArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index c86721dca18..10b4f5fd7d1 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -420,6 +420,15 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/** Element of INTERPOLATE expression + */ +class ParserInterpolateElement : public IParserBase +{ +protected: + const char * getName() const override { return "element of INTERPOLATE expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + /** Parser for function with arguments like KEY VALUE (space separated) * no commas allowed, just space-separated pairs. */ diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 13af308736b..1396e86df39 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -757,6 +757,13 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & } +bool ParserInterpolateExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} + + bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 358fe778f91..d81aa8de380 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -517,6 +517,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserInterpolateExpressionList : public IParserBase +{ +protected: + const char * getName() const override { return "interpolate expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /// Parser for key-value pair, where value can be list of pairs. class ParserKeyValuePair : public IParserBase diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 90ab5911d6b..85f08ad11b0 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -59,12 +60,14 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_rows("ROWS"); ParserKeyword s_first("FIRST"); ParserKeyword s_next("NEXT"); + ParserKeyword s_interpolate("INTERPOLATE"); ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false); ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; + ParserInterpolateExpressionList interpolate_list; ParserToken open_bracket(TokenType::OpeningRoundBracket); ParserToken close_bracket(TokenType::ClosingRoundBracket); @@ -78,6 +81,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr having_expression; ASTPtr window_list; ASTPtr order_expression_list; + ASTPtr interpolate_expression_list; ASTPtr limit_by_length; ASTPtr limit_by_offset; ASTPtr limit_by_expression_list; @@ -239,6 +243,21 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!order_list.parse(pos, order_expression_list, expected)) return false; + + /// if any WITH FILL parse possible INTERPOLATE list + if ( std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), + [](auto & child) { return child->template as()->with_fill; }) ) + { + if (s_interpolate.ignore(pos, expected)) + { + if (!open_bracket.ignore(pos, expected)) + return false; + if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) + return false; + if (!close_bracket.ignore(pos, expected)) + return false; + } + } } /// This is needed for TOP expression, because it can also use WITH TIES. @@ -430,6 +449,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, std::move(limit_offset)); select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(limit_length)); select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings)); + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, std::move(interpolate_expression_list)); return true; } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 223892aa528..47540bb2725 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -28,9 +28,9 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_) - : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) - , sort_description(std::move(sort_description_)) +FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_) + : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_/*, interpolate_description_*/), getTraits()) + , sort_description(std::move(sort_description_)), interpolate_description(std::move(interpolate_description_)) { if (!input_stream_.has_single_port) throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR); @@ -41,7 +41,7 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; - return std::make_shared(header, sort_description, on_totals); + return std::make_shared(header, sort_description, interpolate_description, on_totals); }); } @@ -50,11 +50,14 @@ void FillingStep::describeActions(FormatSettings & settings) const settings.out << String(settings.offset, ' '); dumpSortDescription(sort_description, input_streams.front().header, settings.out); settings.out << '\n'; + dumpInterpolateDescription(interpolate_description, input_streams.front().header, settings.out); + settings.out << '\n'; } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); + map.add("Interpolate Description", explainInterpolateDescription(interpolate_description, input_streams.front().header)); } } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 6a5bca1890d..252f1ba3fd5 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -9,7 +10,7 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_); + FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_); String getName() const override { return "Filling"; } @@ -22,6 +23,7 @@ public: private: SortDescription sort_description; + InterpolateDescription interpolate_description; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 1276157cc91..6113b4479a8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } -Block FillingTransform::transformHeader(Block header, const SortDescription & sort_description) +Block FillingTransform::transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description*/) { NameSet sort_keys; for (const auto & key : sort_description) @@ -140,12 +140,13 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) + const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_) + : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) , sort_description(sort_description_) + , interpolate_description(interpolate_description_) , on_totals(on_totals_) - , filling_row(sort_description_) - , next_row(sort_description_) + , filling_row(sort_description_, interpolate_description_) + , next_row(sort_description_, interpolate_description_) { if (on_totals) return; @@ -173,6 +174,29 @@ FillingTransform::FillingTransform( } } + for (size_t i = 0, size = interpolate_description.size(); i < size; ++i) + { + size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); + is_fill_column[block_position] = true; + fill_column_positions.push_back(block_position); +/* TODO JOO check types? + auto & descr = filling_row.getFillDescription(i); + const auto & type = header_.getByPosition(block_position).type; + + if (!tryConvertFields(descr, type)) + throw Exception("Incompatible types of WITH FILL expression values with column type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (type->isValueRepresentedByUnsignedInteger() && + ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || + (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) + { + throw Exception("WITH FILL bound values cannot be negative for unsigned type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } +*/ + } + std::set unique_positions; for (auto pos : fill_column_positions) if (!unique_positions.insert(pos).second) @@ -189,8 +213,8 @@ IProcessor::Status FillingTransform::prepare() { should_insert_first = next_row < filling_row || first; - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - next_row[i] = filling_row.getFillDescription(i).fill_to; + for (size_t i = 0, size = filling_row.sort.size(); i < size; ++i) + next_row.sort[i] = filling_row.getFillDescription(i).fill_to; if (first || filling_row < next_row) { @@ -251,7 +275,7 @@ void FillingTransform::transform(Chunk & chunk) if (first) { - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.sort.size(); ++i) { auto current_value = (*old_fill_columns[i])[0]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; @@ -272,7 +296,7 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.sort.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -288,9 +312,23 @@ void FillingTransform::transform(Chunk & chunk) if (should_insert_first && filling_row < next_row) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + /// Update interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); + /// Insert generated filling row to block, while it is less than current row in block. while (filling_row.next(next_row)) + { + /// Update interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + } + + /// Reset interpolate fields + for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) + next_row[i] = (*old_fill_columns[i])[row_ind]; copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 7ccebadfb6d..1a8a6c4919c 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include namespace DB @@ -13,13 +14,13 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, bool on_totals_); + FillingTransform(const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_); String getName() const override { return "FillingTransform"; } Status prepare() override; - static Block transformHeader(Block header, const SortDescription & sort_description); + static Block transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description_*/); protected: void transform(Chunk & Chunk) override; @@ -27,7 +28,8 @@ protected: private: void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; - const SortDescription sort_description; /// Contains only rows with WITH FILL. + const SortDescription sort_description; /// Contains only columns with WITH FILL. + const InterpolateDescription interpolate_description; /// Contains INTERPOLATE columns const bool on_totals; /// FillingTransform does nothing on totals. FillingRow filling_row; /// Current row, which is used to fill gaps. From 00700988ee40928fc25d589c4557fdca1fd41924 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 17 Mar 2022 02:31:01 -0400 Subject: [PATCH 071/372] style fix --- src/Core/InterpolateDescription.cpp | 2 +- src/Core/InterpolateDescription.h | 11 +---------- src/Interpreters/FillingRow.cpp | 4 +--- src/Interpreters/FillingRow.h | 6 ++++-- src/Parsers/ASTInterpolateElement.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 4 ++-- src/Processors/Transforms/FillingTransform.cpp | 5 ++--- 8 files changed, 13 insertions(+), 23 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 1f5c343fddd..3545880b5bf 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -25,7 +25,7 @@ void dumpInterpolateDescription(const InterpolateDescription & description, cons void InterpolateColumnDescription::interpolate(Field & field) const { - if(field.isNull()) + if (field.isNull()) return; Block expr_columns; expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 7cd9f86890e..bb2b5febdc9 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -33,22 +33,13 @@ struct InterpolateColumnDescription ColumnWithTypeAndName column; ExpressionActionsPtr actions; - // ExecutableFunctionExpression interpolate_func; - // std::string column_name; /// The name of the column. - // size_t column_number; /// Column number (used if no name is given). - - explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : column(column_), actions(actions_) {} - // explicit InterpolateColumnDescription(size_t column_number_, const ASTInterpolateElement & /*ast*/) : column_number(column_number_) {} - - // explicit InterpolateColumnDescription(const std::string & column_name_, const ASTInterpolateElement & /*ast*/) : column_name(column_name_), column_number(0) {} - bool operator == (const InterpolateColumnDescription & other) const { - return column == other.column;// && column_number == other.column_number; + return column == other.column; } bool operator != (const InterpolateColumnDescription & other) const diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 8cc918d2c72..873c48702ae 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -51,10 +51,8 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; - for(size_t i = 0; i < to_row.interpolate.size(); ++i) { - std::cout << to_row.interpolate[i] <<" : "; + for (size_t i = 0; i < to_row.interpolate.size(); ++i) interpolate[i] = to_row.interpolate[i]; - } /// Find position we need to increment for generating next row. for (; pos < sort.size(); ++pos) diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 6081effe703..10adb6b72a8 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,7 +18,8 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - struct { + struct + { FillingRow & filling_row; Field & operator[](size_t index) { return filling_row.row[index]; } @@ -26,7 +27,8 @@ public: size_t size() const { return filling_row.sort_description.size(); } } sort; - struct { + struct + { FillingRow & filling_row; Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 14aa5f7e672..07d4321e69e 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -7,9 +7,9 @@ namespace DB { +/// TODO JOO void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const { - } } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index b0cd2f5e4db..dbcfeb8e8b5 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2331,7 +2331,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ASTPtr expr; if (!element_p.parse(pos, expr, expected)) return false; - + auto elem = std::make_shared(); elem->column = ident; elem->expr = expr; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 85f08ad11b0..6cc00da1db0 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -245,8 +245,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; /// if any WITH FILL parse possible INTERPOLATE list - if ( std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), - [](auto & child) { return child->template as()->with_fill; }) ) + if (std::any_of(order_expression_list->children.begin(), order_expression_list->children.end(), + [](auto & child) { return child->template as()->with_fill; })) { if (s_interpolate.ignore(pos, expected)) { diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 6113b4479a8..ca50c2fa236 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -179,7 +179,7 @@ FillingTransform::FillingTransform( size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); -/* TODO JOO check types? + /* TODO JOO check types? auto & descr = filling_row.getFillDescription(i); const auto & type = header_.getByPosition(block_position).type; @@ -193,8 +193,7 @@ FillingTransform::FillingTransform( { throw Exception("WITH FILL bound values cannot be negative for unsigned type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - } -*/ + } */ } std::set unique_positions; From d1369165084dabdbdfe72378df8c84c4066235dd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 17 Mar 2022 10:49:14 +0100 Subject: [PATCH 072/372] Remove testmode option --- src/Client/ClientBase.cpp | 22 +++---------------- src/Client/TestHint.cpp | 5 +---- src/Client/TestHint.h | 8 +++---- tests/clickhouse-test | 2 +- ...825_protobuf_format_no_length_delimiter.sh | 2 +- ..._block_size_rows_for_materialized_views.sh | 4 ++-- .../01280_ssd_complex_key_dictionary.sh | 4 ++-- ...006_client_test_hint_no_such_error_name.sh | 2 +- .../02234_clickhouse_local_test_mode.sh | 3 +-- ..._parallel_processing_on_replicas_part_1.sh | 4 ++-- 10 files changed, 18 insertions(+), 38 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c575cd37a5f..be63b96f654 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1487,24 +1487,12 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( bool ClientBase::executeMultiQuery(const String & all_queries_text) { - // It makes sense not to base any control flow on this, so that it is - // the same in tests and in normal usage. The only difference is that in - // normal mode we ignore the test hints. - const bool test_mode = config().has("testmode"); - if (test_mode) - { - /// disable logs if expects errors - TestHint test_hint(test_mode, all_queries_text); - if (test_hint.clientError() || test_hint.serverError()) - processTextAsSingleQuery("SET send_logs_level = 'fatal'"); - } - bool echo_query = echo_queries; /// Test tags are started with "--" so they are interpreted as comments anyway. /// But if the echo is enabled we have to remove the test tags from `all_queries_text` /// because we don't want test tags to be echoed. - size_t test_tags_length = test_mode ? getTestTagsLength(all_queries_text) : 0; + size_t test_tags_length = getTestTagsLength(all_queries_text); /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. @@ -1541,7 +1529,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // Try to find test hint for syntax error. We don't know where // the query ends because we failed to parse it, so we consume // the entire line. - TestHint hint(test_mode, String(this_query_begin, this_query_end - this_query_begin)); + TestHint hint(String(this_query_begin, this_query_end - this_query_begin)); if (hint.serverError()) { // Syntax errors are considered as client errors @@ -1579,7 +1567,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // Look for the hint in the text of query + insert data + trailing // comments, e.g. insert into t format CSV 'a' -- { serverError 123 }. // Use the updated query boundaries we just calculated. - TestHint test_hint(test_mode, full_query); + TestHint test_hint(full_query); // Echo all queries if asked; makes for a more readable reference file. echo_query = test_hint.echoQueries().value_or(echo_query); @@ -2182,8 +2170,6 @@ void ClientBase::init(int argc, char ** argv) ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") - ("testmode,T", "enable test hints in comments") - ("format,f", po::value(), "default output format") ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") @@ -2289,8 +2275,6 @@ void ClientBase::init(int argc, char ** argv) config().setBool("interactive", true); if (options.count("pager")) config().setString("pager", options["pager"].as()); - if (options.count("testmode")) - config().setBool("testmode", true); if (options.count("log-level")) Poco::Logger::root().setLevel(options["log-level"].as()); diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index 2f3be2a5350..f6d1e5d73c3 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -32,12 +32,9 @@ int parseErrorCode(DB::ReadBufferFromString & in) namespace DB { -TestHint::TestHint(bool enabled_, const String & query_) +TestHint::TestHint(const String & query_) : query(query_) { - if (!enabled_) - return; - // Don't parse error hints in leading comments, because it feels weird. // Leading 'echo' hint is OK. bool is_leading_hint = true; diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index 377637d0db8..7fa4e86c025 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -7,7 +7,7 @@ namespace DB { -/// Checks expected server and client error codes in --testmode. +/// Checks expected server and client error codes. /// /// The following comment hints are supported: /// @@ -25,12 +25,12 @@ namespace DB /// /// Examples: /// -/// - echo 'select / -- { clientError 62 }' | clickhouse-client --testmode -nm +/// - echo 'select / -- { clientError 62 }' | clickhouse-client -nm /// // Here the client parses the query but it is incorrect, so it expects /// SYNTAX_ERROR (62). /// -/// - echo 'select foo -- { serverError 47 }' | clickhouse-client --testmode -nm +/// - echo 'select foo -- { serverError 47 }' | clickhouse-client -nm /// /// But here the query is correct, but there is no such column "foo", so it /// is UNKNOWN_IDENTIFIER server error. @@ -43,7 +43,7 @@ namespace DB class TestHint { public: - TestHint(bool enabled_, const String & query_); + TestHint(const String & query_); int serverError() const { return server_error; } int clientError() const { return client_error; } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 121a283d0e4..9c2d599e9cd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -657,7 +657,7 @@ class TestCase: pattern = '{test} > {stdout} 2> {stderr}' if self.ext == '.sql': - pattern = "{client} --send_logs_level={logs_level} --testmode --multiquery {options} < " + pattern + pattern = "{client} --send_logs_level={logs_level} --multiquery {options} < " + pattern command = pattern.format(**params) diff --git a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh index a16345c4bb1..a1bbdc318d5 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh @@ -43,7 +43,7 @@ $CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_no_length_delimiter_protobuf rm "$BINARY_FILE_PATH" # The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. -$CLICKHOUSE_CLIENT --multiquery --testmode > /dev/null < /dev/null < /dev/null 2>&1 # fails echo "Should throw 1" -execute_insert --testmode +execute_insert echo "Should throw 2" -execute_insert --testmode --min_insert_block_size_rows=1 --min_insert_block_size_rows_for_materialized_views=$((1<<20)) +execute_insert --min_insert_block_size_rows=1 --min_insert_block_size_rows_for_materialized_views=$((1<<20)) # passes echo "Should pass 1" diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh index d5cae099f36..0de8b3a1a25 100755 --- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh +++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT -n --query=" LIFETIME(MIN 1000 MAX 2000) LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));" -$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }" +$CLICKHOUSE_CLIENT -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }" $CLICKHOUSE_CLIENT -n --query=" SELECT 'TEST_SMALL'; @@ -65,7 +65,7 @@ $CLICKHOUSE_CLIENT -n --query=" SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('10', toInt32(-20))); SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));" -$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }" +$CLICKHOUSE_CLIENT -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }" $CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY 01280_db.ssd_dict; DROP TABLE IF EXISTS 01280_db.keys_table; diff --git a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh index b846136ae58..972ff3ba73f 100755 --- a/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh +++ b/tests/queries/0_stateless/02006_client_test_hint_no_such_error_name.sh @@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --testmode -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*' +$CLICKHOUSE_CLIENT -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*' diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh index 6abe1e30334..f736751726d 100755 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh @@ -6,5 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL --query="SELECT n" 2>&1 | grep -q "Code: 47. DB::Exception: Missing columns:" && echo 'OK' || echo 'FAIL' ||: -$CLICKHOUSE_LOCAL --testmode --query="SELECT n -- { serverError 47 }" - +$CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }" diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index 276fc0274c2..58ce66056af 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -68,8 +68,8 @@ do TESTNAME_RESULT="/tmp/result_$TESTNAME" NEW_TESTNAME_RESULT="/tmp/result_dist_$TESTNAME" - $CLICKHOUSE_CLIENT $SETTINGS -nm --testmode < $TESTPATH > $TESTNAME_RESULT - $CLICKHOUSE_CLIENT $SETTINGS -nm --testmode < $NEW_TESTNAME > $NEW_TESTNAME_RESULT + $CLICKHOUSE_CLIENT $SETTINGS -nm < $TESTPATH > $TESTNAME_RESULT + $CLICKHOUSE_CLIENT $SETTINGS -nm < $NEW_TESTNAME > $NEW_TESTNAME_RESULT expected=$(cat $TESTNAME_RESULT | md5sum) actual=$(cat $NEW_TESTNAME_RESULT | md5sum) From 07d952b728abfb5966135671d9ee5daacdcc05ef Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Mar 2022 18:26:18 +0100 Subject: [PATCH 073/372] use snapshots for semistructured data, durability fixes --- src/Client/ClientBase.cpp | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterOptimizeQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MergeTreeTransaction.cpp | 11 +++++++- src/Interpreters/MutationsInterpreter.cpp | 2 +- .../TransactionVersionMetadata.cpp | 3 ++- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/IStorage.h | 11 +++++--- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++++++++--- src/Storages/MergeTree/MergeTreeData.h | 3 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +--- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 26 +++++++++++++++++-- .../MergeTree/MergeTreeMutationEntry.h | 4 ++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- .../ReadFinalForExternalReplicaStorage.cpp | 2 +- src/Storages/StorageBuffer.cpp | 4 +-- src/Storages/StorageDistributed.cpp | 6 ++--- src/Storages/StorageDistributed.h | 4 +-- src/Storages/StorageMaterializedView.cpp | 4 +-- src/Storages/StorageMemory.cpp | 11 ++++---- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMerge.cpp | 6 ++--- src/Storages/StorageMergeTree.cpp | 20 ++++++++++++-- src/Storages/StorageMergeTree.h | 2 ++ src/Storages/StorageProxy.h | 2 +- src/Storages/StorageTableFunction.h | 2 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- 31 files changed, 121 insertions(+), 50 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 4f1c1f4539e..c187222bb1e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1096,7 +1096,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, {}, diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index da5fcedd469..9919b1272bd 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -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(); } diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 05b65ac5d51..83bf23ab4ad 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -33,7 +33,7 @@ BlockIO InterpreterOptimizeQuery::execute() 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; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 087108e7d79..708429acf9f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -329,7 +329,7 @@ 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()) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 2442c16772a..b51836ddb03 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -65,8 +65,10 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP else { /// Lock part for removal with special TID, so transactions will not try to remove it concurrently. - /// We lock it only in memory. + /// 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(); } } @@ -189,6 +191,9 @@ void MergeTreeTransaction::afterCommit(CSN assigned_csn) noexcept 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 @@ -220,7 +225,11 @@ bool MergeTreeTransaction::rollback() noexcept /// 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) { diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 5e795c5760a..f89601094f0 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -802,7 +802,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & /// 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()) diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 20ebc1170ed..7c139686916 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -383,7 +383,8 @@ void VersionMetadata::read(ReadBuffer & buf) { /// NOTE Metadata file may actually contain multiple creation TIDs, we need the last one. removal_tid = TransactionID::read(buf); - removal_tid_lock = removal_tid.getHash(); + if (!removal_tid.isEmpty()) + removal_tid_lock = removal_tid.getHash(); } else if (name == REMOVAL_CSN_STR) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 110d4308236..f4a30a9fee7 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -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); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 32ca936f039..85bd8754cb5 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -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(filelog_settings->max_threads.value, file_infos.file_names.size()); /// No files to parse diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a73cd3c4a21..734769a9b18 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -488,6 +488,11 @@ public: 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*/) { @@ -610,15 +615,15 @@ public: virtual std::optional 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(*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: diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 4c7465d587d..0686c9b39cb 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -615,7 +615,7 @@ bool StorageKafka::streamToViews() if (!table) throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); - auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr()); + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); // Create an INSERT query for streaming data auto insert = std::make_shared(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3eba1bb03ea..ef8f44dbae6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1396,7 +1396,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } /// Sanity checks - bool csn_order = !version.removal_csn || version.creation_csn <= version.removal_csn; + 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; @@ -4039,6 +4039,21 @@ DataPartsVector MergeTreeData::getVisibleDataPartsVector(ContextPtr local_contex 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; @@ -6420,12 +6435,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(); auto lock = lockParts(); - snapshot_data->parts = getDataPartsVectorForInternalUsage({DataPartState::Active}, lock); + snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock); return std::make_shared(*this, metadata_snapshot, object_columns, std::move(snapshot_data)); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index eab742a23a1..8c2d31e7b08 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -435,7 +435,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); @@ -473,6 +473,7 @@ public: /// 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; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index cdf3a7b902d..758a2f6b241 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -131,14 +131,11 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const auto & settings = context->getSettingsRef(); - auto parts_in_txn_snapshot = data.getVisibleDataPartsVector(context); - const auto & metadata_for_reading = storage_snapshot->getMetadataForQuery(); const auto & snapshot_data = assert_cast(*storage_snapshot->data); - /// FIXME: use one snapshot - const auto & parts = context->getCurrentTransaction() ? parts_in_txn_snapshot : snapshot_data.parts; + const auto & parts = snapshot_data.parts; if (!query_info.projection) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b636f7cd6af..d80a3525b4f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -284,7 +284,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( TemporaryPart temp_part; Block & block = block_with_partition.block; auto columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - auto storage_snapshot = data.getStorageSnapshot(metadata_snapshot); + auto storage_snapshot = data.getStorageSnapshot(metadata_snapshot, context); if (!storage_snapshot->object_columns.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 6c29714a474..0d557ed36c2 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -62,7 +62,11 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP *out << "commands: "; commands.writeText(*out); *out << "\n"; - if (!tid.isPrehistoric()) + if (tid.isPrehistoric()) + { + csn = Tx::PrehistoricCSN; + } + else { *out << "tid: "; TransactionID::write(tid, *out); @@ -99,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_) @@ -120,11 +132,21 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat commands.readText(*buf); *buf >> "\n"; - if (!buf->eof()) + 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); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 5fb92b9954d..47debe47a22 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -31,7 +31,7 @@ struct MergeTreeMutationEntry /// ID of transaction which has created mutation. TransactionID tid = Tx::PrehistoricTID; - CSN csn; + 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, @@ -45,6 +45,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_); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index cadfa85299c..5a527ca2d23 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1024,7 +1024,7 @@ bool StorageRabbitMQ::streamToViews() InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); auto block_io = interpreter.execute(); - auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr()); + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); auto column_names = block_io.pipeline.getHeader().getNames(); auto sample_block = storage_snapshot->getSampleBlockForColumns(column_names); diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index cf1c5c35629..a03ccb5cf43 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -54,7 +54,7 @@ Pipe readFinalFromNestedStorage( filter_column_name = expressions->children.back()->getColumnName(); } - auto nested_snapshot = nested_storage->getStorageSnapshot(nested_metadata); + auto nested_snapshot = nested_storage->getStorageSnapshot(nested_metadata, context); Pipe pipe = nested_storage->read(require_columns_name, nested_snapshot, query_info, context, processed_stage, max_block_size, num_streams); pipe.addTableLock(lock); pipe.addStorageHolder(nested_storage); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 801e1b80a20..a503e79dc2c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -203,7 +203,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage( /// TODO: Find a way to support projections for StorageBuffer query_info.ignore_projections = true; const auto & destination_metadata = destination->getInMemoryMetadataPtr(); - return destination->getQueryProcessingStage(local_context, to_stage, destination->getStorageSnapshot(destination_metadata), query_info); + return destination->getQueryProcessingStage(local_context, to_stage, destination->getStorageSnapshot(destination_metadata, local_context), query_info); } return QueryProcessingStage::FetchColumns; @@ -248,7 +248,7 @@ void StorageBuffer::read( auto destination_lock = destination->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto destination_metadata_snapshot = destination->getInMemoryMetadataPtr(); - auto destination_snapshot = destination->getStorageSnapshot(destination_metadata_snapshot); + auto destination_snapshot = destination->getStorageSnapshot(destination_metadata_snapshot, local_context); const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [metadata_snapshot, destination_metadata_snapshot](const String& column_name) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1a390f784a2..62ec2524a32 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -617,13 +617,13 @@ static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr return false; } -StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const +StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const { - return getStorageSnapshotForQuery(metadata_snapshot, nullptr); + return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); } StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( - const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query) const + const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr /*query_context*/) const { /// If query doesn't use columns of type Object, don't deduce /// concrete types for them, because it required extra round trip. diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 317463783ee..a890cabd8b1 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -69,9 +69,9 @@ public: ColumnsDescriptionByShardNum objects_by_shard; }; - StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const override; + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; StorageSnapshotPtr getStorageSnapshotForQuery( - const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query) const override; + const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr query_context) const override; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 008b42e3299..72b0433ed6f 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -140,7 +140,7 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( /// converting and use it just like a normal view. query_info.ignore_projections = true; const auto & target_metadata = getTargetTable()->getInMemoryMetadataPtr(); - return getTargetTable()->getQueryProcessingStage(local_context, to_stage, getTargetTable()->getStorageSnapshot(target_metadata), query_info); + return getTargetTable()->getQueryProcessingStage(local_context, to_stage, getTargetTable()->getStorageSnapshot(target_metadata, local_context), query_info); } Pipe StorageMaterializedView::read( @@ -172,7 +172,7 @@ void StorageMaterializedView::read( auto storage = getTargetTable(); auto lock = storage->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto target_metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto target_storage_snapshot = storage->getStorageSnapshot(target_metadata_snapshot); + auto target_storage_snapshot = storage->getStorageSnapshot(target_metadata_snapshot, local_context); if (query_info.order_optimizer) query_info.input_order_info = query_info.order_optimizer->getInputOrder(target_metadata_snapshot, local_context); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c3601b33a04..f2d53dfa0d5 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -110,10 +110,11 @@ class MemorySink : public SinkToStorage public: MemorySink( StorageMemory & storage_, - const StorageMetadataPtr & metadata_snapshot_) + const StorageMetadataPtr & metadata_snapshot_, + ContextPtr context) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) - , storage_snapshot(storage_.getStorageSnapshot(metadata_snapshot_)) + , storage_snapshot(storage_.getStorageSnapshot(metadata_snapshot_, context)) { } @@ -190,7 +191,7 @@ StorageMemory::StorageMemory( setInMemoryMetadata(storage_metadata); } -StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const +StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const { auto snapshot_data = std::make_unique(); snapshot_data->blocks = data.get(); @@ -260,9 +261,9 @@ Pipe StorageMemory::read( } -SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot, context); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 1c4421e51a6..cb308ad5c58 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -36,7 +36,7 @@ public: std::shared_ptr blocks; }; - StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const override; + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; Pipe read( const Names & column_names, diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 96e6070e09e..8b71cfdb102 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -201,7 +201,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( stage_in_source_tables = std::max( stage_in_source_tables, table->getQueryProcessingStage(local_context, to_stage, - table->getStorageSnapshot(table->getInMemoryMetadataPtr()), query_info)); + table->getStorageSnapshot(table->getInMemoryMetadataPtr(), local_context), query_info)); } iterator->next(); @@ -338,7 +338,7 @@ Pipe StorageMerge::read( Aliases aliases; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); - auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot); + auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, local_context); auto modified_query_info = getModifiedQueryInfo(query_info, modified_context, storage->getStorageID(), storage->as()); auto syntax_result = TreeRewriter(local_context).analyzeSelect( @@ -377,7 +377,7 @@ Pipe StorageMerge::read( } syntax_result = TreeRewriter(local_context).analyze( - required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot)); + required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, local_context)); auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, local_context).getActionsDAG(true); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d37554f3d4a..f3425546023 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -539,6 +539,18 @@ void StorageMergeTree::waitForMutation(const String & mutation_id) LOG_INFO(log, "Mutation {} done", mutation_id); } +void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn) +{ + LOG_INFO(log, "Writing CSN {} for mutation {}", csn, mutation_id); + UInt64 version = MergeTreeMutationEntry::parseFileName(mutation_id); + + std::lock_guard lock(currently_processing_in_background_mutex); + auto it = current_mutations_by_version.find(version); + if (it == current_mutations_by_version.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find mutation {}", mutation_id); + it->second.writeCSN(csn); +} + void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { /// Validate partition IDs (if any) before starting mutation @@ -718,9 +730,13 @@ void StorageMergeTree::loadMutations() UInt64 block_number = entry.block_number; LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); - if (!entry.tid.isPrehistoric()) + if (!entry.tid.isPrehistoric() && !entry.csn) { - if (!TransactionLog::getCSN(entry.tid)) + if (auto csn = TransactionLog::getCSN(entry.tid)) + { + entry.writeCSN(csn); + } + else { TransactionLog::assertTIDIsNotOutdated(entry.tid); LOG_DEBUG(log, "Mutation entry {} was created by transaction {}, but it was not committed. Removing mutation entry", diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index cb0436c7af5..72900fd7a3c 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -175,6 +175,8 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version); void waitForMutation(const String & mutation_id) override; + void setMutationCSN(const String & mutation_id, CSN csn) override; + friend struct CurrentlyMergingPartsTagger; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index ce72633f024..b1eb190bd1d 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -41,7 +41,7 @@ public: /// TODO: Find a way to support projections for StorageProxy info.ignore_projections = true; const auto & nested_metadata = getNested()->getInMemoryMetadataPtr(); - return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata), info); + return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata, context), info); } Pipe watch( diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 4616421b24a..8bc1b160e77 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -104,7 +104,7 @@ public: for (const auto & c : column_names) cnames += c + " "; auto storage = getNested(); - auto nested_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr()); + auto nested_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); auto pipe = storage->read(column_names, nested_snapshot, query_info, context, processed_stage, max_block_size, num_streams); if (!pipe.empty() && add_conversion) diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 4cda9d6c9f5..66922afdd9c 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -117,7 +117,7 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) { using namespace DB; auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot); + auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, context); Names column_names; column_names.push_back("a"); From c28008a2df0156273e6bc166e665db1a6f6b9dd8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Mar 2022 12:01:26 +0100 Subject: [PATCH 074/372] more comments, minor fixes --- src/Common/TransactionID.h | 19 +++++---- src/Interpreters/Context.h | 2 +- src/Interpreters/MergeTreeTransaction.h | 10 ++--- src/Interpreters/MergeTreeTransactionHolder.h | 4 ++ src/Interpreters/TransactionLog.cpp | 2 +- .../TransactionVersionMetadata.cpp | 6 ++- src/Interpreters/TransactionVersionMetadata.h | 9 +++++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 40 +++++++++---------- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.h | 2 + src/Storages/StorageMergeTree.cpp | 6 +-- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++--- .../01169_alter_partition_isolation_stress.sh | 4 -- 14 files changed, 69 insertions(+), 50 deletions(-) diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 8cbb8b84217..574b42a8cb0 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -19,21 +19,29 @@ class MergeTreeTransaction; #define NO_TRANSACTION_RAW static_cast(nullptr) #endif +/// Commit Sequence Number using CSN = UInt64; -using Snapshot = CSN; +/// 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 MaxReservedCSN = 16; + const CSN MaxReservedCSN = 32; + + /// So far, that changes will never become visible + const CSN RolledBackCSN = std::numeric_limits::max(); const LocalTID PrehistoricLocalTID = 1; - const LocalTID DummyLocalTID = 1; - const LocalTID MaxReservedLocalTID = 16; + const LocalTID DummyLocalTID = 2; + const LocalTID MaxReservedLocalTID = 32; } struct TransactionID @@ -85,9 +93,6 @@ namespace Tx const TransactionID EmptyTID = {0, 0, UUIDHelpers::Nil}; const TransactionID PrehistoricTID = {PrehistoricCSN, PrehistoricLocalTID, UUIDHelpers::Nil}; const TransactionID DummyTID = {PrehistoricCSN, DummyLocalTID, UUIDHelpers::Nil}; - - /// So far, that changes will never become visible - const CSN RolledBackCSN = std::numeric_limits::max(); } } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dffd00954fd..8f3f7f3ed70 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -25,7 +26,6 @@ #include #include -#include #include diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 8d28c25fc51..a23418c7452 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -59,18 +59,18 @@ private: mutable std::mutex mutex; Stopwatch elapsed; + /// For now it's always equal to tid.start_csn, but it's possible to implement something like SET SNAPSHOT query for time-traveling CSN snapshot; + std::list::iterator snapshot_in_use_it; + /// Lists of changes made by transaction std::unordered_set storages; DataPartsVector creating_parts; DataPartsVector removing_parts; - - std::atomic csn; - - std::list::iterator snapshot_in_use_it; - using RunningMutationsList = std::vector>; RunningMutationsList mutations; + + std::atomic csn; }; using MergeTreeTransactionPtr = std::shared_ptr; diff --git a/src/Interpreters/MergeTreeTransactionHolder.h b/src/Interpreters/MergeTreeTransactionHolder.h index 88f226e18a1..4e8a196f4d2 100644 --- a/src/Interpreters/MergeTreeTransactionHolder.h +++ b/src/Interpreters/MergeTreeTransactionHolder.h @@ -10,6 +10,10 @@ 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; +/// 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: diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 7b8a85ea9ab..efc06fe7171 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -457,7 +457,7 @@ 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 + /// 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; diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index 7c139686916..d9a5beb1c8c 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -26,6 +26,10 @@ inline static CSN getCSNAndAssert(TIDHash tid_hash, std::atomic & csn, cons 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; @@ -212,7 +216,7 @@ bool VersionMetadata::isVisible(CSN snapshot_version, TransactionID current_tid) /// NOTE: Old enough committed parts always have written CSNs, /// so we can determine their visibility through fast path. /// But for long-running writing transactions we will always do - /// CNS lookup and get 0 (UnknownCSN) until the transaction is committer/rolled back. + /// CNS lookup and get 0 (UnknownCSN) until the transaction is committed/rolled back. creation = getCSNAndAssert(creation_tid.getHash(), creation_csn, &creation_tid); if (!creation) { diff --git a/src/Interpreters/TransactionVersionMetadata.h b/src/Interpreters/TransactionVersionMetadata.h index a397a4983b3..18ac445cc29 100644 --- a/src/Interpreters/TransactionVersionMetadata.h +++ b/src/Interpreters/TransactionVersionMetadata.h @@ -24,14 +24,21 @@ struct TransactionInfoContext 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 removal_tid_lock = 0; + /// CSN of transaction that has created this object std::atomic creation_csn = Tx::UnknownCSN; + /// CSN of transaction that has removed this object std::atomic removal_csn = Tx::UnknownCSN; /// Checks if an object is visible for transaction or not. @@ -41,8 +48,10 @@ struct VersionMetadata 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; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1f466214a71..e0348b6c3e6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1196,7 +1196,7 @@ void IMergeTreeDataPart::appendRemovalTIDToVersionMetadata(bool clear) const version.writeRemovalTID(*out, clear); out->finalize(); - /// fsync is not required when we clearing removal TID, because after hard restart we will fix metedata + /// fsync is not required when we clearing removal TID, because after hard restart we will fix metadata if (!clear) out->sync(); } @@ -1272,6 +1272,7 @@ 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; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ef8f44dbae6..8d819931e62 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4078,30 +4078,28 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVector(CSN snap void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts, CSN snapshot_version, TransactionID current_tid) const { - if (maybe_visible_parts.empty()) - return; + [[maybe_unused]] size_t total_size = maybe_visible_parts.size(); - auto it = maybe_visible_parts.begin(); - auto it_last = maybe_visible_parts.end() - 1; - String visible_parts_str; - while (it <= it_last) + auto need_remove_pred = [snapshot_version, ¤t_tid] (const DataPartPtr & part) -> bool { - if ((*it)->version.isVisible(snapshot_version, current_tid)) - { - visible_parts_str += (*it)->name; - visible_parts_str += " "; - ++it; - } - else - { - std::swap(*it, *it_last); - --it_last; - } - } + return !part->version.isVisible(snapshot_version, current_tid); + }; - size_t new_size = it_last - maybe_visible_parts.begin() + 1; - LOG_TEST(log, "Got {} parts visible in snapshot {} (TID {}): {}", new_size, snapshot_version, current_tid, visible_parts_str); - maybe_visible_parts.resize(new_size); + 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(), ", ")); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index a66f040b3ab..b63d46ee463 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -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, nullptr); + 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) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 47debe47a22..466ab0e3223 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -31,6 +31,8 @@ struct MergeTreeMutationEntry /// 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. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f3425546023..bf2433ed0f4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -734,6 +734,7 @@ void StorageMergeTree::loadMutations() { if (auto csn = TransactionLog::getCSN(entry.tid)) { + /// Transaction is committed => mutation is finished, but let's load it anyway (so it will be shown in system.mutations) entry.writeCSN(csn); } else @@ -744,7 +745,6 @@ void StorageMergeTree::loadMutations() disk->removeFile(it->path()); continue; } - /// Transaction is committed => mutation is finished, but let's load it anyway (so it will be shown in system.mutations) } auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; @@ -1011,7 +1011,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto last_mutation_to_apply = mutations_end_it; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { - /// Do not squash mutation from different transactions to be able to commit/rollback them independently. + /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; @@ -1447,7 +1447,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(MergeTreeTransaction * txn, c void StorageMergeTree::dropPartNoWaitNoThrow(const String & part_name) { - if (auto part = outdatePart(nullptr, part_name, /*force=*/ false)) + if (auto part = outdatePart(NO_TRANSACTION_RAW, part_name, /*force=*/ false)) dropPartsImpl({part}, /*detach=*/ false); /// Else nothing to do, part was removed in some different way diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1f8eeb21486..7dd2d79dc6f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2445,7 +2445,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo } } - removePartsFromWorkingSet(nullptr, parts_to_remove_from_working_set, true); + removePartsFromWorkingSet(NO_TRANSACTION_RAW, parts_to_remove_from_working_set, true); std::unordered_set created_get_parts; @@ -3841,7 +3841,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora { get_part = [&, part_to_clone]() { - return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, nullptr); + return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, NO_TRANSACTION_PTR); }; } else @@ -6347,7 +6347,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - Transaction transaction(*this, nullptr); + Transaction transaction(*this, NO_TRANSACTION_RAW); { auto data_parts_lock = lockParts(); @@ -6564,7 +6564,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta entry.toString(), zkutil::CreateMode::PersistentSequential)); { - Transaction transaction(*dest_table_storage, nullptr); + Transaction transaction(*dest_table_storage, NO_TRANSACTION_RAW); auto src_data_parts_lock = lockParts(); auto dest_data_parts_lock = dest_table_storage->lockParts(); @@ -7618,7 +7618,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, nullptr); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, NO_TRANSACTION_PTR); bool sync_on_insert = settings->fsync_after_insert; diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 99ae9741aab..908b3ab9f12 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -20,10 +20,6 @@ function thread_insert() trap "exit 0" INT val=1 while true; do - action="ROLLBACK" - if (( RANDOM % 2 )); then - action="COMMIT" - fi $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); From af3bb3b7aff2aee5620c1bdf5943e59feb1bfc0c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Mar 2022 19:40:53 +0100 Subject: [PATCH 075/372] Add WriteSettings --- src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp | 3 ++- src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h | 3 ++- src/Disks/DiskCacheWrapper.cpp | 2 +- src/Disks/DiskCacheWrapper.h | 2 +- src/Disks/DiskDecorator.cpp | 4 ++-- src/Disks/DiskDecorator.h | 3 ++- src/Disks/DiskEncrypted.cpp | 2 +- src/Disks/DiskEncrypted.h | 3 ++- src/Disks/DiskLocal.cpp | 2 +- src/Disks/DiskLocal.h | 3 ++- src/Disks/DiskMemory.cpp | 2 +- src/Disks/DiskMemory.h | 3 ++- src/Disks/DiskRestartProxy.cpp | 4 ++-- src/Disks/DiskRestartProxy.h | 2 +- src/Disks/DiskWebServer.h | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/HDFS/DiskHDFS.h | 2 +- src/Disks/IDisk.h | 4 +++- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/DiskS3.h | 3 ++- src/IO/WriteSettings.h | 11 +++++++++++ 21 files changed, 42 insertions(+), 22 deletions(-) create mode 100644 src/IO/WriteSettings.h diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index fb07d8c356b..97e98fb3a3e 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -90,7 +90,8 @@ std::unique_ptr DiskAzureBlobStorage::readFile( std::unique_ptr DiskAzureBlobStorage::writeFile( const String & path, size_t buf_size, - WriteMode mode) + WriteMode mode, + const WriteSettings &) { auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h index 63c3c735812..efc245e7eb3 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h @@ -56,7 +56,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; DiskType getType() const override; diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 3519b1212a4..d73c1ed5042 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -196,7 +196,7 @@ DiskCacheWrapper::readFile( } std::unique_ptr -DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode) +DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { if (!cache_file_predicate(path)) return DiskDecorator::writeFile(path, buf_size, mode); diff --git a/src/Disks/DiskCacheWrapper.h b/src/Disks/DiskCacheWrapper.h index dc66333758f..e413a3742f3 100644 --- a/src/Disks/DiskCacheWrapper.h +++ b/src/Disks/DiskCacheWrapper.h @@ -40,7 +40,7 @@ public: std::optional read_hint, std::optional file_size) const override; - std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; + std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index 37911f16913..14f507af55d 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -121,9 +121,9 @@ DiskDecorator::readFile( } std::unique_ptr -DiskDecorator::writeFile(const String & path, size_t buf_size, WriteMode mode) +DiskDecorator::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) { - return delegate->writeFile(path, buf_size, mode); + return delegate->writeFile(path, buf_size, mode, settings); } void DiskDecorator::removeFile(const String & path) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index bace54ff22a..33272ba385b 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -44,7 +44,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 714264b7720..3cee205fafc 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -269,7 +269,7 @@ std::unique_ptr DiskEncrypted::readFile( return std::make_unique(settings.local_fs_buffer_size, std::move(buffer), key, header); } -std::unique_ptr DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode) +std::unique_ptr DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { auto wrapped_path = wrappedPath(path); FileEncryption::Header header; diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index d99fe17457d..07a2ad81010 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -126,7 +126,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; void removeFile(const String & path) override { diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 44fdbb77323..c07a6ed32f6 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -344,7 +344,7 @@ std::unique_ptr DiskLocal::readFile(const String & path, } std::unique_ptr -DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) +DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; return std::make_unique(fs::path(disk_path) / path, buf_size, flags); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 76d5a88a626..59dcf5e5c13 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -79,7 +79,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index abaea0846a5..4f0e881e079 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -326,7 +326,7 @@ std::unique_ptr DiskMemory::readFile(const String & path return std::make_unique(path, iter->second.data); } -std::unique_ptr DiskMemory::writeFile(const String & path, size_t buf_size, WriteMode mode) +std::unique_ptr DiskMemory::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index fe108f53c68..726be8bc3b5 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -71,7 +71,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; diff --git a/src/Disks/DiskRestartProxy.cpp b/src/Disks/DiskRestartProxy.cpp index 43011a4cf72..a1c63d1e5a9 100644 --- a/src/Disks/DiskRestartProxy.cpp +++ b/src/Disks/DiskRestartProxy.cpp @@ -214,10 +214,10 @@ std::unique_ptr DiskRestartProxy::readFile( return std::make_unique(*this, std::move(impl)); } -std::unique_ptr DiskRestartProxy::writeFile(const String & path, size_t buf_size, WriteMode mode) +std::unique_ptr DiskRestartProxy::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) { ReadLock lock (mutex); - auto impl = DiskDecorator::writeFile(path, buf_size, mode); + auto impl = DiskDecorator::writeFile(path, buf_size, mode, settings); return std::make_unique(*this, std::move(impl)); } diff --git a/src/Disks/DiskRestartProxy.h b/src/Disks/DiskRestartProxy.h index 30f553f4fe0..2a0d40bffb6 100644 --- a/src/Disks/DiskRestartProxy.h +++ b/src/Disks/DiskRestartProxy.h @@ -48,7 +48,7 @@ public: const ReadSettings & settings, std::optional read_hint, std::optional file_size) const override; - std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; + std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; void removeDirectory(const String & path) override; diff --git a/src/Disks/DiskWebServer.h b/src/Disks/DiskWebServer.h index e2da0b2a1e1..5dbfbe8994a 100644 --- a/src/Disks/DiskWebServer.h +++ b/src/Disks/DiskWebServer.h @@ -100,7 +100,7 @@ public: /// Write and modification part - std::unique_ptr writeFile(const String &, size_t, WriteMode) override + std::unique_ptr writeFile(const String &, size_t, WriteMode, const WriteSettings &) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk {} is read-only", getName()); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 7f60b219a4b..f78ecd2669a 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -88,7 +88,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, } -std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode) +std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { /// Path to store new HDFS object. auto file_name = getRandomName(); diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 23a108507b4..eba58101bc4 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -60,7 +60,7 @@ public: std::optional read_hint, std::optional file_size) const override; - std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; + std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) override; void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 4fa73b8eba8..ac48f4f4d89 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -168,7 +169,8 @@ public: virtual std::unique_ptr writeFile( /// NOLINT const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - WriteMode mode = WriteMode::Rewrite) = 0; + WriteMode mode = WriteMode::Rewrite, + const WriteSettings & settings = {}) = 0; /// Remove file. Throws exception if file doesn't exists or it's a directory. virtual void removeFile(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index e46620d9d1f..3061754bd2d 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -244,7 +244,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co } } -std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode) +std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { auto settings = current_settings.get(); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 7e39c9d9b3c..32eb9ee7aef 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -88,7 +88,8 @@ public: std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode) override; + WriteMode mode, + const WriteSettings & settings) override; void removeFromRemoteFS(RemoteFSPathKeeperPtr keeper) override; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h new file mode 100644 index 00000000000..bcc89fbfc68 --- /dev/null +++ b/src/IO/WriteSettings.h @@ -0,0 +1,11 @@ + + +namespace DB +{ + +struct WriteSettings +{ + bool remote_fs_cache_on_insert = false; +}; + +} From 0719d01d39ae3e2853f67f67d53cc77028377584 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Mar 2022 14:33:59 +0100 Subject: [PATCH 076/372] more introspection --- src/Common/TransactionID.h | 1 + .../InterpreterTransactionControlQuery.cpp | 15 +++++++++++++ .../InterpreterTransactionControlQuery.h | 1 + src/Interpreters/MergeTreeTransaction.cpp | 5 +++++ src/Interpreters/MergeTreeTransaction.h | 3 ++- src/Interpreters/TransactionLog.cpp | 2 ++ .../TransactionVersionMetadata.cpp | 4 ++++ src/Parsers/ASTTransactionControl.cpp | 3 +++ src/Parsers/ASTTransactionControl.h | 3 +++ src/Parsers/ParserSetQuery.cpp | 4 ++++ src/Parsers/ParserTransactionControl.cpp | 18 +++++++++++++++- .../01169_alter_partition_isolation_stress.sh | 21 ++++++++++++++++--- .../01171_mv_select_insert_isolation_long.sh | 14 +++++++++++-- ...1173_transaction_control_queries.reference | 4 ++++ .../01173_transaction_control_queries.sql | 13 ++++++++++++ 15 files changed, 104 insertions(+), 7 deletions(-) diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 574b42a8cb0..3ab86f7589c 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -34,6 +34,7 @@ namespace Tx 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 diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index 962784f7671..61b2a4e865f 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -28,6 +28,8 @@ BlockIO InterpreterTransactionControlQuery::execute() 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(); @@ -72,4 +74,17 @@ BlockIO InterpreterTransactionControlQuery::executeRollback(ContextMutablePtr se 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 {}; +} + } diff --git a/src/Interpreters/InterpreterTransactionControlQuery.h b/src/Interpreters/InterpreterTransactionControlQuery.h index 996fb3247a6..05d3068e095 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.h +++ b/src/Interpreters/InterpreterTransactionControlQuery.h @@ -24,6 +24,7 @@ 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; diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index b51836ddb03..4d2ea266e50 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -20,6 +20,11 @@ MergeTreeTransaction::MergeTreeTransaction(CSN snapshot_, LocalTID local_tid_, U { } +void MergeTreeTransaction::setSnapshot(CSN new_snapshot) +{ + snapshot = new_snapshot; +} + MergeTreeTransaction::State MergeTreeTransaction::getState() const { CSN c = csn.load(); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index a23418c7452..fde11654a34 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -27,6 +27,7 @@ public: }; CSN getSnapshot() const { return snapshot; } + void setSnapshot(CSN new_snapshot); State getState() const; const TransactionID tid; @@ -59,7 +60,7 @@ private: mutable std::mutex mutex; Stopwatch elapsed; - /// For now it's always equal to tid.start_csn, but it's possible to implement something like SET SNAPSHOT query for time-traveling + /// 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::iterator snapshot_in_use_it; diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index efc06fe7171..393a8aa848b 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -275,6 +275,8 @@ void TransactionLog::removeOldEntries() 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); diff --git a/src/Interpreters/TransactionVersionMetadata.cpp b/src/Interpreters/TransactionVersionMetadata.cpp index d9a5beb1c8c..ac02f29661d 100644 --- a/src/Interpreters/TransactionVersionMetadata.cpp +++ b/src/Interpreters/TransactionVersionMetadata.cpp @@ -171,6 +171,10 @@ bool VersionMetadata::isVisible(CSN snapshot_version, TransactionID current_tid) 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: diff --git a/src/Parsers/ASTTransactionControl.cpp b/src/Parsers/ASTTransactionControl.cpp index 2eb74903522..3ff29d9e43e 100644 --- a/src/Parsers/ASTTransactionControl.cpp +++ b/src/Parsers/ASTTransactionControl.cpp @@ -18,6 +18,9 @@ void ASTTransactionControl::formatImpl(const FormatSettings & format /*state*/, 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; } } diff --git a/src/Parsers/ASTTransactionControl.h b/src/Parsers/ASTTransactionControl.h index c01c4172627..06f578ff138 100644 --- a/src/Parsers/ASTTransactionControl.h +++ b/src/Parsers/ASTTransactionControl.h @@ -13,10 +13,13 @@ public: 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"; } diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index d50de5f700d..76d6a299857 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -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; diff --git a/src/Parsers/ParserTransactionControl.cpp b/src/Parsers/ParserTransactionControl.cpp index a5591a0447e..da593170002 100644 --- a/src/Parsers/ParserTransactionControl.cpp +++ b/src/Parsers/ParserTransactionControl.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include namespace DB { @@ -8,6 +10,7 @@ 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; @@ -15,10 +18,23 @@ bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & ex 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()->value; + if (!snapshot_num.tryGet(snapshot)) + return false; + } else return false; - node = std::make_shared(action); + auto ast = std::make_shared(action); + ast->snapshot = snapshot; + node = ast; return true; } diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 908b3ab9f12..ab348fd31fb 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -50,7 +50,12 @@ function thread_partition_src_to_dst() SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; COMMIT;" 2>&1) ||: - echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" ||: + echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select $i, 'src', type, n, _part from src order by type, n; + select $i, 'dst', type, n, _part from dst order by type, n; + rollback" ||: echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1)) echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || sum=$((sum+i)) done @@ -74,7 +79,12 @@ function thread_partition_dst_to_src() SET throw_on_unsupported_query_inside_transaction=0; SYSTEM START MERGES dst; SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=4) != (toUInt8($i/2 + 1), (select sum(number) from numbers(1, $i) where number % 2 or number=$i))) FORMAT Null; - $action;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" + $action;" || $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select $i, 'src', type, n, _part from src order by type, n; + select $i, 'dst', type, n, _part from dst order by type, n; + rollback" ||: done } @@ -93,7 +103,12 @@ function thread_select() SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null; -- all rows are inserted in insert_thread SELECT type, throwIf(count(n) != max(n)), throwIf(sum(n) != max(n)*(max(n)+1)/2) FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type FORMAT Null; - COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT _table, type, arraySort(groupArray(n)) FROM merge(currentDatabase(), '') GROUP BY _table, type ORDER BY _table, type" + COMMIT;" || $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select $i, 'src', type, n, _part from src order by type, n; + select $i, 'dst', type, n, _part from dst order by type, n; + rollback" ||: done } diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 755c47011e5..3de63615bc4 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -89,7 +89,12 @@ function thread_select() SELECT throwIf((SELECT (sum(nm), count() % 2) FROM dst) != (0, 1)) FORMAT Null; SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(nm)) FROM dst)) FORMAT Null; SELECT throwIf((SELECT arraySort(groupArray(nm)) FROM mv) != (SELECT arraySort(groupArray(n*m)) FROM src)) FORMAT Null; - COMMIT;" || $CLICKHOUSE_CLIENT -q "SELECT 'src', arraySort(groupArray(n*m)), arraySort(groupArray((n*m, _part))) FROM src UNION ALL SELECT 'mv', arraySort(groupArray(nm)), arraySort(groupArray((nm, _part))) FROM mv" + COMMIT;" || $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select 'src', n, m, _part from src order by n, m; + select 'dst', nm, _part from dst order by nm; + rollback" ||: done } @@ -110,7 +115,12 @@ function thread_select_insert() -- now check that all results are the same SELECT throwIf(1 != (SELECT countDistinct(arr) FROM (SELECT x, arraySort(groupArray(nm)) AS arr FROM tmp WHERE x!=4 GROUP BY x))) FORMAT Null; SELECT throwIf((SELECT count(), sum(nm) FROM tmp WHERE x=4) != (SELECT count(), sum(nm) FROM tmp WHERE x!=4)) FORMAT Null; - ROLLBACK;" || $CLICKHOUSE_CLIENT -q "SELECT x, arraySort(groupArray(nm)) AS arr FROM tmp GROUP BY x" + ROLLBACK;" || $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select 'src', n, m, _part from src order by n, m; + select 'dst', nm, _part from dst order by nm; + rollback" ||: done } diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.reference b/tests/queries/0_stateless/01173_transaction_control_queries.reference index 6e27f29012b..01acdffc581 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.reference +++ b/tests/queries/0_stateless/01173_transaction_control_queries.reference @@ -6,3 +6,7 @@ on exception while processing [1,4,10,40] on session close [1,6,10,60] commit [1,7,10,70] readonly [1,7,10,70] +snapshot 2 8 +snapshot1 0 0 +snapshot3 1 +snapshot100500 2 8 diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index e8432978c89..6cf90ae4bfd 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -67,6 +67,19 @@ begin transaction; select 'readonly', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); commit; +begin transaction; +select 'snapshot', count(), sum(n) from mt1; +set transaction snapshot 1; +select 'snapshot1', count(), sum(n) from mt1; +set transaction snapshot 3; +set throw_on_unsupported_query_inside_transaction=0; +select 'snapshot3', count() = (select count() from system.parts where database=currentDatabase() and table='mt1' and _state in ('Active', 'Outdated')) from mt1; +set throw_on_unsupported_query_inside_transaction=1; +set transaction snapshot 1000000000000000; +select 'snapshot100500', count(), sum(n) from mt1; +set transaction snapshot 5; -- { serverError 646 } +rollback; + begin transaction; create table m (n int) engine=Memory; -- { serverError 48 } commit; -- { serverError 646 } From 418d52044a6f7ad40b78b521fe1ab9884550ad4b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Mar 2022 20:31:44 +0100 Subject: [PATCH 077/372] try fix merge assignment --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 66 ++++++++++++++++++- src/Storages/StorageMergeTree.cpp | 6 ++ 2 files changed, 69 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index d8dec4a0d24..f1225739656 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -29,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -128,9 +130,67 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const MergeTreeTransactionPtr & txn, String * out_disable_reason) { - /// NOTE It will contain uncommitted parts and future parts. - /// But It's ok since merge predicate allows to include in range visible parts only. - MergeTreeData::DataPartsVector data_parts = data.getDataPartsVectorForInternalUsage(); + 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(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bf2433ed0f4..df2ae72d0ff 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -795,6 +795,12 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( return false; if (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) return false; + + /// Do not try to merge parts that are locked for removal (merge will probably fail) + if (left && left->version.isRemovalTIDLocked()) + return false; + if (right && right->version.isRemovalTIDLocked()) + return false; } /// This predicate is checked for the first part of each range. From a8e1671a767d0f346121a8f33b36af48505c9c88 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 18 Mar 2022 16:44:27 -0400 Subject: [PATCH 078/372] type match check for INTERPOLATE expressions added, bugfix, printout fixed --- src/Interpreters/InterpreterSelectQuery.cpp | 22 +++++++++----- src/Parsers/ASTInterpolateElement.cpp | 6 ++-- src/Parsers/ASTSelectQuery.cpp | 9 ++++++ .../Transforms/FillingTransform.cpp | 29 +++++++++++-------- 4 files changed, 44 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6a4456b1caf..eb7faaf6bb5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -831,15 +831,21 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { InterpolateDescription interpolate_descr; - interpolate_descr.reserve(query.interpolate()->children.size()); - - for (const auto & elem : query.interpolate()->children) + if (query.interpolate()) { - auto interpolate = elem->as(); - auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); - interpolate_descr.emplace_back(block.findByName(interpolate.column->getColumnName())->cloneEmpty(), actions); + interpolate_descr.reserve(query.interpolate()->children.size()); + + for (const auto & elem : query.interpolate()->children) + { + auto interpolate = elem->as(); + ColumnWithTypeAndName column = block.findByName(interpolate.column->getColumnName())->cloneEmpty(); + + auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + + interpolate_descr.emplace_back(column, actions); + } } return interpolate_descr; diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 07d4321e69e..3da94f772f2 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -7,9 +7,11 @@ namespace DB { -/// TODO JOO -void ASTInterpolateElement::formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const +void ASTInterpolateElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + column->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + expr->formatImpl(settings, state, frame); } } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 3f40167b1d1..22b34150d7a 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -129,6 +129,15 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F s.one_line ? orderBy()->formatImpl(s, state, frame) : orderBy()->as().formatImplMultiline(s, state, frame); + + if (interpolate()) + { + s.ostr + << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : "") + << " ("; + interpolate()->formatImpl(s, state, frame); + s.ostr << " )"; + } } if (limitByLength()) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index ca50c2fa236..d9de72de16d 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -179,21 +179,26 @@ FillingTransform::FillingTransform( size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); - /* TODO JOO check types? - auto & descr = filling_row.getFillDescription(i); - const auto & type = header_.getByPosition(block_position).type; - if (!tryConvertFields(descr, type)) - throw Exception("Incompatible types of WITH FILL expression values with column type " - + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + /// Check column-expression compatibility + auto column = interpolate_description[i].column; + auto exp_type = interpolate_description[i].actions->getActionsDAG().getResultColumns()[0].type; + auto exp_column = exp_type->createColumn(); + exp_column->insertDefault(); - if (type->isValueRepresentedByUnsignedInteger() && - ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || - (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) + try { - throw Exception("WITH FILL bound values cannot be negative for unsigned type " - + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - } */ + if (auto exp_field = (*exp_column)[0]; convertFieldToType(exp_field, *column.type).isNull()) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", + exp_type->getName(), column.name, column.type->getName()); + } + catch (const Exception &) + { + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", + exp_type->getName(), column.name, column.type->getName()); + } } std::set unique_positions; From ecf05ecc49547b9dddfeb4877a0db5d0ba0e5b6e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 19 Mar 2022 02:41:12 -0400 Subject: [PATCH 079/372] tests are added, bugfix --- src/Core/InterpolateDescription.cpp | 6 ++- .../0_stateless/02233_interpolate_1.reference | 52 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 39 ++++++++++++++ 3 files changed, 95 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02233_interpolate_1.reference create mode 100644 tests/queries/0_stateless/02233_interpolate_1.sql diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 3545880b5bf..176909ca929 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -28,9 +29,10 @@ void InterpolateColumnDescription::interpolate(Field & field) const if (field.isNull()) return; Block expr_columns; - expr_columns.insert({column.type->createColumnConst(1, field), column.type, column.name}); + Field column_field = convertFieldToType(field, *column.type.get()); + expr_columns.insert({column.type->createColumnConst(1, column_field), column.type, column.name}); actions->execute(expr_columns); - expr_columns.getByPosition(0).column->get(0, field); + field = convertFieldToType((*expr_columns.getByPosition(0).column)[0], *column.type.get()); } void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference new file mode 100644 index 00000000000..45377d486f9 --- /dev/null +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -0,0 +1,52 @@ +0 0 +0.5 0 +1 original 1 +1.5 0 +2 0 +2.5 0 +3 0 +3.5 0 +4 original 4 +4.5 0 +5 0 +5.5 0 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 42 +2 42 +2.5 42 +3 42 +3.5 42 +4 original 4 +4.5 42 +5 42 +5.5 42 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 1 +2 1 +2.5 1 +3 1 +3.5 1 +4 original 4 +4.5 4 +5 4 +5.5 4 +7 original 7 +0 0 +0.5 0 +1 original 1 +1.5 2 +2 3 +2.5 4 +3 5 +3.5 6 +4 original 4 +4.5 5 +5 6 +5.5 7 +7 original 7 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql new file mode 100644 index 00000000000..dacfbd068ae --- /dev/null +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -0,0 +1,39 @@ +# Test WITH FILL without INTERPOLATE +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; + +# Test INTERPOLATE with const +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 42); + +# Test INTERPOLATE with field value +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter); + +# Test INTERPOLATE with expression +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); + +# Test INTERPOLATE with incompatible const - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 475 } + +# Test INTERPOLATE with incompatible expression - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 475 } + +# Test INTERPOLATE with column from WITH FILL expression - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } + +# Test INTERPOLATE with inconsistent column - should produce error +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 475 } From 90888ea95b4a11a792dd9d67761922a1c676d45c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:18:12 -0400 Subject: [PATCH 080/372] Update index.md INTERPOLATE is added --- docs/en/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 33644133153..0386083b5df 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -22,7 +22,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] From 40c91c3f981d0c7abd7fe4b684da07530bce8b52 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:21:10 -0400 Subject: [PATCH 081/372] Update index.md INTERPOLATE is added --- docs/ru/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 4a409dc7743..240a0fff7b6 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -20,7 +20,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] From 4f892dc9daac9a7704b444b004ceb777a4e57645 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 20:55:02 -0400 Subject: [PATCH 082/372] Update order-by.md INTERPOLATE is added --- .../statements/select/order-by.md | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index b24f0213e4e..b7e84173284 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -280,6 +280,7 @@ To fill multiple columns, add `WITH FILL` modifier with optional parameters afte ``` sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] +[INTERPOLATE (col AS expr, ... colN AS exprN)] ``` `WITH FILL` can be applied for fields with Numeric (all kinds of float, decimal, int) or Date/DateTime types. When applied for `String` fields, missed values are filled with empty strings. @@ -287,6 +288,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous field value by applying `expr`. Example of a query without `WITH FILL`: @@ -483,4 +485,62 @@ Result: └────────────┴────────────┴──────────┘ ``` +Example of a query without `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +``` + +Result: + +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 0 │ +│ 2 │ │ 0 │ +│ 2.5 │ │ 0 │ +│ 3 │ │ 0 │ +│ 3.5 │ │ 0 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 0 │ +│ 5 │ │ 0 │ +│ 5.5 │ │ 0 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + +Same query after applying `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +``` + +Result: + +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 2 │ +│ 2 │ │ 3 │ +│ 2.5 │ │ 4 │ +│ 3 │ │ 5 │ +│ 3.5 │ │ 6 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 5 │ +│ 5 │ │ 6 │ +│ 5.5 │ │ 7 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + [Original article](https://clickhouse.com/docs/en/sql-reference/statements/select/order-by/) From f9ed659959cc893a8cb19ef27b010d3aa70db522 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 21:04:54 -0400 Subject: [PATCH 083/372] Update order-by.md INTERPOLATE is added --- .../statements/select/order-by.md | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 3f52b260423..71f12ec73b0 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -280,6 +280,7 @@ SELECT * FROM collate_test ORDER BY s ASC COLLATE 'en'; ```sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] +[INTERPOLATE (col AS expr, ... colN AS exprN)] ``` `WITH FILL` может быть применен к полям с числовыми (все разновидности float, int, decimal) или временными (все разновидности Date, DateTime) типами. В случае применения к полям типа `String` недостающие значения заполняются пустой строкой. @@ -289,6 +290,8 @@ ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_ Когда `STEP const_numeric_expr` не указан, тогда используется `1.0` для числовых типов, `1 день` для типа Date и `1 секунда` для типа DateTime. +`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. + Пример запроса без использования `WITH FILL`: ```sql SELECT n, source FROM ( @@ -395,3 +398,58 @@ ORDER BY │ 1970-03-12 │ 1970-01-08 │ original │ └────────────┴────────────┴──────────┘ ``` + +Пример запроса без `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +``` + +Результат: +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 0 │ +│ 2 │ │ 0 │ +│ 2.5 │ │ 0 │ +│ 3 │ │ 0 │ +│ 3.5 │ │ 0 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 0 │ +│ 5 │ │ 0 │ +│ 5.5 │ │ 0 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ +``` + +Тот же запрос с `INTERPOLATE`: + +``` sql +SELECT n, source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + FROM numbers(10) WHERE number % 3 = 1 +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +``` + +Результат: +``` text +┌───n─┬─source───┬─inter─┐ +│ 0 │ │ 0 │ +│ 0.5 │ │ 0 │ +│ 1 │ original │ 1 │ +│ 1.5 │ │ 2 │ +│ 2 │ │ 3 │ +│ 2.5 │ │ 4 │ +│ 3 │ │ 5 │ +│ 3.5 │ │ 6 │ +│ 4 │ original │ 4 │ +│ 4.5 │ │ 5 │ +│ 5 │ │ 6 │ +│ 5.5 │ │ 7 │ +│ 7 │ original │ 7 │ +└─────┴──────────┴───────┘ From 5c8a77dadeb60a3b2688601070a75e37559996fe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:37:58 -0400 Subject: [PATCH 084/372] Update order-by.md INTERPOLATE fix --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index b7e84173284..2ec1d23ef5a 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -518,7 +518,7 @@ Same query after applying `INTERPOLATE`: ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); ``` From b01f965f31db5b5ed9591bbddebaed2963342e70 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:39:30 -0400 Subject: [PATCH 085/372] Update order-by.md fix --- docs/ru/sql-reference/statements/select/order-by.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 71f12ec73b0..8a4a8d5d111 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -403,7 +403,7 @@ ORDER BY ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; ``` @@ -431,7 +431,7 @@ SELECT n, source, inter FROM ( ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); ``` From 5ae6f800c6cebc78d9f1f1353e00534b62233550 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 18 Mar 2022 23:40:09 -0400 Subject: [PATCH 086/372] Update order-by.md INTERPOLATE fix --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 2ec1d23ef5a..06d3b985854 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -489,7 +489,7 @@ Example of a query without `INTERPOLATE`: ``` sql SELECT n, source, inter FROM ( - SELECT toFloat32(number % 10) AS n, 'original' AS source, n as inter + SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; ``` From 481ee8aad5a5b1d7d2b22a0891d8e32b03e0c8e4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sat, 19 Mar 2022 12:17:56 -0400 Subject: [PATCH 087/372] Update FillingTransform.cpp use range-based for loop --- src/Processors/Transforms/FillingTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index d9de72de16d..c3991e393b9 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -174,15 +174,15 @@ FillingTransform::FillingTransform( } } - for (size_t i = 0, size = interpolate_description.size(); i < size; ++i) + for (const auto & descr : interpolate_description) { - size_t block_position = header_.getPositionByName(interpolate_description[i].column.name); + size_t block_position = header_.getPositionByName(descr.column.name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); /// Check column-expression compatibility - auto column = interpolate_description[i].column; - auto exp_type = interpolate_description[i].actions->getActionsDAG().getResultColumns()[0].type; + auto column = descr.column; + auto exp_type = descr.actions->getActionsDAG().getResultColumns()[0].type; auto exp_column = exp_type->createColumn(); exp_column->insertDefault(); From c4daf514d6aa99cd9445846d549b0795bc3bdfbd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sat, 19 Mar 2022 14:12:29 -0400 Subject: [PATCH 088/372] Update InterpreterSelectQuery.cpp bugfix: check column existence for INTERPOLATE expression target --- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fa3d2698cdc..4bef4c66ade 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -101,6 +101,7 @@ namespace ErrorCodes extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; + extern const int UNKNOWN_IDENTIFIER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -842,7 +843,11 @@ static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & q for (const auto & elem : query.interpolate()->children) { auto interpolate = elem->as(); - ColumnWithTypeAndName column = block.findByName(interpolate.column->getColumnName())->cloneEmpty(); + ColumnWithTypeAndName *block_column = block.findByName(interpolate.column->getColumnName()); + if (!block_column) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); + ColumnWithTypeAndName column = block_column->cloneEmpty(); auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); From bf05b949401a493aefc02e061730e941a7d9d3c0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 21 Mar 2022 15:03:28 +0800 Subject: [PATCH 089/372] fix build --- src/Storages/Hive/HiveFile.cpp | 3 --- src/Storages/Hive/HiveFile.h | 14 +++----------- src/TableFunctions/CMakeLists.txt | 5 ++--- 3 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 896c464c80f..e18eef58947 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -3,12 +3,10 @@ #if USE_HIVE #include -#include #include #include #include #include -#include #include #include #include @@ -19,7 +17,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index dfecd79f932..7d38efe6acb 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -7,11 +7,14 @@ #include #include +#include +#include #include #include #include #include +#include namespace orc { @@ -19,16 +22,6 @@ class Statistics; class ColumnStatistics; } -namespace parquet::arrow -{ -class FileReader; -} - -namespace arrow::adapters::orc -{ -class ORCFileReader; -} - namespace DB { namespace ErrorCodes @@ -36,7 +29,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -class ReadBufferFromHDFS; class IHiveFile : public WithContext { public: diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index c9948a4b131..1d6eb9eb02e 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -8,10 +8,9 @@ list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp TableFunc list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h) add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) +target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) if (TARGET ch_contrib::hivemetastore) - target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms ch_contrib::hivemetastore ch_contrib::hdfs) -else () - target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) + target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::hivemetastore ch_contrib::hdfs ch_contrib::parquet) endif () From 1d80262a2a4d150098e267b52660d768807bbe08 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Mar 2022 20:15:07 +0100 Subject: [PATCH 090/372] Add write settings, file cache settings --- src/Common/FileCache.cpp | 17 ++++---- src/Common/FileCache.h | 8 +--- src/Common/FileCacheFactory.cpp | 8 +--- src/Common/FileCacheFactory.h | 2 +- src/Common/FileCacheSettings.cpp | 16 ++++++++ src/Common/FileCacheSettings.h | 18 +++++++++ src/Common/FileCache_fwd.h | 3 ++ src/Disks/RemoteDisksCommon.cpp | 18 ++++++--- src/IO/WriteBufferFromS3.cpp | 40 ++++++++++++++----- src/IO/WriteBufferFromS3.h | 26 +++++++----- .../MergeTree/MergedBlockOutputStream.cpp | 8 ++-- .../MergeTree/MergedBlockOutputStream.h | 7 +++- 12 files changed, 118 insertions(+), 53 deletions(-) create mode 100644 src/Common/FileCacheSettings.cpp create mode 100644 src/Common/FileCacheSettings.h diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index dffa4fac44d..5cc2e707667 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -31,13 +32,11 @@ namespace IFileCache::IFileCache( const String & cache_base_path_, - size_t max_size_, - size_t max_element_size_, - size_t max_file_segment_size_) + const FileCacheSettings & cache_settings_) : cache_base_path(cache_base_path_) - , max_size(max_size_) - , max_element_size(max_element_size_) - , max_file_segment_size(max_file_segment_size_) + , max_size(cache_settings_.max_cache_size) + , max_element_size(cache_settings_.max_cache_elements) + , max_file_segment_size(cache_settings_.max_cache_elements) { } @@ -71,8 +70,8 @@ void IFileCache::assertInitialized() const throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Cache not initialized"); } -LRUFileCache::LRUFileCache(const String & cache_base_path_, size_t max_size_, size_t max_element_size_, size_t max_file_segment_size_) - : IFileCache(cache_base_path_, max_size_, max_element_size_, max_file_segment_size_) +LRUFileCache::LRUFileCache(const String & cache_base_path_, const FileCacheSettings & cache_settings_) + : IFileCache(cache_base_path_, cache_settings_) , log(&Poco::Logger::get("LRUFileCache")) { } @@ -364,7 +363,7 @@ bool LRUFileCache::tryReserve( auto is_overflow = [&] { - return (current_size + size - removed_size > max_size) + return (max_size != 0 && current_size + size - removed_size > max_size) || (max_element_size != 0 && queue_size > max_element_size); }; diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index d51dfe7a9ff..dd585369853 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -32,9 +32,7 @@ public: IFileCache( const String & cache_base_path_, - size_t max_size_, - size_t max_element_size_, - size_t max_file_segment_size_); + const FileCacheSettings & cache_settings_); virtual ~IFileCache() = default; @@ -111,9 +109,7 @@ class LRUFileCache final : public IFileCache public: LRUFileCache( const String & cache_base_path_, - size_t max_size_, - size_t max_element_size_ = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS, - size_t max_file_segment_size_ = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE); + const FileCacheSettings & cache_settings_); FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override; diff --git a/src/Common/FileCacheFactory.cpp b/src/Common/FileCacheFactory.cpp index fc8dff0b26c..567d091fdeb 100644 --- a/src/Common/FileCacheFactory.cpp +++ b/src/Common/FileCacheFactory.cpp @@ -24,18 +24,14 @@ FileCachePtr FileCacheFactory::getImpl(const std::string & cache_base_path, std: } FileCachePtr FileCacheFactory::getOrCreate( - const std::string & cache_base_path, size_t max_size, size_t max_elements_size, size_t max_file_segment_size) + const std::string & cache_base_path, const FileCacheSettings & file_cache_settings) { std::lock_guard lock(mutex); auto cache = getImpl(cache_base_path, lock); if (cache) - { - if (cache->capacity() != max_size) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cache with path `{}` already exists, but has different max size", cache_base_path); return cache; - } - cache = std::make_shared(cache_base_path, max_size, max_elements_size, max_file_segment_size); + cache = std::make_shared(cache_base_path, file_cache_settings); caches.emplace(cache_base_path, cache); return cache; } diff --git a/src/Common/FileCacheFactory.h b/src/Common/FileCacheFactory.h index f2432f03cae..176b96a862e 100644 --- a/src/Common/FileCacheFactory.h +++ b/src/Common/FileCacheFactory.h @@ -17,7 +17,7 @@ class FileCacheFactory final : private boost::noncopyable public: static FileCacheFactory & instance(); - FileCachePtr getOrCreate(const std::string & cache_base_path, size_t max_size, size_t max_elements_size, size_t max_file_segment_size); + FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings); private: FileCachePtr getImpl(const std::string & cache_base_path, std::lock_guard &); diff --git a/src/Common/FileCacheSettings.cpp b/src/Common/FileCacheSettings.cpp new file mode 100644 index 00000000000..39a6a2ec723 --- /dev/null +++ b/src/Common/FileCacheSettings.cpp @@ -0,0 +1,16 @@ +#include "FileCacheSettings.h" + +#include + +namespace DB +{ + +void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + max_cache_size = config.getUInt64(config_prefix + ".data_cache_max_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_CACHE_SIZE); + max_cache_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS); + max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE); + cache_on_insert = config.getUInt64(config_prefix + ".cache_on_insert", false); +} + +} diff --git a/src/Common/FileCacheSettings.h b/src/Common/FileCacheSettings.h new file mode 100644 index 00000000000..8aa2bfb9214 --- /dev/null +++ b/src/Common/FileCacheSettings.h @@ -0,0 +1,18 @@ +#include + +namespace Poco { namespace Util { class AbstractConfiguration; }} + +namespace DB +{ + +struct FileCacheSettings +{ + size_t max_cache_size = 0; + size_t max_cache_elements = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS; + size_t max_file_segment_size = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE; + bool cache_on_insert = false; + + void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); +}; + +} diff --git a/src/Common/FileCache_fwd.h b/src/Common/FileCache_fwd.h index cab1525600b..7448f0c8c89 100644 --- a/src/Common/FileCache_fwd.h +++ b/src/Common/FileCache_fwd.h @@ -4,10 +4,13 @@ namespace DB { +static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_CACHE_SIZE = 1024 * 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024; class IFileCache; using FileCachePtr = std::shared_ptr; +struct FileCacheSettings; + } diff --git a/src/Disks/RemoteDisksCommon.cpp b/src/Disks/RemoteDisksCommon.cpp index 36f2aed3e7c..4805434e5ee 100644 --- a/src/Disks/RemoteDisksCommon.cpp +++ b/src/Disks/RemoteDisksCommon.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -64,18 +65,23 @@ FileCachePtr getCachePtrForDisk( if (!fs::exists(cache_base_path)) fs::create_directories(cache_base_path); - LOG_INFO(&Poco::Logger::get("Disk(" + name + ")"), "Disk registered with cache path: {}", cache_base_path); - auto metadata_path = getDiskMetadataPath(name, config, config_prefix, context); if (metadata_path == cache_base_path) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata path and cache base path must be different: {}", metadata_path); - size_t max_cache_size = config.getUInt64(config_prefix + ".data_cache_max_size", 1024*1024*1024); - size_t max_cache_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS); - size_t max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE); + FileCacheSettings file_cache_settings; + file_cache_settings.loadFromConfig(config, config_prefix); - auto cache = FileCacheFactory::instance().getOrCreate(cache_base_path, max_cache_size, max_cache_elements, max_file_segment_size); + auto cache = FileCacheFactory::instance().getOrCreate(cache_base_path, file_cache_settings); cache->initialize(); + + auto * log = &Poco::Logger::get("Disk(" + name + ")"); + LOG_INFO(log, "Disk registered with cache path: {}. Cache size: {}, max cache elements size: {}, max_file_segment_size: {}", + cache_base_path, + max_cache_size ? toString(max_cache_size) : "UNLIMITED", + max_cache_elements ? toString(max_cache_elements) : "UNLIMITED", + max_file_segment_size); + return cache; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index eda7bb6f8ae..4f1016d43bd 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -2,17 +2,19 @@ #if USE_AWS_S3 -# include -# include +#include +#include -# include -# include -# include -# include -# include -# include +#include +#include -# include +#include +#include +#include +#include +#include + +#include namespace ProfileEvents @@ -59,7 +61,8 @@ WriteBufferFromS3::WriteBufferFromS3( size_t max_single_part_upload_size_, std::optional> object_metadata_, size_t buffer_size_, - ScheduleFunc schedule_) + ScheduleFunc schedule_, + FileCachePtr cache_) : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) , key(key_) @@ -70,6 +73,7 @@ WriteBufferFromS3::WriteBufferFromS3( , upload_part_size_multiply_threshold(upload_part_size_multiply_threshold_) , max_single_part_upload_size(max_single_part_upload_size_) , schedule(std::move(schedule_)) + , cache(cache_) { allocateBuffer(); } @@ -95,7 +99,6 @@ void WriteBufferFromS3::nextImpl() if (!multipart_upload_id.empty() && last_part_size > upload_part_size) { - writePart(); allocateBuffer(); @@ -126,6 +129,21 @@ WriteBufferFromS3::~WriteBufferFromS3() } } +void WriteBufferFromS3::tryWriteToCacheIfNeeded() +{ + if (!cache || IFileCache::shouldBypassCache()) + return; + + try + { + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } +} + void WriteBufferFromS3::preFinalize() { next(); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index a4fbcbcdeeb..595a7a929c1 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -4,16 +4,19 @@ #if USE_AWS_S3 -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include -# include -# include +#include +#include -# include +#include +#include + +#include namespace Aws::S3 { @@ -51,7 +54,8 @@ public: size_t max_single_part_upload_size_, std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - ScheduleFunc schedule_ = {}); + ScheduleFunc schedule_ = {}, + FileCachePtr cache_ = nullptr); ~WriteBufferFromS3() override; @@ -82,6 +86,8 @@ private: void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); + void tryWriteToCacheIfNeeded(); + String bucket; String key; std::optional> object_metadata; @@ -113,6 +119,8 @@ private: std::condition_variable bg_tasks_condvar; Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); + + FileCachePtr cache; }; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index f94c89e20bd..5bd9217226d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -122,7 +122,8 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( MergeTreeData::MutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list, - MergeTreeData::DataPart::Checksums * additional_column_checksums) + MergeTreeData::DataPart::Checksums * additional_column_checksums, + const WriteSettings & write_settings) { /// Finish write and get checksums. MergeTreeData::DataPart::Checksums checksums; @@ -156,7 +157,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( auto finalizer = std::make_unique(*writer, new_part, files_to_remove_after_sync, sync); if (new_part->isStoredOnDisk()) - finalizer->written_files = finalizePartOnDisk(new_part, checksums); + finalizer->written_files = finalizePartOnDisk(new_part, checksums, write_settings); new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); @@ -174,7 +175,8 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::DataPartPtr & new_part, - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, + const WriteSettings & write_settings) { WrittenFiles written_files; if (new_part->isProjectionPart()) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index c17cfd22cd8..05f70239517 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -54,7 +55,8 @@ public: MergeTreeData::MutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list = nullptr, - MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); + MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr, + const WriteSettings & write_settings = {}); void finalizePart( MergeTreeData::MutableDataPartPtr & new_part, @@ -71,7 +73,8 @@ private: using WrittenFiles = std::vector>; WrittenFiles finalizePartOnDisk( const MergeTreeData::DataPartPtr & new_part, - MergeTreeData::DataPart::Checksums & checksums); + MergeTreeData::DataPart::Checksums & checksums, + const WriteSettings & write_settings); NamesAndTypesList columns_list; IMergeTreeDataPart::MinMaxIndex minmax_idx; From 59b7394caf516e9ecca91f529dc02f7bf7955cf0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Mar 2022 09:52:48 +0100 Subject: [PATCH 091/372] WriteSettings --- src/Disks/DiskCacheWrapper.cpp | 19 +++++++++++++------ src/Disks/DiskLocal.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 4 ++-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 10 ++++++++++ src/Interpreters/Context.h | 4 ++++ .../MergeTree/MergeTreeDataWriter.cpp | 6 +++++- .../MergeTree/MergedBlockOutputStream.cpp | 18 +++++++++--------- .../MergeTree/MergedBlockOutputStream.h | 2 +- 9 files changed, 46 insertions(+), 21 deletions(-) diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index d73c1ed5042..de5397a87c2 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -167,7 +167,11 @@ DiskCacheWrapper::readFile( auto tmp_path = path + ".tmp"; { auto src_buffer = DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); - auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite); + + WriteSettings write_settings; + write_settings.remote_fs_cache_on_insert = false; + + auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite, write_settings); copyData(*src_buffer, *dst_buffer); } cache_disk->moveFile(tmp_path, path); @@ -196,10 +200,13 @@ DiskCacheWrapper::readFile( } std::unique_ptr -DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) +DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) { if (!cache_file_predicate(path)) - return DiskDecorator::writeFile(path, buf_size, mode); + return DiskDecorator::writeFile(path, buf_size, mode, settings); + + WriteSettings current_settings = settings; + current_settings.remote_fs_cache_on_insert = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); @@ -208,15 +215,15 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode cache_disk->createDirectories(dir_path); return std::make_unique( - cache_disk->writeFile(path, buf_size, mode), + cache_disk->writeFile(path, buf_size, mode, current_settings), [this, path]() { /// Copy file from cache to actual disk when cached buffer is finalized. return cache_disk->readFile(path, ReadSettings(), /* read_hint= */ {}, /* file_size= */ {}); }, - [this, path, buf_size, mode]() + [this, path, buf_size, mode, current_settings]() { - return DiskDecorator::writeFile(path, buf_size, mode); + return DiskDecorator::writeFile(path, buf_size, mode, current_settings); }); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c07a6ed32f6..78be223edee 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -623,7 +623,7 @@ bool DiskLocal::setup() pcg32_fast rng(randomSeed()); UInt32 magic_number = rng(); { - auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); writeIntBinary(magic_number, *buf); } disk_checker_magic_number = magic_number; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 3061754bd2d..797b7ed15f4 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -244,7 +244,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co } } -std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) +std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & write_settings) { auto settings = current_settings.get(); @@ -299,7 +299,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, settings->s3_upload_part_size_multiply_parts_count_threshold, settings->s3_max_single_part_upload_size, std::move(object_metadata), - buf_size, std::move(schedule)); + buf_size, std::move(schedule), write_settings.remote_fs_cache_on_insert ? cache : nullptr); auto create_metadata_callback = [this, path, blob_name, mode] (size_t count) { diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index bcc89fbfc68..c34193574bb 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -1,4 +1,4 @@ - +#pragma once namespace DB { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ac1bfc620b0..a64ef3a88be 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #include #include #include @@ -3198,4 +3199,13 @@ ReadSettings Context::getReadSettings() const return res; } +WriteSettings Context::getWriteSettings() const +{ + WriteSettings res; + + res.remote_fs_cache_on_insert = settings.remote_fs_cache_on_insert; + + return res; +} + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c3615db9068..9043a690cff 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -116,6 +116,7 @@ struct PartUUIDs; using PartUUIDsPtr = std::shared_ptr; class KeeperDispatcher; class Session; +struct WriteSettings; class IInputFormat; class IOutputFormat; @@ -892,6 +893,9 @@ public: /** Get settings for reading from filesystem. */ ReadSettings getReadSettings() const; + /** Get settings for writing to filesystem. */ + WriteSettings getWriteSettings() const; + private: std::unique_lock getLock() const; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 4805a273c70..2fb075a54bb 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -452,7 +452,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( temp_part.streams.emplace_back(std::move(stream)); } } - auto finalizer = out->finalizePartAsync(new_data_part, data_settings->fsync_after_insert); + auto finalizer = out->finalizePartAsync( + new_data_part, + data_settings->fsync_after_insert, + nullptr, nullptr, + context->getWriteSettings()); temp_part.part = new_data_part; temp_part.streams.emplace_back(TemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)}); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 5bd9217226d..be6bb453ad5 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -176,14 +176,14 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::DataPartPtr & new_part, MergeTreeData::DataPart::Checksums & checksums, - const WriteSettings & write_settings) + const WriteSettings & settings) { WrittenFiles written_files; if (new_part->isProjectionPart()) { if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { - auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096); + auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096, WriteMode::Rewrite, settings); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); count_out_hashing.next(); @@ -197,7 +197,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis { if (new_part->uuid != UUIDHelpers::Nil) { - auto out = volume->getDisk()->writeFile(fs::path(part_path) / IMergeTreeDataPart::UUID_FILE_NAME, 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / IMergeTreeDataPart::UUID_FILE_NAME, 4096, WriteMode::Rewrite, settings); HashingWriteBuffer out_hashing(*out); writeUUIDText(new_part->uuid, out_hashing); checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count(); @@ -223,7 +223,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis } { - auto count_out = volume->getDisk()->writeFile(fs::path(part_path) / "count.txt", 4096); + auto count_out = volume->getDisk()->writeFile(fs::path(part_path) / "count.txt", 4096, WriteMode::Rewrite, settings); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); count_out_hashing.next(); @@ -237,7 +237,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (!new_part->ttl_infos.empty()) { /// Write a file with ttl infos in json format. - auto out = volume->getDisk()->writeFile(fs::path(part_path) / "ttl.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "ttl.txt", 4096, WriteMode::Rewrite, settings); HashingWriteBuffer out_hashing(*out); new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -248,7 +248,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (!new_part->getSerializationInfos().empty()) { - auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); + auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096, WriteMode::Rewrite, settings); HashingWriteBuffer out_hashing(*out); new_part->getSerializationInfos().writeJSON(out_hashing); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); @@ -259,7 +259,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis { /// Write a file with a description of columns. - auto out = volume->getDisk()->writeFile(fs::path(part_path) / "columns.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "columns.txt", 4096, WriteMode::Rewrite, settings); new_part->getColumns().writeText(*out); out->preFinalize(); written_files.emplace_back(std::move(out)); @@ -267,7 +267,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (default_codec != nullptr) { - auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); + auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096, WriteMode::Rewrite, settings); DB::writeText(queryToString(default_codec->getFullCodecDesc()), *out); out->preFinalize(); written_files.emplace_back(std::move(out)); @@ -280,7 +280,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis { /// Write file with checksums. - auto out = volume->getDisk()->writeFile(fs::path(part_path) / "checksums.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "checksums.txt", 4096, WriteMode::Rewrite, settings); checksums.write(*out); out->preFinalize(); written_files.emplace_back(std::move(out)); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 05f70239517..6c26e918673 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -56,7 +56,7 @@ public: bool sync, const NamesAndTypesList * total_columns_list = nullptr, MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr, - const WriteSettings & write_settings = {}); + const WriteSettings & settings = {}); void finalizePart( MergeTreeData::MutableDataPartPtr & new_part, From eabbce69a72f515f0b769c05d7eb1d78e8448dd1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Mar 2022 12:30:25 +0100 Subject: [PATCH 092/372] Add system drop remote filesystem cache query, allow to write cache on insert --- src/Common/FileCache.cpp | 77 ++++++++++++++++--- src/Common/FileCache.h | 18 ++++- src/Common/FileCacheFactory.cpp | 16 ++++ src/Common/FileCacheFactory.h | 8 +- src/Common/FileCacheSettings.cpp | 4 +- src/Common/FileCacheSettings.h | 4 +- src/Common/FileSegment.cpp | 32 +++++++- src/Core/Settings.h | 1 + src/Disks/IDiskRemote.cpp | 6 ++ src/Disks/IDiskRemote.h | 3 + src/Disks/RemoteDisksCommon.cpp | 6 +- src/IO/WriteBufferFromS3.cpp | 52 +++++++++---- src/IO/WriteBufferFromS3.h | 5 +- src/Interpreters/InterpreterSystemQuery.cpp | 18 +++++ src/Parsers/ASTSystemQuery.h | 2 + .../MergeTreeDataPartWriterCompact.cpp | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MergeTreeIOSettings.h | 4 + .../MergeTree/MergedBlockOutputStream.cpp | 5 +- .../MergeTree/MergedBlockOutputStream.h | 3 +- .../MergedColumnOnlyOutputStream.cpp | 2 + src/Storages/System/StorageSystemDisks.cpp | 9 +++ .../StorageSystemRemoteFilesystemCache.cpp | 49 ++++++++++++ .../StorageSystemRemoteFilesystemCache.h | 24 ++++++ src/Storages/System/attachSystemTables.cpp | 2 + ...0_system_remote_filesystem_cache.reference | 18 +++++ .../02240_system_remote_filesystem_cache.sql | 17 ++++ ...emote_filesystem_cache_on_insert.reference | 16 ++++ ...2241_remote_filesystem_cache_on_insert.sql | 16 ++++ 29 files changed, 381 insertions(+), 44 deletions(-) create mode 100644 src/Storages/System/StorageSystemRemoteFilesystemCache.cpp create mode 100644 src/Storages/System/StorageSystemRemoteFilesystemCache.h create mode 100644 tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference create mode 100644 tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql create mode 100644 tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference create mode 100644 tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 5cc2e707667..ba39039c378 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int REMOTE_FS_OBJECT_CACHE_ERROR; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace @@ -34,9 +35,9 @@ IFileCache::IFileCache( const String & cache_base_path_, const FileCacheSettings & cache_settings_) : cache_base_path(cache_base_path_) - , max_size(cache_settings_.max_cache_size) - , max_element_size(cache_settings_.max_cache_elements) - , max_file_segment_size(cache_settings_.max_cache_elements) + , max_size(cache_settings_.max_size) + , max_element_size(cache_settings_.max_elements) + , max_file_segment_size(cache_settings_.max_file_segment_size) { } @@ -204,8 +205,8 @@ FileSegments LRUFileCache::getImpl( return result; } -FileSegments LRUFileCache::splitRangeIntoEmptyCells( - const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock) +FileSegments LRUFileCache::splitRangeIntoCells( + const Key & key, size_t offset, size_t size, FileSegment::State state, std::lock_guard & cache_lock) { assert(size > 0); @@ -221,9 +222,10 @@ FileSegments LRUFileCache::splitRangeIntoEmptyCells( current_cell_size = std::min(remaining_size, max_file_segment_size); remaining_size -= current_cell_size; - auto * cell = addCell(key, current_pos, current_cell_size, FileSegment::State::EMPTY, cache_lock); + auto * cell = addCell(key, current_pos, current_cell_size, state, cache_lock); if (cell) file_segments.push_back(cell->file_segment); + assert(cell); current_pos += current_cell_size; } @@ -245,7 +247,7 @@ FileSegmentsHolder LRUFileCache::getOrSet(const Key & key, size_t offset, size_t if (file_segments.empty()) { - file_segments = splitRangeIntoEmptyCells(key, offset, size, cache_lock); + file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, cache_lock); } else { @@ -290,7 +292,7 @@ FileSegmentsHolder LRUFileCache::getOrSet(const Key & key, size_t offset, size_t assert(current_pos < segment_range.left); auto hole_size = segment_range.left - current_pos; - file_segments.splice(it, splitRangeIntoEmptyCells(key, current_pos, hole_size, cache_lock)); + file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); current_pos = segment_range.right + 1; ++it; @@ -304,7 +306,7 @@ FileSegmentsHolder LRUFileCache::getOrSet(const Key & key, size_t offset, size_t /// segmentN auto hole_size = range.right - current_pos + 1; - file_segments.splice(file_segments.end(), splitRangeIntoEmptyCells(key, current_pos, hole_size, cache_lock)); + file_segments.splice(file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); } } @@ -345,6 +347,22 @@ LRUFileCache::FileSegmentCell * LRUFileCache::addCell( return &(it->second); } +FileSegmentsHolder LRUFileCache::setDownloading(const Key & key, size_t offset, size_t size) +{ + std::lock_guard cache_lock(mutex); + + auto * cell = getCell(key, offset, cache_lock); + if (cell) + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Cache cell already exists for key `{}` and offset {}", + keyToStr(key), offset); + + auto file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::DOWNLOADING, cache_lock); + + return FileSegmentsHolder(std::move(file_segments)); +} + bool LRUFileCache::tryReserve( const Key & key_, size_t offset_, size_t size, std::lock_guard & cache_lock) { @@ -475,6 +493,30 @@ void LRUFileCache::remove(const Key & key) fs::remove(key_path); } +void LRUFileCache::tryRemoveAll() +{ + /// Try remove all cached files by cache_base_path. + /// Only releasable file segments are evicted. + + std::lock_guard cache_lock(mutex); + + for (auto it = queue.begin(); it != queue.end();) + { + auto & [key, offset] = *it++; + + auto * cell = getCell(key, offset, cache_lock); + if (cell->releasable()) + { + auto file_segment = cell->file_segment; + if (file_segment) + { + std::lock_guard segment_lock(file_segment->mutex); + remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); + } + } + } +} + void LRUFileCache::remove( Key key, size_t offset, std::lock_guard & cache_lock, std::lock_guard & /* segment_lock */) @@ -659,6 +701,20 @@ bool LRUFileCache::isLastFileSegmentHolder( return cell->file_segment.use_count() == 2; } +FileSegmentsHolder LRUFileCache::getAll() +{ + std::lock_guard cache_lock(mutex); + + FileSegments file_segments; + for (const auto & [key, offset] : queue) + { + auto * cell = getCell(key, offset, cache_lock); + file_segments.push_back(cell->file_segment); + } + + return FileSegmentsHolder(std::move(file_segments)); +} + LRUFileCache::FileSegmentCell::FileSegmentCell(FileSegmentPtr file_segment_, LRUQueue & queue_) : file_segment(file_segment_) { @@ -676,12 +732,13 @@ LRUFileCache::FileSegmentCell::FileSegmentCell(FileSegmentPtr file_segment_, LRU break; } case FileSegment::State::EMPTY: + case FileSegment::State::DOWNLOADING: { break; } default: throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Can create cell with either DOWNLOADED or EMPTY state, got: {}", + "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state, got: {}", FileSegment::stateToString(file_segment->download_state)); } } diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index dd585369853..30bd3679d71 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -41,6 +41,8 @@ public: virtual void remove(const Key & key) = 0; + virtual void tryRemoveAll() = 0; + static bool shouldBypassCache(); /// Cache capacity in bytes. @@ -52,6 +54,8 @@ public: String getPathInLocalCache(const Key & key); + const String & getBasePath() const { return cache_base_path; } + /** * Given an `offset` and `size` representing [offset, offset + size) bytes interval, * return list of cached non-overlapping non-empty @@ -65,6 +69,10 @@ public: */ virtual FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) = 0; + virtual FileSegmentsHolder getAll() = 0; + + virtual FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) = 0; + /// For debug. virtual String dumpStructure(const Key & key) = 0; @@ -113,10 +121,16 @@ public: FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override; + FileSegmentsHolder getAll() override; + + FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) override; + void initialize() override; void remove(const Key & key) override; + void tryRemoveAll() override; + private: using FileKeyAndOffset = std::pair; using LRUQueue = std::list; @@ -189,8 +203,8 @@ private: void loadCacheInfoIntoMemory(); - FileSegments splitRangeIntoEmptyCells( - const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock); + FileSegments splitRangeIntoCells( + const Key & key, size_t offset, size_t size, FileSegment::State state, std::lock_guard & cache_lock); public: struct Stat diff --git a/src/Common/FileCacheFactory.cpp b/src/Common/FileCacheFactory.cpp index 567d091fdeb..d5a63153e83 100644 --- a/src/Common/FileCacheFactory.cpp +++ b/src/Common/FileCacheFactory.cpp @@ -15,6 +15,12 @@ FileCacheFactory & FileCacheFactory::instance() return ret; } +FileCacheFactory::CacheByBasePath FileCacheFactory::getAll() +{ + std::lock_guard lock(mutex); + return caches; +} + FileCachePtr FileCacheFactory::getImpl(const std::string & cache_base_path, std::lock_guard &) { auto it = caches.find(cache_base_path); @@ -23,6 +29,16 @@ FileCachePtr FileCacheFactory::getImpl(const std::string & cache_base_path, std: return it->second; } +FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) +{ + std::lock_guard lock(mutex); + auto cache = getImpl(cache_base_path, lock); + if (cache) + return cache; + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); +} + FileCachePtr FileCacheFactory::getOrCreate( const std::string & cache_base_path, const FileCacheSettings & file_cache_settings) { diff --git a/src/Common/FileCacheFactory.h b/src/Common/FileCacheFactory.h index 176b96a862e..7156d8e6f04 100644 --- a/src/Common/FileCacheFactory.h +++ b/src/Common/FileCacheFactory.h @@ -14,16 +14,22 @@ namespace DB */ class FileCacheFactory final : private boost::noncopyable { + using CacheByBasePath = std::unordered_map; + public: static FileCacheFactory & instance(); FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings); + FileCachePtr get(const std::string & cache_base_path); + + CacheByBasePath getAll(); + private: FileCachePtr getImpl(const std::string & cache_base_path, std::lock_guard &); std::mutex mutex; - std::unordered_map caches; + CacheByBasePath caches; }; } diff --git a/src/Common/FileCacheSettings.cpp b/src/Common/FileCacheSettings.cpp index 39a6a2ec723..02009d95550 100644 --- a/src/Common/FileCacheSettings.cpp +++ b/src/Common/FileCacheSettings.cpp @@ -7,8 +7,8 @@ namespace DB void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { - max_cache_size = config.getUInt64(config_prefix + ".data_cache_max_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_CACHE_SIZE); - max_cache_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS); + max_size = config.getUInt64(config_prefix + ".data_cache_max_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_CACHE_SIZE); + max_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS); max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE); cache_on_insert = config.getUInt64(config_prefix + ".cache_on_insert", false); } diff --git a/src/Common/FileCacheSettings.h b/src/Common/FileCacheSettings.h index 8aa2bfb9214..9025f35445c 100644 --- a/src/Common/FileCacheSettings.h +++ b/src/Common/FileCacheSettings.h @@ -7,8 +7,8 @@ namespace DB struct FileCacheSettings { - size_t max_cache_size = 0; - size_t max_cache_elements = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS; + size_t max_size = 0; + size_t max_elements = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS; size_t max_file_segment_size = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE; bool cache_on_insert = false; diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index ceb16249c66..0a83f96e49f 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -31,10 +31,34 @@ FileSegment::FileSegment( , log(&Poco::Logger::get("FileSegment")) #endif { - if (download_state == State::DOWNLOADED) - reserved_size = downloaded_size = size_; - else if (download_state != State::EMPTY) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either DOWNLOADED or EMPTY state"); + /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. + switch (download_state) + { + /// EMPTY is used when file segment is not in cache and + /// someone will _potentially_ want to download it (after calling getOrSetDownloader()). + case (State::EMPTY): + { + break; + } + /// DOWNLOADED is used either on inital cache metadata load into memory on server startup + /// or on reduceSizeToDownloaded() -- when file segment object is updated. + case (State::DOWNLOADED): + { + reserved_size = downloaded_size = size_; + break; + } + /// DOWNLOADING is used only for write-through caching (e.g. getOrSetDownloader() is not + /// needed, downloader is set on file segment creation). + case (State::DOWNLOADING): + { + downloader_id = getCallerId(); + break; + } + default: + { + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state"); + } + } } FileSegment::State FileSegment::state() const diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8d28696094b..662d2c612d2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -555,6 +555,7 @@ class IColumn; M(UInt64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ M(Bool, remote_fs_enable_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ M(UInt64, remote_fs_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ + M(Bool, remote_fs_cache_on_insert, false, "Write into cache on INSERT query", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index fa4189abc53..6d4350dcfcc 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -402,6 +402,12 @@ IDiskRemote::IDiskRemote( } +String IDiskRemote::getCachePath() const +{ + return cache ? cache->getBasePath() : ""; +} + + bool IDiskRemote::exists(const String & path) const { return metadata_disk->exists(path); diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 82e76b8f68d..54f1604b99e 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -66,6 +66,8 @@ public: const String & getPath() const final override { return metadata_disk->getPath(); } + String getCachePath() const; + /// Methods for working with metadata. For some operations (like hardlink /// creation) metadata can be updated concurrently from multiple threads /// (file actually rewritten on disk). So additional RW lock is required for @@ -163,6 +165,7 @@ protected: const String remote_fs_root_path; DiskPtr metadata_disk; + FileCachePtr cache; private: diff --git a/src/Disks/RemoteDisksCommon.cpp b/src/Disks/RemoteDisksCommon.cpp index 4805434e5ee..da6ffed5f11 100644 --- a/src/Disks/RemoteDisksCommon.cpp +++ b/src/Disks/RemoteDisksCommon.cpp @@ -78,9 +78,9 @@ FileCachePtr getCachePtrForDisk( auto * log = &Poco::Logger::get("Disk(" + name + ")"); LOG_INFO(log, "Disk registered with cache path: {}. Cache size: {}, max cache elements size: {}, max_file_segment_size: {}", cache_base_path, - max_cache_size ? toString(max_cache_size) : "UNLIMITED", - max_cache_elements ? toString(max_cache_elements) : "UNLIMITED", - max_file_segment_size); + file_cache_settings.max_size ? toString(file_cache_settings.max_size) : "UNLIMITED", + file_cache_settings.max_elements ? toString(file_cache_settings.max_elements) : "UNLIMITED", + file_cache_settings.max_file_segment_size); return cache; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 4f1016d43bd..bd8cf1baff1 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -20,9 +20,9 @@ namespace ProfileEvents { extern const Event S3WriteBytes; + extern const Event RemoteFSCacheDownloadBytes; } - namespace DB { // S3 protocol does not allow to have multipart upload with more than 10000 parts. @@ -87,7 +87,41 @@ void WriteBufferFromS3::nextImpl() if (temporary_buffer->tellp() == -1) allocateBuffer(); - temporary_buffer->write(working_buffer.begin(), offset()); + size_t size = offset(); + temporary_buffer->write(working_buffer.begin(), size); + + if (cacheEnabled()) + { + std::cerr << "\n\n\n\n\n\n\nCache is enabled!\n\n\n\n\n"; + + /// Use max_single_part_upload_size as file segment size. Space reservation is incremental, + /// so this size does not really mean anything apart from the final file segment size limit. + /// If single part is uploaded with the smaller size, just resize file segment. + + // size_t max_file_segment_size = max_single_part_upload_size; + auto cache_key = cache->hash(key); + + auto file_segments_holder = cache->setDownloading(cache_key, current_download_offset, size); + assert(file_segments_holder.file_segments.back()->range().right - file_segments_holder.file_segments.begin()->range().left + 1 == size); + + size_t remaining_size = size; + for (const auto & file_segment : file_segments_holder.file_segments) + { + size_t current_size = std::min(file_segment->range().size(), remaining_size); + remaining_size -= current_size; + + if (file_segment->reserve(current_size)) + { + file_segment->write(working_buffer.begin(), current_size); + ProfileEvents::increment(ProfileEvents::RemoteFSCacheDownloadBytes, current_size); + } + else + { + /// TODO: add try catch, add complete() + break; + } + } + } ProfileEvents::increment(ProfileEvents::S3WriteBytes, offset()); @@ -129,19 +163,9 @@ WriteBufferFromS3::~WriteBufferFromS3() } } -void WriteBufferFromS3::tryWriteToCacheIfNeeded() +bool WriteBufferFromS3::cacheEnabled() const { - if (!cache || IFileCache::shouldBypassCache()) - return; - - try - { - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } + return cache && IFileCache::shouldBypassCache() == false; } void WriteBufferFromS3::preFinalize() diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 595a7a929c1..9773eedcce8 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -33,6 +33,7 @@ namespace DB { using ScheduleFunc = std::function)>; +class WriteBufferFromFile; /** * Buffer to write a data to a S3 object with specified bucket and key. @@ -86,7 +87,7 @@ private: void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); - void tryWriteToCacheIfNeeded(); + bool cacheEnabled() const; String bucket; String key; @@ -121,6 +122,8 @@ private: Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); FileCachePtr cache; + std::unique_ptr cache_writer; + size_t current_download_offset = 0; }; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f2ac2565a7f..c70205769b7 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -296,6 +298,21 @@ BlockIO InterpreterSystemQuery::execute() cache->reset(); break; #endif + case Type::DROP_REMOTE_FILESYSTEM_CACHE: + { + if (query.remote_filesystem_cache_path.empty()) + { + auto caches = FileCacheFactory::instance().getAll(); + for (const auto & [_, cache] : caches) + cache->tryRemoveAll(); + } + else + { + auto cache = FileCacheFactory::instance().get(query.remote_filesystem_cache_path); + cache->tryRemoveAll(); + } + break; + } case Type::RELOAD_DICTIONARY: { getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); @@ -758,6 +775,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_UNCOMPRESSED_CACHE: case Type::DROP_INDEX_MARK_CACHE: case Type::DROP_INDEX_UNCOMPRESSED_CACHE: + case Type::DROP_REMOTE_FILESYSTEM_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 22488e35e12..7113698789f 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -28,6 +28,7 @@ public: #if USE_EMBEDDED_COMPILER DROP_COMPILED_EXPRESSION_CACHE, #endif + DROP_REMOTE_FILESYSTEM_CACHE, STOP_LISTEN_QUERIES, START_LISTEN_QUERIES, RESTART_REPLICAS, @@ -88,6 +89,7 @@ public: String volume; String disk; UInt64 seconds{}; + String remote_filesystem_cache_path; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index d7b8f2c4165..6caff7c683f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -24,12 +24,14 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, - WriteMode::Rewrite)) + WriteMode::Rewrite, + settings_.query_write_settings)) , plain_hashing(*plain_file) , marks_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, 4096, - WriteMode::Rewrite)) + WriteMode::Rewrite, + settings_.query_write_settings)) , marks(*marks_file) { const auto & storage_columns = metadata_snapshot->getColumns(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2fb075a54bb..2803514cb9a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -437,7 +437,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( const auto & index_factory = MergeTreeIndexFactory::instance(); auto out = std::make_unique(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, false, false, context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index aaa8fae7dba..5841daafb8f 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -28,6 +29,7 @@ struct MergeTreeWriterSettings MergeTreeWriterSettings( const Settings & global_settings, + const WriteSettings & query_write_settings_, const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, bool rewrite_primary_key_, @@ -40,6 +42,7 @@ struct MergeTreeWriterSettings , can_use_adaptive_granularity(can_use_adaptive_granularity_) , rewrite_primary_key(rewrite_primary_key_) , blocks_are_granules_size(blocks_are_granules_size_) + , query_write_settings(query_write_settings_) { } @@ -48,6 +51,7 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity; bool rewrite_primary_key; bool blocks_are_granules_size; + WriteSettings query_write_settings; }; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index be6bb453ad5..b4440dffe75 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -19,13 +19,15 @@ MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, bool reset_columns_, - bool blocks_are_granules_size) + bool blocks_are_granules_size, + const WriteSettings & write_settings) : IMergedBlockOutputStream(data_part, metadata_snapshot_, columns_list_, reset_columns_) , columns_list(columns_list_) , default_codec(default_codec_) { MergeTreeWriterSettings writer_settings( storage.getContext()->getSettings(), + write_settings, storage.getSettings(), data_part->index_granularity_info.is_adaptive, /* rewrite_primary_key = */ true, @@ -125,6 +127,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( MergeTreeData::DataPart::Checksums * additional_column_checksums, const WriteSettings & write_settings) { + std::cerr << "\n\n\n\nCACHE ON INSERT: " << write_settings.remote_fs_cache_on_insert << "\n\n\n"; /// Finish write and get checksums. MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 6c26e918673..7beb9c65ca5 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -21,7 +21,8 @@ public: const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, bool reset_columns_ = false, - bool blocks_are_granules_size = false); + bool blocks_are_granules_size = false, + const WriteSettings & write_settings = {}); Block getHeader() const { return metadata_snapshot->getSampleBlock(); } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 5a706165000..005d8093bba 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -26,6 +27,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeWriterSettings writer_settings( global_settings, + WriteSettings{}, storage_settings, index_granularity_info ? index_granularity_info->is_adaptive : data_part->storage.canUseAdaptiveGranularity(), /* rewrite_primary_key = */false); diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 3841abc2f2d..ddebfe3d63e 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -22,6 +23,7 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) {"total_space", std::make_shared()}, {"keep_free_space", std::make_shared()}, {"type", std::make_shared()}, + {"cache_path", std::make_shared()}, })); setInMemoryMetadata(storage_metadata); } @@ -43,6 +45,7 @@ Pipe StorageSystemDisks::read( MutableColumnPtr col_total = ColumnUInt64::create(); MutableColumnPtr col_keep = ColumnUInt64::create(); MutableColumnPtr col_type = ColumnString::create(); + MutableColumnPtr col_cache_path = ColumnString::create(); for (const auto & [disk_name, disk_ptr] : context->getDisksMap()) { @@ -52,6 +55,12 @@ Pipe StorageSystemDisks::read( col_total->insert(disk_ptr->getTotalSpace()); col_keep->insert(disk_ptr->getKeepingFreeSpace()); col_type->insert(toString(disk_ptr->getType())); + + if (disk_ptr->isRemote()) + { + const auto * remote_disk = assert_cast(disk_ptr.get()); + col_cache_path->insert(remote_disk->getCachePath()); + } } Columns res_columns; diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp new file mode 100644 index 00000000000..69927564b27 --- /dev/null +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp @@ -0,0 +1,49 @@ +#include "StorageSystemRemoteFilesystemCache.h" +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemRemoteFilesystemCache::getNamesAndTypes() +{ + return { + {"cache_base_path", std::make_shared()}, + {"cache_path", std::make_shared()}, + {"file_segment_range", std::make_shared(DataTypes{std::make_shared(), std::make_shared()})}, + {"size", std::make_shared()}, + }; +} + +StorageSystemRemoteFilesystemCache::StorageSystemRemoteFilesystemCache(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) +{ +} + +void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + auto caches = FileCacheFactory::instance().getAll(); + + for (const auto & [cache_base_path, cache] : caches) + { + auto holder = cache->getAll(); + for (const auto & file_segment : holder.file_segments) + { + res_columns[0]->insert(cache_base_path); + res_columns[1]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset())); + + const auto & range = file_segment->range(); + res_columns[2]->insert(Tuple({range.left, range.right})); + res_columns[3]->insert(range.size()); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.h b/src/Storages/System/StorageSystemRemoteFilesystemCache.h new file mode 100644 index 00000000000..6f7053b9197 --- /dev/null +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageSystemRemoteFilesystemCache final : public shared_ptr_helper, + public IStorageSystemOneBlock +{ + friend struct shared_ptr_helper; +public: + std::string getName() const override { return "SystemRemoteFilesystemCache"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + StorageSystemRemoteFilesystemCache(const StorageID & table_id_); + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 023ced35a6b..21f2211a9d2 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -159,6 +160,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "replicated_fetches"); attach(context, system_database, "part_moves_between_shards"); attach(context, system_database, "asynchronous_inserts"); + attach(context, system_database, "remote_filesystem_cache"); if (has_zookeeper) attach(context, system_database, "zookeeper"); diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference new file mode 100644 index 00000000000..c938833f060 --- /dev/null +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -0,0 +1,18 @@ +-- Tags: no-parallel + +-- { echo } + +DROP TABLE IF EXISTS test; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +./disks/s3/data_cache/ (0,745) 746 +SYSTEM DROP REMOTE FILESYSTEM CACHE; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +./disks/s3/data_cache/ (0,745) 746 +SYSTEM DROP REMOTE FILESYSTEM CACHE; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql new file mode 100644 index 00000000000..59f487c8350 --- /dev/null +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -0,0 +1,17 @@ +-- Tags: no-parallel, no-fasttest + +-- { echo } + +DROP TABLE IF EXISTS test; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); + +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference new file mode 100644 index 00000000000..5e72207ae55 --- /dev/null +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -0,0 +1,16 @@ +-- { echo } + +DROP TABLE IF EXISTS test; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +-- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +./disks/s3/data_cache/ (0,745) 746 +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +./disks/s3/data_cache/ (0,745) 746 +./disks/s3/data_cache/ (0,745) 746 +SYSTEM DROP REMOTE FILESYSTEM CACHE; +INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql new file mode 100644 index 00000000000..1151d097ecf --- /dev/null +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -0,0 +1,16 @@ +-- Tags: no-parallel, no-fasttest, long + +-- { echo } + +DROP TABLE IF EXISTS test; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +-- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; + +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT * FROM test FORMAT Null; +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SYSTEM DROP REMOTE FILESYSTEM CACHE; +INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); +SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; From fe67d86869a67027e4ea286d15540a0bdb4b39a0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 21 Mar 2022 20:08:56 +0800 Subject: [PATCH 093/372] update contrib/arrow --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index ce6b7af516c..efdcd015cfd 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit ce6b7af516cff9b106e0f7b1c30628f18e7a6169 +Subproject commit efdcd015cfdee1b6aa349c9ca227ca12c3d697f5 From afd0c64a1ad43589dfe53e567de1c31702526112 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Mar 2022 14:56:38 +0100 Subject: [PATCH 094/372] Better --- src/Common/FileCache.cpp | 2 +- src/Common/FileCacheFactory.cpp | 31 +++++++++++++------ src/Common/FileCacheFactory.h | 15 +++++++-- src/Common/FileCacheSettings.h | 2 ++ src/Disks/S3/DiskS3.cpp | 5 ++- src/IO/WriteBufferFromS3.cpp | 8 ----- src/Interpreters/InterpreterSystemQuery.cpp | 4 +-- .../MergeTreeDataPartWriterOnDisk.cpp | 11 ++++--- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 3 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 3 +- .../MergeTree/MergedBlockOutputStream.cpp | 1 - .../StorageSystemRemoteFilesystemCache.cpp | 4 ++- ...0_system_remote_filesystem_cache.reference | 2 +- .../02240_system_remote_filesystem_cache.sql | 2 +- ...emote_filesystem_cache_on_insert.reference | 18 ++++++----- ...2241_remote_filesystem_cache_on_insert.sql | 13 ++++---- 16 files changed, 75 insertions(+), 49 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index ba39039c378..cff70fe0fb4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -359,7 +359,6 @@ FileSegmentsHolder LRUFileCache::setDownloading(const Key & key, size_t offset, keyToStr(key), offset); auto file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::DOWNLOADING, cache_lock); - return FileSegmentsHolder(std::move(file_segments)); } @@ -381,6 +380,7 @@ bool LRUFileCache::tryReserve( auto is_overflow = [&] { + /// max_size == 0 means unlimited cache size, max_element_size means unlimited number of cache elements. return (max_size != 0 && current_size + size - removed_size > max_size) || (max_element_size != 0 && queue_size > max_element_size); }; diff --git a/src/Common/FileCacheFactory.cpp b/src/Common/FileCacheFactory.cpp index d5a63153e83..683676041d2 100644 --- a/src/Common/FileCacheFactory.cpp +++ b/src/Common/FileCacheFactory.cpp @@ -21,20 +21,31 @@ FileCacheFactory::CacheByBasePath FileCacheFactory::getAll() return caches; } -FileCachePtr FileCacheFactory::getImpl(const std::string & cache_base_path, std::lock_guard &) +const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cache_base_path) +{ + std::lock_guard lock(mutex); + auto * cache_data = getImpl(cache_base_path, lock); + + if (cache_data) + return cache_data->settings; + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); +} + +FileCacheFactory::CacheData * FileCacheFactory::getImpl(const std::string & cache_base_path, std::lock_guard &) { auto it = caches.find(cache_base_path); if (it == caches.end()) return nullptr; - return it->second; + return &it->second; } FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) { std::lock_guard lock(mutex); - auto cache = getImpl(cache_base_path, lock); - if (cache) - return cache; + auto * cache_data = getImpl(cache_base_path, lock); + if (cache_data) + return cache_data->cache; throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); } @@ -43,12 +54,12 @@ FileCachePtr FileCacheFactory::getOrCreate( const std::string & cache_base_path, const FileCacheSettings & file_cache_settings) { std::lock_guard lock(mutex); - auto cache = getImpl(cache_base_path, lock); - if (cache) - return cache; + auto * cache_data = getImpl(cache_base_path, lock); + if (cache_data) + return cache_data->cache; - cache = std::make_shared(cache_base_path, file_cache_settings); - caches.emplace(cache_base_path, cache); + auto cache = std::make_shared(cache_base_path, file_cache_settings); + caches.emplace(cache_base_path, CacheData(cache, file_cache_settings)); return cache; } diff --git a/src/Common/FileCacheFactory.h b/src/Common/FileCacheFactory.h index 7156d8e6f04..3518f487b6d 100644 --- a/src/Common/FileCacheFactory.h +++ b/src/Common/FileCacheFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -14,7 +15,15 @@ namespace DB */ class FileCacheFactory final : private boost::noncopyable { - using CacheByBasePath = std::unordered_map; + struct CacheData + { + FileCachePtr cache; + FileCacheSettings settings; + + CacheData(FileCachePtr cache_, const FileCacheSettings & settings_) : cache(cache_), settings(settings_) {} + }; + + using CacheByBasePath = std::unordered_map; public: static FileCacheFactory & instance(); @@ -25,8 +34,10 @@ public: CacheByBasePath getAll(); + const FileCacheSettings & getSettings(const std::string & cache_base_path); + private: - FileCachePtr getImpl(const std::string & cache_base_path, std::lock_guard &); + CacheData * getImpl(const std::string & cache_base_path, std::lock_guard &); std::mutex mutex; CacheByBasePath caches; diff --git a/src/Common/FileCacheSettings.h b/src/Common/FileCacheSettings.h index 9025f35445c..c7956e48282 100644 --- a/src/Common/FileCacheSettings.h +++ b/src/Common/FileCacheSettings.h @@ -1,3 +1,5 @@ +#pragma once + #include namespace Poco { namespace Util { class AbstractConfiguration; }} diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 797b7ed15f4..eb74ef614b5 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -290,6 +291,8 @@ std::unique_ptr DiskS3::writeFile(const String & path, }); }; + bool cache_on_insert = write_settings.remote_fs_cache_on_insert || FileCacheFactory::instance().getSettings(getCachePath()).cache_on_insert; + auto s3_buffer = std::make_unique( settings->client, bucket, @@ -299,7 +302,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, settings->s3_upload_part_size_multiply_parts_count_threshold, settings->s3_max_single_part_upload_size, std::move(object_metadata), - buf_size, std::move(schedule), write_settings.remote_fs_cache_on_insert ? cache : nullptr); + buf_size, std::move(schedule), cache_on_insert ? cache : nullptr); auto create_metadata_callback = [this, path, blob_name, mode] (size_t count) { diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index bd8cf1baff1..20cb689fbd7 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -92,13 +92,6 @@ void WriteBufferFromS3::nextImpl() if (cacheEnabled()) { - std::cerr << "\n\n\n\n\n\n\nCache is enabled!\n\n\n\n\n"; - - /// Use max_single_part_upload_size as file segment size. Space reservation is incremental, - /// so this size does not really mean anything apart from the final file segment size limit. - /// If single part is uploaded with the smaller size, just resize file segment. - - // size_t max_file_segment_size = max_single_part_upload_size; auto cache_key = cache->hash(key); auto file_segments_holder = cache->setDownloading(cache_key, current_download_offset, size); @@ -117,7 +110,6 @@ void WriteBufferFromS3::nextImpl() } else { - /// TODO: add try catch, add complete() break; } } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c70205769b7..5f030159b2e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -303,8 +303,8 @@ BlockIO InterpreterSystemQuery::execute() if (query.remote_filesystem_cache_path.empty()) { auto caches = FileCacheFactory::instance().getAll(); - for (const auto & [_, cache] : caches) - cache->tryRemoveAll(); + for (const auto & [_, cache_data] : caches) + cache_data.cache->tryRemoveAll(); } else { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 8dca93f574f..6cba4db19e3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -47,15 +47,16 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( const std::string & marks_path_, const std::string & marks_file_extension_, const CompressionCodecPtr & compression_codec_, - size_t max_compress_block_size_) : + size_t max_compress_block_size_, + const WriteSettings & query_write_settings) : escaped_column_name(escaped_column_name_), data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, - plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite)), + plain_file(disk_->writeFile(data_path_ + data_file_extension, max_compress_block_size_, WriteMode::Rewrite, query_write_settings)), plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_codec_, max_compress_block_size_), compressed(compressed_buf), - marks_file(disk_->writeFile(marks_path_ + marks_file_extension, 4096, WriteMode::Rewrite)), marks(*marks_file) + marks_file(disk_->writeFile(marks_path_ + marks_file_extension, 4096, WriteMode::Rewrite, query_write_settings)), marks(*marks_file) { } @@ -156,7 +157,7 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() { if (metadata_snapshot->hasPrimaryKey()) { - index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings.query_write_settings); index_stream = std::make_unique(*index_file_stream); } } @@ -172,7 +173,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() data_part->volume->getDisk(), part_path + stream_name, index_helper->getSerializedFileExtension(), part_path + stream_name, marks_file_extension, - default_codec, settings.max_compress_block_size)); + default_codec, settings.max_compress_block_size, settings.query_write_settings)); skip_indices_aggregators.push_back(index_helper->createIndexAggregator()); skip_index_accumulated_marks.push_back(0); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 5af8cbc1650..67b51df7d56 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -55,7 +55,8 @@ public: const std::string & marks_path_, const std::string & marks_file_extension_, const CompressionCodecPtr & compression_codec_, - size_t max_compress_block_size_); + size_t max_compress_block_size_, + const WriteSettings & query_write_settings); String escaped_column_name; std::string data_file_extension; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a3eec3e54bc..933814d27ba 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -115,7 +115,8 @@ void MergeTreeDataPartWriterWide::addStreams( part_path + stream_name, DATA_FILE_EXTENSION, part_path + stream_name, marks_file_extension, compression_codec, - settings.max_compress_block_size); + settings.max_compress_block_size, + settings.query_write_settings); }; ISerialization::SubstreamPath path; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index b4440dffe75..4fce24fae74 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -127,7 +127,6 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( MergeTreeData::DataPart::Checksums * additional_column_checksums, const WriteSettings & write_settings) { - std::cerr << "\n\n\n\nCACHE ON INSERT: " << write_settings.remote_fs_cache_on_insert << "\n\n\n"; /// Finish write and get checksums. MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp index 69927564b27..23aaffd3a4c 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp @@ -31,9 +31,11 @@ void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, { auto caches = FileCacheFactory::instance().getAll(); - for (const auto & [cache_base_path, cache] : caches) + for (const auto & [cache_base_path, cache_data] : caches) { + auto & cache = cache_data.cache; auto holder = cache->getAll(); + for (const auto & file_segment : holder.file_segments) { res_columns[0]->insert(cache_base_path); diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference index c938833f060..db07f1dd577 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -2,10 +2,10 @@ -- { echo } -DROP TABLE IF EXISTS test; SYSTEM DROP REMOTE FILESYSTEM CACHE; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); +DROP TABLE IF EXISTS test; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; ./disks/s3/data_cache/ (0,745) 746 diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index 59f487c8350..fb6dd8d61b4 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -2,10 +2,10 @@ -- { echo } -DROP TABLE IF EXISTS test; SYSTEM DROP REMOTE FILESYSTEM CACHE; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); +DROP TABLE IF EXISTS test; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 5e72207ae55..3c3e31d215a 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -2,15 +2,17 @@ DROP TABLE IF EXISTS test; SYSTEM DROP REMOTE FILESYSTEM CACHE; --- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; -./disks/s3/data_cache/ (0,745) 746 +SELECT count() FROM system.remote_filesystem_cache; +8 SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; -./disks/s3/data_cache/ (0,745) 746 -./disks/s3/data_cache/ (0,745) 746 +SELECT count() size FROM system.remote_filesystem_cache; +9 SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.remote_filesystem_cache; +7 +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; +SELECT count() size FROM system.remote_filesystem_cache; +14 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 1151d097ecf..48ae85c8e61 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -1,16 +1,17 @@ --- Tags: no-parallel, no-fasttest, long +-- Tags: no-parallel, no-fasttest -- { echo } DROP TABLE IF EXISTS test; SYSTEM DROP REMOTE FILESYSTEM CACHE; --- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT count() FROM system.remote_filesystem_cache; SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.remote_filesystem_cache; SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.remote_filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used +SELECT count() size FROM system.remote_filesystem_cache; From a9ba14e38606ca7fb7ac2109983481fe2346d9c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Mar 2022 21:20:15 +0100 Subject: [PATCH 095/372] Fix checks --- src/Common/FileCache.cpp | 1 - src/Common/FileSegment.cpp | 2 +- src/Storages/System/StorageSystemDisks.cpp | 5 ++++- .../0_stateless/02117_show_create_table_system.reference | 2 +- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index cff70fe0fb4..f3db4a2de88 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -20,7 +20,6 @@ namespace ErrorCodes { extern const int REMOTE_FS_OBJECT_CACHE_ERROR; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } namespace diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 0a83f96e49f..f13cbc6a464 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -40,7 +40,7 @@ FileSegment::FileSegment( { break; } - /// DOWNLOADED is used either on inital cache metadata load into memory on server startup + /// DOWNLOADED is used either on initial cache metadata load into memory on server startup /// or on reduceSizeToDownloaded() -- when file segment object is updated. case (State::DOWNLOADED): { diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index ddebfe3d63e..f1896b5e49c 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -56,11 +56,13 @@ Pipe StorageSystemDisks::read( col_keep->insert(disk_ptr->getKeepingFreeSpace()); col_type->insert(toString(disk_ptr->getType())); + String cache_path; if (disk_ptr->isRemote()) { const auto * remote_disk = assert_cast(disk_ptr.get()); - col_cache_path->insert(remote_disk->getCachePath()); + cache_path = remote_disk->getCachePath(); } + col_cache_path->insert(cache_path); } Columns res_columns; @@ -70,6 +72,7 @@ Pipe StorageSystemDisks::read( res_columns.emplace_back(std::move(col_total)); res_columns.emplace_back(std::move(col_keep)); res_columns.emplace_back(std::move(col_type)); + res_columns.emplace_back(std::move(col_cache_path)); UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index cecdd0498b1..432dcd2068d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -12,7 +12,7 @@ CREATE TABLE system.data_type_families\n(\n `name` String,\n `case_insensi CREATE TABLE system.databases\n(\n `name` String,\n `engine` String,\n `data_path` String,\n `metadata_path` String,\n `uuid` UUID,\n `comment` String,\n `database` String\n)\nENGINE = SystemDatabases()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.detached_parts\n(\n `database` String,\n `table` String,\n `partition_id` Nullable(String),\n `name` String,\n `disk` String,\n `reason` Nullable(String),\n `min_block_number` Nullable(Int64),\n `max_block_number` Nullable(Int64),\n `level` Nullable(UInt32)\n)\nENGINE = SystemDetachedParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.dictionaries\n(\n `database` String,\n `name` String,\n `uuid` UUID,\n `status` Enum8(\'NOT_LOADED\' = 0, \'LOADED\' = 1, \'FAILED\' = 2, \'LOADING\' = 3, \'FAILED_AND_RELOADING\' = 4, \'LOADED_AND_RELOADING\' = 5, \'NOT_EXIST\' = 6),\n `origin` String,\n `type` String,\n `key.names` Array(String),\n `key.types` Array(String),\n `attribute.names` Array(String),\n `attribute.types` Array(String),\n `bytes_allocated` UInt64,\n `query_count` UInt64,\n `hit_rate` Float64,\n `found_rate` Float64,\n `element_count` UInt64,\n `load_factor` Float64,\n `source` String,\n `lifetime_min` UInt64,\n `lifetime_max` UInt64,\n `loading_start_time` DateTime,\n `last_successful_update_time` DateTime,\n `loading_duration` Float32,\n `last_exception` String,\n `comment` String\n)\nENGINE = SystemDictionaries()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.disks\n(\n `name` String,\n `path` String,\n `free_space` UInt64,\n `total_space` UInt64,\n `keep_free_space` UInt64,\n `type` String\n)\nENGINE = SystemDisks()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.disks\n(\n `name` String,\n `path` String,\n `free_space` UInt64,\n `total_space` UInt64,\n `keep_free_space` UInt64,\n `type` String,\n `cache_path` String\n)\nENGINE = SystemDisks()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.distributed_ddl_queue\n(\n `entry` String,\n `entry_version` Nullable(UInt8),\n `initiator_host` Nullable(String),\n `initiator_port` Nullable(UInt16),\n `cluster` String,\n `query` String,\n `settings` Map(String, String),\n `query_create_time` DateTime,\n `host` Nullable(String),\n `port` Nullable(UInt16),\n `status` Nullable(Enum8(\'Inactive\' = 0, \'Active\' = 1, \'Finished\' = 2, \'Removing\' = 3, \'Unknown\' = 4)),\n `exception_code` Nullable(UInt16),\n `exception_text` Nullable(String),\n `query_finish_time` Nullable(DateTime),\n `query_duration_ms` Nullable(UInt64)\n)\nENGINE = SystemDDLWorkerQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.distribution_queue\n(\n `database` String,\n `table` String,\n `data_path` String,\n `is_blocked` UInt8,\n `error_count` UInt64,\n `data_files` UInt64,\n `data_compressed_bytes` UInt64,\n `broken_data_files` UInt64,\n `broken_data_compressed_bytes` UInt64,\n `last_exception` String\n)\nENGINE = SystemDistributionQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.enabled_roles\n(\n `role_name` String,\n `with_admin_option` UInt8,\n `is_current` UInt8,\n `is_default` UInt8\n)\nENGINE = SystemEnabledRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From 68d5b538aa37d7dbdb002672fe459f38345015ab Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 23 Mar 2022 11:15:42 +0800 Subject: [PATCH 096/372] fix build error --- src/Storages/Hive/HiveFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index e18eef58947..2862e654ddc 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes #define THROW_ARROW_NOT_OK(status) \ do \ { \ - if (::arrow::Status _s = (status); !_s.ok()) \ + if (const ::arrow::Status & _s = (status); !_s.ok()) \ throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ } while (false) From f3e1ca44a9253d3ecda7689c230a0854e1eb56de Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 13:01:18 +0100 Subject: [PATCH 097/372] Add system.remote_data_paths table --- src/Common/FileCache.cpp | 21 +++- src/Common/FileCache.h | 4 + src/Disks/DiskDecorator.h | 3 + src/Disks/DiskLocal.cpp | 6 ++ src/Disks/DiskRestartProxy.cpp | 18 ++++ src/Disks/DiskRestartProxy.h | 3 + src/Disks/IDisk.h | 26 +++++ src/Disks/IDiskRemote.cpp | 31 +++++- src/Disks/IDiskRemote.h | 8 +- .../System/StorageSystemRemoteDataPaths.cpp | 100 ++++++++++++++++++ .../System/StorageSystemRemoteDataPaths.h | 30 ++++++ 11 files changed, 243 insertions(+), 7 deletions(-) create mode 100644 src/Storages/System/StorageSystemRemoteDataPaths.cpp create mode 100644 src/Storages/System/StorageSystemRemoteDataPaths.h diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index f3db4a2de88..07429a448f2 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -705,15 +705,30 @@ FileSegmentsHolder LRUFileCache::getAll() std::lock_guard cache_lock(mutex); FileSegments file_segments; - for (const auto & [key, offset] : queue) + + for (const auto & [key, cells_by_offset] : files) { - auto * cell = getCell(key, offset, cache_lock); - file_segments.push_back(cell->file_segment); + for (const auto & [offset, cell] : cells_by_offset) + file_segments.push_back(cell.file_segment); } return FileSegmentsHolder(std::move(file_segments)); } +std::vector LRUFileCache::tryGetCachePaths(const Key & key) +{ + std::lock_guard cache_lock(mutex); + + std::vector cache_paths; + const auto & cells_by_offset = files[key]; + + for (const auto & [offset, cell] : cells_by_offset) + if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) + cache_paths.push_back(getPathInLocalCache(key, offset)); + + return cache_paths; +} + LRUFileCache::FileSegmentCell::FileSegmentCell(FileSegmentPtr file_segment_, LRUQueue & queue_) : file_segment(file_segment_) { diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 30bd3679d71..e8280fba08a 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -56,6 +56,8 @@ public: const String & getBasePath() const { return cache_base_path; } + virtual std::vector tryGetCachePaths(const Key & key) = 0; + /** * Given an `offset` and `size` representing [offset, offset + size) bytes interval, * return list of cached non-overlapping non-empty @@ -131,6 +133,8 @@ public: void tryRemoveAll() override; + std::vector tryGetCachePaths(const Key & key) override; + private: using FileKeyAndOffset = std::pair; using LRUQueue = std::list; diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 33272ba385b..e5c9c7699bf 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -72,6 +72,9 @@ public: void shutdown() override; void startup() override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override; + String getCacheBasePath() const override { return delegate->getCacheBasePath(); } + std::vector getRemotePaths(const String & path) const override { return delegate->getRemotePaths(path); } + void getRemotePathsRecursive(const String & path, std::vector & paths_map) override { return delegate->getRemotePathsRecursive(path, paths_map); } DiskPtr getMetadataDiskIfExistsOrSelf() override { return delegate->getMetadataDiskIfExistsOrSelf(); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 78be223edee..874405cda53 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -318,10 +318,16 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { fs::path meta_path = fs::path(disk_path) / path; + std::cerr << "Disk local: " << meta_path << "\n"; if (!broken && fs::exists(meta_path) && fs::is_directory(meta_path)) + { return std::make_unique(disk_path, path); + } else + { + std::cerr << "\n\n\n iterating Fail\n\n"; return std::make_unique(); + } } void DiskLocal::moveFile(const String & from_path, const String & to_path) diff --git a/src/Disks/DiskRestartProxy.cpp b/src/Disks/DiskRestartProxy.cpp index a1c63d1e5a9..8045a0e8c72 100644 --- a/src/Disks/DiskRestartProxy.cpp +++ b/src/Disks/DiskRestartProxy.cpp @@ -305,6 +305,24 @@ bool DiskRestartProxy::checkUniqueId(const String & id) const return DiskDecorator::checkUniqueId(id); } +String DiskRestartProxy::getCacheBasePath() const +{ + ReadLock lock (mutex); + return DiskDecorator::getCacheBasePath(); +} + +std::vector DiskRestartProxy::getRemotePaths(const String & path) const +{ + ReadLock lock (mutex); + return DiskDecorator::getRemotePaths(path); +} + +void DiskRestartProxy::getRemotePathsRecursive(const String & path, std::vector & paths_map) +{ + ReadLock lock (mutex); + return DiskDecorator::getRemotePathsRecursive(path, paths_map); +} + void DiskRestartProxy::restart() { /// Speed up processing unhealthy requests. diff --git a/src/Disks/DiskRestartProxy.h b/src/Disks/DiskRestartProxy.h index 2a0d40bffb6..baa57386e68 100644 --- a/src/Disks/DiskRestartProxy.h +++ b/src/Disks/DiskRestartProxy.h @@ -63,6 +63,9 @@ public: void truncateFile(const String & path, size_t size) override; String getUniqueId(const String & path) const override; bool checkUniqueId(const String & id) const override; + String getCacheBasePath() const override; + std::vector getRemotePaths(const String & path) const override; + void getRemotePathsRecursive(const String & path, std::vector & paths_map) override; void restart(); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index ac48f4f4d89..0f6e31dd0f7 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -32,6 +32,11 @@ namespace Poco namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class IDiskDirectoryIterator; using DiskDirectoryIteratorPtr = std::unique_ptr; @@ -199,6 +204,27 @@ public: /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); } + + virtual String getCacheBasePath() const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getCacheBasePath() not implemented fro disk: {}`", getType()); + } + + /// Returnes a list of paths because for Log family engines + /// there might be multiple files in remote fs for single clickhouse file. + virtual std::vector getRemotePaths(const String &) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePaths() not implemented fro disk: {}`", getType()); + } + + /// For one local path there might be multiple remote paths in case of Log family engines. + using LocalPathWithRemotePaths = std::pair>; + + virtual void getRemotePathsRecursive(const String &, std::vector &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented fro disk: {}`", getType()); + } + struct RemoveRequest { String path; diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 6d4350dcfcc..0f430e69a5e 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -136,13 +136,15 @@ void IDiskRemote::Metadata::load() } catch (Exception & e) { + tryLogCurrentException(__PRETTY_FUNCTION__); + if (e.code() == ErrorCodes::UNKNOWN_FORMAT) throw; if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED) throw; - throw Exception("Failed to read metadata file", e, ErrorCodes::UNKNOWN_FORMAT); + throw Exception("Failed to read metadata file: " + metadata_file_path, e, ErrorCodes::UNKNOWN_FORMAT); } } @@ -341,6 +343,30 @@ void IDiskRemote::removeMetadataRecursive(const String & path, RemoteFSPathKeepe } } +std::vector IDiskRemote::getRemotePaths(const String & path) const +{ + auto metadata = readMetadata(path); + + std::vector remote_paths; + for (const auto & [remote_path, _] : metadata.remote_fs_objects) + remote_paths.push_back(remote_path); + + return remote_paths; +} + +void IDiskRemote::getRemotePathsRecursive(const String & path, std::vector & paths_map) +{ + if (metadata_disk->isFile(path)) + { + paths_map.emplace_back(path, getRemotePaths(path)); + } + else + { + for (auto it = iterateDirectory(path); it->isValid(); it->next()) + IDiskRemote::getRemotePathsRecursive(fs::path(path) / it->name(), paths_map); + } +} + DiskPtr DiskRemoteReservation::getDisk(size_t i) const { if (i != 0) @@ -348,7 +374,6 @@ DiskPtr DiskRemoteReservation::getDisk(size_t i) const return disk; } - void DiskRemoteReservation::update(UInt64 new_size) { std::lock_guard lock(disk->reservation_mutex); @@ -402,7 +427,7 @@ IDiskRemote::IDiskRemote( } -String IDiskRemote::getCachePath() const +String IDiskRemote::getCacheBasePath() const { return cache ? cache->getBasePath() : ""; } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 54f1604b99e..a1d6092a286 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -66,7 +66,13 @@ public: const String & getPath() const final override { return metadata_disk->getPath(); } - String getCachePath() const; + String getCacheBasePath() const final override; + + /// Returnes a list of paths because for Log family engines + /// there might be multiple files in remote fs for single clickhouse file. + std::vector getRemotePaths(const String & path) const final override; + + void getRemotePathsRecursive(const String & path, std::vector & paths_map) override; /// Methods for working with metadata. For some operations (like hardlink /// creation) metadata can be updated concurrently from multiple threads diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp new file mode 100644 index 00000000000..28ae20be8e1 --- /dev/null +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -0,0 +1,100 @@ +#include "StorageSystemRemoteDataPaths.h" +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & table_id_) + : IStorage(table_id_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( + { + {"disk_name", std::make_shared()}, + {"path", std::make_shared()}, + {"cache_dir", std::make_shared()}, + {"local_path", std::make_shared()}, + {"remote_path", std::make_shared()}, + {"cache_paths", std::make_shared(std::make_shared())}, + })); + setInMemoryMetadata(storage_metadata); +} + +Pipe StorageSystemRemoteDataPaths::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + storage_snapshot->check(column_names); + + MutableColumnPtr col_disk_name = ColumnString::create(); + MutableColumnPtr col_base_path = ColumnString::create(); + MutableColumnPtr col_cache_base_path = ColumnString::create(); + MutableColumnPtr col_local_path = ColumnString::create(); + MutableColumnPtr col_remote_path = ColumnString::create(); + MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); + + auto disks = context->getDisksMap(); + for (const auto & [disk_name, disk] : disks) + { + if (disk->isRemote()) + { + std::vector remote_paths_by_local_path; + disk->getRemotePathsRecursive("store", remote_paths_by_local_path); + + FileCachePtr cache; + auto cache_base_path = disk->getCacheBasePath(); + if (!cache_base_path.empty()) + cache = FileCacheFactory::instance().get(cache_base_path); + + for (const auto & [local_path, remote_paths] : remote_paths_by_local_path) + { + for (const auto & remote_path : remote_paths) + { + col_disk_name->insert(disk_name); + col_base_path->insert(disk->getPath()); + col_cache_base_path->insert(cache_base_path); + col_local_path->insert(local_path); + col_remote_path->insert(remote_path); + + if (cache) + { + auto cache_paths = cache->tryGetCachePaths(cache->hash(remote_path)); + col_cache_paths->insert(Array(cache_paths.begin(), cache_paths.end())); + } + else + { + col_cache_paths->insertDefault(); + } + } + } + } + } + + Columns res_columns; + res_columns.emplace_back(std::move(col_disk_name)); + res_columns.emplace_back(std::move(col_base_path)); + res_columns.emplace_back(std::move(col_cache_base_path)); + res_columns.emplace_back(std::move(col_local_path)); + res_columns.emplace_back(std::move(col_remote_path)); + res_columns.emplace_back(std::move(col_cache_paths)); + + UInt64 num_rows = res_columns.at(0)->size(); + Chunk chunk(std::move(res_columns), num_rows); + + return Pipe(std::make_shared(storage_snapshot->metadata->getSampleBlock(), std::move(chunk))); +} + +} diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.h b/src/Storages/System/StorageSystemRemoteDataPaths.h new file mode 100644 index 00000000000..0057dcf12f2 --- /dev/null +++ b/src/Storages/System/StorageSystemRemoteDataPaths.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageSystemRemoteDataPaths : public shared_ptr_helper, public IStorage +{ + friend struct shared_ptr_helper; +public: + std::string getName() const override { return "SystemRemoteDataPaths"; } + + bool isSystemStorage() const override { return true; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +protected: + explicit StorageSystemRemoteDataPaths(const StorageID & table_id_); +}; + +} From 9cc528b01f50e11e0095fe763b02d31659f306dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 23 Mar 2022 21:57:58 +0800 Subject: [PATCH 098/372] Update HiveFile.h --- src/Storages/Hive/HiveFile.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 7d38efe6acb..aef9d72755a 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -156,7 +156,8 @@ protected: NamesAndTypesList index_names_and_types; MinMaxIndexPtr minmax_idx; std::vector sub_minmax_idxes; - std::set skip_splits; // skip splits for this file after applying minmax index (if any) + /// Skip splits for this file after applying minmax index (if any) + std::set skip_splits; std::shared_ptr storage_settings; }; From 34c0690a69622ba8c3837cf58ca9b5987366ff56 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 15:35:15 +0100 Subject: [PATCH 099/372] Better --- src/Common/FileSegment.h | 5 ++ src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 12 ++++- src/Disks/S3/DiskS3.cpp | 8 ++- src/IO/WriteBufferFromS3.cpp | 7 ++- src/IO/WriteBufferFromS3.h | 3 +- src/Storages/System/StorageSystemDisks.cpp | 6 +-- .../StorageSystemRemoteFilesystemCache.cpp | 4 ++ .../StorageSystemRemoteFilesystemCache.h | 18 ++++++- src/Storages/System/attachSystemTables.cpp | 2 + ...emote_filesystem_cache_on_insert.reference | 51 ++++++++++++++----- ...2241_remote_filesystem_cache_on_insert.sql | 40 +++++++++++---- 11 files changed, 122 insertions(+), 34 deletions(-) diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index f0c6c5de9b6..66dd186d1a9 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -121,6 +121,10 @@ public: String getInfoForLog() const; + size_t hits() const { return hits_num; } + + void hit() { ++hits_num; } + private: size_t availableSize() const { return reserved_size - downloaded_size; } bool lastFileSegmentHolder() const; @@ -162,6 +166,7 @@ private: bool detached = false; std::atomic is_downloaded{false}; + std::atomic hits_num = 0; /// cache hits. }; struct FileSegmentsHolder : private boost::noncopyable diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 5cab2cb2995..d093e29b28e 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -365,9 +365,14 @@ bool CachedReadBufferFromRemoteFS::completeFileSegmentAndGetNext() if (current_file_segment_it == file_segments_holder->file_segments.end()) return false; - implementation_buffer = getImplementationBuffer(*current_file_segment_it); + file_segment = *current_file_segment_it; - LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString()); + implementation_buffer = getImplementationBuffer(file_segment); + + if (read_type == ReadType::CACHED) + file_segment->hit(); + + LOG_TEST(log, "New segment: {}", file_segment->range().toString()); return true; } @@ -583,6 +588,9 @@ bool CachedReadBufferFromRemoteFS::nextImplStep() else { implementation_buffer = getImplementationBuffer(*current_file_segment_it); + + if (read_type == ReadType::CACHED) + (*current_file_segment_it)->hit(); } assert(!internal_buffer.empty()); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index eb74ef614b5..18cbcc3d659 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -46,6 +46,8 @@ #include +#include +#include namespace DB { @@ -291,7 +293,9 @@ std::unique_ptr DiskS3::writeFile(const String & path, }); }; - bool cache_on_insert = write_settings.remote_fs_cache_on_insert || FileCacheFactory::instance().getSettings(getCachePath()).cache_on_insert; + bool cache_on_insert = fs::path(path).extension() != ".tmp" + && write_settings.remote_fs_cache_on_insert + && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_insert; auto s3_buffer = std::make_unique( settings->client, @@ -302,7 +306,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, settings->s3_upload_part_size_multiply_parts_count_threshold, settings->s3_max_single_part_upload_size, std::move(object_metadata), - buf_size, std::move(schedule), cache_on_insert ? cache : nullptr); + buf_size, std::move(schedule), blob_name, cache_on_insert ? cache : nullptr); auto create_metadata_callback = [this, path, blob_name, mode] (size_t count) { diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 20cb689fbd7..53118bbf867 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -62,6 +62,7 @@ WriteBufferFromS3::WriteBufferFromS3( std::optional> object_metadata_, size_t buffer_size_, ScheduleFunc schedule_, + const String & blob_name_, FileCachePtr cache_) : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) @@ -73,6 +74,7 @@ WriteBufferFromS3::WriteBufferFromS3( , upload_part_size_multiply_threshold(upload_part_size_multiply_threshold_) , max_single_part_upload_size(max_single_part_upload_size_) , schedule(std::move(schedule_)) + , blob_name(blob_name_) , cache(cache_) { allocateBuffer(); @@ -92,10 +94,11 @@ void WriteBufferFromS3::nextImpl() if (cacheEnabled()) { - auto cache_key = cache->hash(key); + if (blob_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty blob name"); + auto cache_key = cache->hash(blob_name); auto file_segments_holder = cache->setDownloading(cache_key, current_download_offset, size); - assert(file_segments_holder.file_segments.back()->range().right - file_segments_holder.file_segments.begin()->range().left + 1 == size); size_t remaining_size = size; for (const auto & file_segment : file_segments_holder.file_segments) diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 9773eedcce8..ecddd72b9e8 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -56,6 +56,7 @@ public: std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, ScheduleFunc schedule_ = {}, + const String & blob_name = "", FileCachePtr cache_ = nullptr); ~WriteBufferFromS3() override; @@ -121,8 +122,8 @@ private: Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); + const String blob_name; FileCachePtr cache; - std::unique_ptr cache_writer; size_t current_download_offset = 0; }; diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index f1896b5e49c..fb6a055c6e5 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -58,10 +58,8 @@ Pipe StorageSystemDisks::read( String cache_path; if (disk_ptr->isRemote()) - { - const auto * remote_disk = assert_cast(disk_ptr.get()); - cache_path = remote_disk->getCachePath(); - } + cache_path = disk_ptr->getCacheBasePath(); + col_cache_path->insert(cache_path); } diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp index 23aaffd3a4c..e37dcb74829 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp @@ -19,6 +19,8 @@ NamesAndTypesList StorageSystemRemoteFilesystemCache::getNamesAndTypes() {"cache_path", std::make_shared()}, {"file_segment_range", std::make_shared(DataTypes{std::make_shared(), std::make_shared()})}, {"size", std::make_shared()}, + {"state", std::make_shared()}, + {"cache_hits", std::make_shared()}, }; } @@ -44,6 +46,8 @@ void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, const auto & range = file_segment->range(); res_columns[2]->insert(Tuple({range.left, range.right})); res_columns[3]->insert(range.size()); + res_columns[4]->insert(FileSegment::stateToString(file_segment->state())); + res_columns[5]->insert(file_segment->hits()); } } } diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.h b/src/Storages/System/StorageSystemRemoteFilesystemCache.h index 6f7053b9197..4ac68671823 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.h +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.h @@ -6,6 +6,22 @@ namespace DB { +/** + * SELECT + * cache_path, + * local_path, + * remote_path + * FROM + * ( + * SELECT + * arrayJoin(cache_paths) AS cache_path, + * local_path, + * remote_path + * FROM system.remote_data_paths + * ) AS data_paths + * INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path + */ + class StorageSystemRemoteFilesystemCache final : public shared_ptr_helper, public IStorageSystemOneBlock { @@ -16,7 +32,7 @@ public: static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemRemoteFilesystemCache(const StorageID & table_id_); + explicit StorageSystemRemoteFilesystemCache(const StorageID & table_id_); void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 21f2211a9d2..db30c265dc2 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -69,6 +69,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -161,6 +162,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "part_moves_between_shards"); attach(context, system_database, "asynchronous_inserts"); attach(context, system_database, "remote_filesystem_cache"); + attach(context, system_database, "remote_data_paths"); if (has_zookeeper) attach(context, system_database, "zookeeper"); diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 3c3e31d215a..0ba020a359b 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -1,18 +1,45 @@ -- { echo } +SET remote_fs_cache_on_insert=1; DROP TABLE IF EXISTS test; -SYSTEM DROP REMOTE FILESYSTEM CACHE; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; -SELECT count() FROM system.remote_filesystem_cache; -8 -SELECT * FROM test FORMAT Null; -SELECT count() size FROM system.remote_filesystem_cache; -9 SYSTEM DROP REMOTE FILESYSTEM CACHE; -INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; +Row 1: +────── +file_segment_range: (0,745) +size: 746 +state: DOWNLOADED +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +Row 1: +────── +file_segment_range: (0,745) +size: 746 +state: DOWNLOADED +SELECT cache_hits FROM system.remote_filesystem_cache; +0 +SELECT * FROM test FORMAT Null; +SELECT cache_hits FROM system.remote_filesystem_cache; +1 +SELECT * FROM test FORMAT Null; +SELECT cache_hits FROM system.remote_filesystem_cache; +2 SELECT count() size FROM system.remote_filesystem_cache; -7 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -SELECT count() size FROM system.remote_filesystem_cache; -14 +1 +SYSTEM DROP REMOTE FILESYSTEM CACHE; +INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path ORDER BY size FORMAT Vertical; +Row 1: +────── +file_segment_range: (0,1659) +size: 1660 +state: DOWNLOADED +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +Row 1: +────── +file_segment_range: (0,1659) +size: 1660 +state: DOWNLOADED diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 48ae85c8e61..8a4339db731 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -2,16 +2,36 @@ -- { echo } -DROP TABLE IF EXISTS test; -SYSTEM DROP REMOTE FILESYSTEM CACHE; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=1; +SET remote_fs_cache_on_insert=1; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -SELECT count() FROM system.remote_filesystem_cache; -SELECT * FROM test FORMAT Null; -SELECT count() size FROM system.remote_filesystem_cache; SYSTEM DROP REMOTE FILESYSTEM CACHE; -INSERT INTO test SELECT number, toString(number) FROM numbers(100, 100); -SELECT count() size FROM system.remote_filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used + +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; + +INSERT INTO test SELECT number, toString(number) FROM numbers(100); + +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; + +SELECT cache_hits FROM system.remote_filesystem_cache; + +SELECT * FROM test FORMAT Null; +SELECT cache_hits FROM system.remote_filesystem_cache; + +SELECT * FROM test FORMAT Null; +SELECT cache_hits FROM system.remote_filesystem_cache; + SELECT count() size FROM system.remote_filesystem_cache; + +SYSTEM DROP REMOTE FILESYSTEM CACHE; + +INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); + +SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path ORDER BY size FORMAT Vertical; +SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; + +-- INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used From 8c1be8950eda174a44260aa164b9f8d744fccded Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 16:54:43 +0100 Subject: [PATCH 100/372] Improve test --- src/Storages/MergeTree/MergeTreeData.cpp | 10 ++-- src/Storages/StorageMergeTree.cpp | 2 +- tests/config/config.d/storage_conf.xml | 2 + ...emote_filesystem_cache_on_insert.reference | 48 ++++++++++--------- ...2241_remote_filesystem_cache_on_insert.sql | 29 ++++++----- 5 files changed, 52 insertions(+), 39 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4fbc3376b7e..29b9460729f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -303,7 +303,7 @@ MergeTreeData::MergeTreeData( format_version = min_format_version; if (!version_file.second->isReadOnly()) { - auto buf = version_file.second->writeFile(version_file.first); + auto buf = version_file.second->writeFile(version_file.first, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, context_->getWriteSettings()); writeIntText(format_version.toUnderType(), *buf); if (getContext()->getSettingsRef().fsync_metadata) buf->sync(); @@ -3699,9 +3699,9 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac continue; UInt64 total_size_of_part = 0; - Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", ""); + Strings filenames = backup->listFiles(fs::path(data_path_in_backup) / part_name / "", ""); for (const String & filename : filenames) - total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename); + total_size_of_part += backup->getFileSize(fs::path(data_path_in_backup) / part_name / filename); std::shared_ptr reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part); @@ -3725,9 +3725,9 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac for (const String & filename : filenames) { - auto backup_entry = backup->readFile(data_path_in_backup + part_name + "/" + filename); + auto backup_entry = backup->readFile(fs::path(data_path_in_backup) / part_name / filename); auto read_buffer = backup_entry->getReadBuffer(); - auto write_buffer = disk->writeFile(temp_part_dir + "/" + filename); + auto write_buffer = disk->writeFile(fs::path(temp_part_dir) / filename); copyData(*read_buffer, *write_buffer); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 812e2264adb..0f2a7e90870 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1613,7 +1613,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ { auto calculated_checksums = checkDataPart(part, false); calculated_checksums.checkEqual(part->checksums, true); - auto out = disk->writeFile(tmp_checksums_path, 4096); + auto out = disk->writeFile(tmp_checksums_path, 4096, WriteMode::Rewrite, local_context->getWriteSettings()); part->checksums.write(*out); disk->moveFile(tmp_checksums_path, checksums_path); results.emplace_back(part->name, true, "Checksums recounted and written to disk."); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 2e43f735605..dd148147367 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -7,7 +7,9 @@ clickhouse clickhouse 1 + 0 22548578304 + 1 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 0ba020a359b..3d0d9dadf6a 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -2,44 +2,48 @@ SET remote_fs_cache_on_insert=1; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +0 +SELECT count() FROM system.remote_filesystem_cache; +0 INSERT INTO test SELECT number, toString(number) FROM numbers(100); -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range: (0,745) size: 746 state: DOWNLOADED -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; -Row 1: -────── -file_segment_range: (0,745) -size: 746 -state: DOWNLOADED -SELECT cache_hits FROM system.remote_filesystem_cache; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +7 +SELECT count() FROM system.remote_filesystem_cache; +7 +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; 0 SELECT * FROM test FORMAT Null; -SELECT cache_hits FROM system.remote_filesystem_cache; -1 +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +2 SELECT * FROM test FORMAT Null; -SELECT cache_hits FROM system.remote_filesystem_cache; +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; 2 SELECT count() size FROM system.remote_filesystem_cache; -1 +7 SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path ORDER BY size FORMAT Vertical; -Row 1: -────── -file_segment_range: (0,1659) -size: 1660 -state: DOWNLOADED -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range: (0,1659) size: 1660 state: DOWNLOADED +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +7 +SELECT count() FROM system.remote_filesystem_cache; +7 +SELECT count() FROM system.remote_filesystem_cache; +7 +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used +SELECT count() FROM system.remote_filesystem_cache; +7 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 8a4339db731..dfcc617e0f4 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -5,25 +5,27 @@ SET remote_fs_cache_on_insert=1; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.remote_filesystem_cache; INSERT INTO test SELECT number, toString(number) FROM numbers(100); -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path FORMAT Vertical; -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.remote_filesystem_cache; -SELECT cache_hits FROM system.remote_filesystem_cache; +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT cache_hits FROM system.remote_filesystem_cache; +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT cache_hits FROM system.remote_filesystem_cache; +SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; SELECT count() size FROM system.remote_filesystem_cache; @@ -31,7 +33,12 @@ SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range, size, state FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path ORDER BY size FORMAT Vertical; -SELECT file_segment_range, size, state FROM system.remote_filesystem_cache format Vertical; +SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.remote_filesystem_cache; + +SELECT count() FROM system.remote_filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used +SELECT count() FROM system.remote_filesystem_cache; + --- INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used From e39aba37a2bd5273f7afa3791396b3e8ae4e4456 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 18:11:52 +0100 Subject: [PATCH 101/372] Minor changes --- src/Common/FileCache.cpp | 2 ++ src/Common/FileCacheFactory.cpp | 4 ++- src/Disks/DiskLocal.cpp | 6 ---- src/Disks/S3/DiskS3.cpp | 2 -- .../System/StorageSystemRemoteDataPaths.cpp | 2 +- .../StorageSystemRemoteFilesystemCache.h | 31 +++++++++++-------- 6 files changed, 24 insertions(+), 23 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 07429a448f2..f0d2bba33d4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -723,8 +723,10 @@ std::vector LRUFileCache::tryGetCachePaths(const Key & key) const auto & cells_by_offset = files[key]; for (const auto & [offset, cell] : cells_by_offset) + { if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) cache_paths.push_back(getPathInLocalCache(key, offset)); + } return cache_paths; } diff --git a/src/Common/FileCacheFactory.cpp b/src/Common/FileCacheFactory.cpp index 683676041d2..9eadea05547 100644 --- a/src/Common/FileCacheFactory.cpp +++ b/src/Common/FileCacheFactory.cpp @@ -24,8 +24,8 @@ FileCacheFactory::CacheByBasePath FileCacheFactory::getAll() const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cache_base_path) { std::lock_guard lock(mutex); - auto * cache_data = getImpl(cache_base_path, lock); + auto * cache_data = getImpl(cache_base_path, lock); if (cache_data) return cache_data->settings; @@ -43,6 +43,7 @@ FileCacheFactory::CacheData * FileCacheFactory::getImpl(const std::string & cach FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) { std::lock_guard lock(mutex); + auto * cache_data = getImpl(cache_base_path, lock); if (cache_data) return cache_data->cache; @@ -54,6 +55,7 @@ FileCachePtr FileCacheFactory::getOrCreate( const std::string & cache_base_path, const FileCacheSettings & file_cache_settings) { std::lock_guard lock(mutex); + auto * cache_data = getImpl(cache_base_path, lock); if (cache_data) return cache_data->cache; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 874405cda53..78be223edee 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -318,16 +318,10 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { fs::path meta_path = fs::path(disk_path) / path; - std::cerr << "Disk local: " << meta_path << "\n"; if (!broken && fs::exists(meta_path) && fs::is_directory(meta_path)) - { return std::make_unique(disk_path, path); - } else - { - std::cerr << "\n\n\n iterating Fail\n\n"; return std::make_unique(); - } } void DiskLocal::moveFile(const String & from_path, const String & to_path) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 18cbcc3d659..a9e58efbfb9 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -46,8 +46,6 @@ #include -#include -#include namespace DB { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 28ae20be8e1..410d1ae6dd4 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -20,7 +20,7 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab { {"disk_name", std::make_shared()}, {"path", std::make_shared()}, - {"cache_dir", std::make_shared()}, + {"cache_base_path", std::make_shared()}, {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.h b/src/Storages/System/StorageSystemRemoteFilesystemCache.h index 4ac68671823..b4ace8a7fe8 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.h +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.h @@ -7,19 +7,24 @@ namespace DB { /** - * SELECT - * cache_path, - * local_path, - * remote_path - * FROM - * ( - * SELECT - * arrayJoin(cache_paths) AS cache_path, - * local_path, - * remote_path - * FROM system.remote_data_paths - * ) AS data_paths - * INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path + * SELECT + * cache_path, + * cache_hits, + * remote_path, + * local_path, + * file_segment_range, + * size, + * state + * FROM + * ( + * SELECT + * arrayJoin(cache_paths) AS cache_path, + * local_path, + * remote_path + * FROM system.remote_data_paths + * ) AS data_paths + * INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path + * FORMAT Vertical */ class StorageSystemRemoteFilesystemCache final : public shared_ptr_helper, From d2a3cfe5dc4fc42d6ea3f536f28d97008ff77234 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 19:00:42 +0100 Subject: [PATCH 102/372] Cache on all write operations --- src/Common/FileCacheSettings.cpp | 2 +- src/Common/FileCacheSettings.h | 2 +- src/Core/Settings.h | 2 +- src/Disks/DiskCacheWrapper.cpp | 4 ++-- src/Disks/S3/DiskS3.cpp | 4 ++-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 4 ++-- .../MergeTree/MergeTreeMutationEntry.h | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 10 ++++++---- src/Storages/MergeTree/MergeTreePartition.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 17 +++++++++-------- src/Storages/StorageMergeTree.cpp | 2 +- ...emote_filesystem_cache_on_insert.reference | 19 +++++++++++++++---- ...2241_remote_filesystem_cache_on_insert.sql | 16 ++++++++++++---- 16 files changed, 57 insertions(+), 35 deletions(-) diff --git a/src/Common/FileCacheSettings.cpp b/src/Common/FileCacheSettings.cpp index 02009d95550..f555de277b2 100644 --- a/src/Common/FileCacheSettings.cpp +++ b/src/Common/FileCacheSettings.cpp @@ -10,7 +10,7 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & max_size = config.getUInt64(config_prefix + ".data_cache_max_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_CACHE_SIZE); max_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS); max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE); - cache_on_insert = config.getUInt64(config_prefix + ".cache_on_insert", false); + cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false); } } diff --git a/src/Common/FileCacheSettings.h b/src/Common/FileCacheSettings.h index c7956e48282..53c28400c86 100644 --- a/src/Common/FileCacheSettings.h +++ b/src/Common/FileCacheSettings.h @@ -12,7 +12,7 @@ struct FileCacheSettings size_t max_size = 0; size_t max_elements = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS; size_t max_file_segment_size = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE; - bool cache_on_insert = false; + bool cache_on_write_operations = false; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f8a0ea3c7e7..6d275ad6790 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -557,7 +557,7 @@ class IColumn; M(UInt64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ M(Bool, remote_fs_enable_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ M(UInt64, remote_fs_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ - M(Bool, remote_fs_cache_on_insert, false, "Write into cache on INSERT query", 0) \ + M(Bool, remote_fs_cache_on_write_operations, true, "Write into cache on INSERT query To actually work this setting requires be added to disk config too", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index de5397a87c2..568fbf160c0 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -169,7 +169,7 @@ DiskCacheWrapper::readFile( auto src_buffer = DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); WriteSettings write_settings; - write_settings.remote_fs_cache_on_insert = false; + write_settings.remote_fs_cache_on_write_operations = false; auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite, write_settings); copyData(*src_buffer, *dst_buffer); @@ -206,7 +206,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode return DiskDecorator::writeFile(path, buf_size, mode, settings); WriteSettings current_settings = settings; - current_settings.remote_fs_cache_on_insert = false; + current_settings.remote_fs_cache_on_write_operations = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index a9e58efbfb9..292699b5e22 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -292,8 +292,8 @@ std::unique_ptr DiskS3::writeFile(const String & path, }; bool cache_on_insert = fs::path(path).extension() != ".tmp" - && write_settings.remote_fs_cache_on_insert - && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_insert; + && write_settings.remote_fs_cache_on_write_operations + && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; auto s3_buffer = std::make_unique( settings->client, diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index c34193574bb..81a6705cbab 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -5,7 +5,7 @@ namespace DB struct WriteSettings { - bool remote_fs_cache_on_insert = false; + bool remote_fs_cache_on_write_operations = false; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a64ef3a88be..a3169f435e4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3203,7 +3203,7 @@ WriteSettings Context::getWriteSettings() const { WriteSettings res; - res.remote_fs_cache_on_insert = settings.remote_fs_cache_on_insert; + res.remote_fs_cache_on_write_operations = settings.remote_fs_cache_on_write_operations; return res; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 935a11ec5fa..22c7c6af83e 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -230,7 +230,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() case MergeAlgorithm::Vertical : { ctx->rows_sources_file = createTemporaryFile(ctx->tmp_disk->getPath()); - ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->writeFile(fileName(ctx->rows_sources_file->path())); + ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->writeFile(fileName(ctx->rows_sources_file->path()), DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, global_ctx->context->getWriteSettings()); ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); MergeTreeDataPartInMemory::ColumnToSize local_merged_column_to_size; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 0f71742fb09..2147575f1d5 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -43,7 +43,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse mutation version from file name, expected 'mutation_.txt', got '{}'", file_name_); } -MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number) +MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const WriteSettings & settings) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -53,7 +53,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP { try { - auto out = disk->writeFile(path_prefix + file_name); + auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; *out << "commands: "; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 7554a03836e..fa3a4058ae6 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -29,7 +29,7 @@ struct MergeTreeMutationEntry String latest_fail_reason; /// Create a new entry and write it to a temporary file. - MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number); + MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, const WriteSettings & settings); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 4edf23bc0fb..128c40929b3 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -390,16 +390,18 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); - const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; - return store(partition_key_sample, disk, part_path, checksums); + const auto & context = storage.getContext(); + const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, context).sample_block; + return store(partition_key_sample, disk, part_path, checksums, context->getWriteSettings()); } -std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr MergeTreePartition::store( + const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const { if (!partition_key_sample) return nullptr; - auto out = disk->writeFile(part_path + "partition.dat"); + auto out = disk->writeFile(std::filesystem::path(part_path) / "partition.dat", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); HashingWriteBuffer out_hashing(*out); for (size_t i = 0; i < value.size(); ++i) partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index f149fcbcb7e..3bd9202822f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -41,7 +41,7 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; - [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 1fe701c54ae..9641299f1f8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -419,14 +419,15 @@ void finalizeMutatedPart( const MergeTreeDataPartPtr & source_part, MergeTreeData::MutableDataPartPtr new_data_part, ExecuteTTLType execute_ttl_type, - const CompressionCodecPtr & codec) + const CompressionCodecPtr & codec, + ContextPtr context) { auto disk = new_data_part->volume->getDisk(); auto part_path = fs::path(new_data_part->getFullRelativePath()); if (new_data_part->uuid != UUIDHelpers::Nil) { - auto out = disk->writeFile(part_path / IMergeTreeDataPart::UUID_FILE_NAME, 4096); + auto out = disk->writeFile(part_path / IMergeTreeDataPart::UUID_FILE_NAME, 4096, WriteMode::Rewrite, context->getWriteSettings()); HashingWriteBuffer out_hashing(*out); writeUUIDText(new_data_part->uuid, out_hashing); new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count(); @@ -436,7 +437,7 @@ void finalizeMutatedPart( if (execute_ttl_type != ExecuteTTLType::NONE) { /// Write a file with ttl infos in json format. - auto out_ttl = disk->writeFile(part_path / "ttl.txt", 4096); + auto out_ttl = disk->writeFile(part_path / "ttl.txt", 4096, WriteMode::Rewrite, context->getWriteSettings()); HashingWriteBuffer out_hashing(*out_ttl); new_data_part->ttl_infos.write(out_hashing); new_data_part->checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -445,7 +446,7 @@ void finalizeMutatedPart( if (!new_data_part->getSerializationInfos().empty()) { - auto out = disk->writeFile(part_path / IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); + auto out = disk->writeFile(part_path / IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096, WriteMode::Rewrite, context->getWriteSettings()); HashingWriteBuffer out_hashing(*out); new_data_part->getSerializationInfos().writeJSON(out_hashing); new_data_part->checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); @@ -454,18 +455,18 @@ void finalizeMutatedPart( { /// Write file with checksums. - auto out_checksums = disk->writeFile(part_path / "checksums.txt", 4096); + auto out_checksums = disk->writeFile(part_path / "checksums.txt", 4096, WriteMode::Rewrite, context->getWriteSettings()); new_data_part->checksums.write(*out_checksums); } /// close fd { - auto out = disk->writeFile(part_path / IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); + auto out = disk->writeFile(part_path / IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096, WriteMode::Rewrite, context->getWriteSettings()); DB::writeText(queryToString(codec->getFullCodecDesc()), *out); } { /// Write a file with a description of columns. - auto out_columns = disk->writeFile(part_path / "columns.txt", 4096); + auto out_columns = disk->writeFile(part_path / "columns.txt", 4096, WriteMode::Rewrite, context->getWriteSettings()); new_data_part->getColumns().writeText(*out_columns); } /// close fd @@ -1162,7 +1163,7 @@ private: } } - MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec); + MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5de1b959d7c..ae6d9e5474e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -423,7 +423,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), getContext()->getWriteSettings()); version = increment.get(); entry.commit(version); mutation_file_name = entry.file_name; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 3d0d9dadf6a..5bc2049204a 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -1,15 +1,15 @@ -- { echo } -SET remote_fs_cache_on_insert=1; +SET remote_fs_cache_on_write_operations=1; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; SYSTEM DROP REMOTE FILESYSTEM CACHE; SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 0 SELECT count() FROM system.remote_filesystem_cache; 0 -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── @@ -44,6 +44,17 @@ SELECT count() FROM system.remote_filesystem_cache; 7 SELECT count() FROM system.remote_filesystem_cache; 7 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; SELECT count() FROM system.remote_filesystem_cache; 7 +INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); +SELECT count() FROM system.remote_filesystem_cache; +21 +OPTIMIZE TABLE test FINAL; +SELECT count() FROM system.remote_filesystem_cache; +24 +SET mutations_sync=2; +ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; +SELECT count() FROM system.remote_filesystem_cache; +25 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index dfcc617e0f4..58f4adb5980 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -2,10 +2,10 @@ -- { echo } -SET remote_fs_cache_on_insert=1; +SET remote_fs_cache_on_write_operations=1; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; SYSTEM DROP REMOTE FILESYSTEM CACHE; @@ -13,7 +13,7 @@ SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, st SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; SELECT count() FROM system.remote_filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; @@ -38,7 +38,15 @@ SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, re SELECT count() FROM system.remote_filesystem_cache; SELECT count() FROM system.remote_filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_insert=0; -- still writes cache because now config setting is used +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; SELECT count() FROM system.remote_filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); +SELECT count() FROM system.remote_filesystem_cache; +OPTIMIZE TABLE test FINAL; +SELECT count() FROM system.remote_filesystem_cache; +SET mutations_sync=2; +ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; +SELECT count() FROM system.remote_filesystem_cache; From d4161b59256e1b3d41fed267a4d43d4e5373513a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 23 Mar 2022 19:46:28 +0100 Subject: [PATCH 103/372] Add optin `read_from_cache_if_exists_otherwise_bypass_cache` (for merges) --- src/Common/FileCache.cpp | 2 +- src/Common/FileCache.h | 3 ++- src/Common/FileSegment.cpp | 3 ++- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 24 +++++++++++++++++-- src/Disks/IO/CachedReadBufferFromRemoteFS.h | 2 ++ src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 5 +++- src/IO/ReadSettings.h | 1 + src/IO/WriteBufferFromS3.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 3 ++- tests/config/config.d/storage_conf.xml | 2 +- 10 files changed, 38 insertions(+), 9 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index f0d2bba33d4..37a8ac78e98 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -57,7 +57,7 @@ String IFileCache::getPathInLocalCache(const Key & key) return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str; } -bool IFileCache::shouldBypassCache() +bool IFileCache::isReadOnly() { return !CurrentThread::isInitialized() || !CurrentThread::get().getQueryContext() diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index e8280fba08a..90632a54edd 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -43,7 +43,8 @@ public: virtual void tryRemoveAll() = 0; - static bool shouldBypassCache(); + /// If cache can be used as read only. (For merges, for example). + static bool isReadOnly(); /// Cache capacity in bytes. size_t capacity() const { return max_size; } diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index f13cbc6a464..ce1253a6f5c 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -89,7 +89,8 @@ String FileSegment::getCallerId() String FileSegment::getCallerIdImpl(bool allow_non_strict_checking) { - if (IFileCache::shouldBypassCache()) + /// Cache is read only, if it is read operation (which can potentially do cache writes), but there is no query attached. + if (IFileCache::isReadOnly()) { /// getCallerId() can be called from completeImpl(), which can be called from complete(). /// complete() is called from destructor of CachedReadBufferFromRemoteFS when there is no query id anymore. diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index d093e29b28e..84490d84801 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -124,6 +124,21 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getReadBufferForFileSegment( size_t wait_download_tries = 0; auto download_state = file_segment->state(); + + if (settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + { + if (download_state == FileSegment::State::DOWNLOADED) + { + read_type = ReadType::CACHED; + return getCacheReadBuffer(range.left); + } + else + { + read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; + return getRemoteFSReadBuffer(file_segment, read_type); + } + } + while (true) { switch (download_state) @@ -544,8 +559,7 @@ bool CachedReadBufferFromRemoteFS::nextImpl() bool CachedReadBufferFromRemoteFS::nextImplStep() { - if (IFileCache::shouldBypassCache()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Using cache when not allowed"); + assertCacheAllowed(); if (!initialized) initialize(file_offset_of_buffer_end, getTotalSizeToRead()); @@ -758,6 +772,12 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() return std::nullopt; } +void CachedReadBufferFromRemoteFS::assertCacheAllowed() const +{ + if (IFileCache::isReadOnly() && !settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache used when not allowed"); +} + String CachedReadBufferFromRemoteFS::getInfoForLog() { return fmt::format("Buffer path: {}, hash key: {}, file_offset_of_buffer_end: {}, internal buffer remaining read range: {}, file segment info: {}", diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.h b/src/Disks/IO/CachedReadBufferFromRemoteFS.h index 3d03debcd01..d5567588019 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.h +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.h @@ -50,6 +50,8 @@ private: bool nextImplStep(); + void assertCacheAllowed() const; + enum class ReadType { CACHED, diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8f91804bbbe..91f448a2ea7 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.remote_fs_enable_cache && !IFileCache::shouldBypassCache(); + bool with_cache = cache && settings.remote_fs_enable_cache; auto remote_file_reader_creator = [=, this]() { @@ -49,6 +49,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S if (with_cache) { + if (IFileCache::isReadOnly()) + settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; + return std::make_shared( path, cache, remote_file_reader_creator, settings, read_until_position ? read_until_position : file_size); } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e321eecf104..936de1673b4 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -79,6 +79,7 @@ struct ReadSettings size_t remote_fs_read_backoff_max_tries = 4; bool remote_fs_enable_cache = true; size_t remote_fs_cache_max_wait_sec = 1; + bool remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = false; size_t remote_read_min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 53118bbf867..9ed008907c9 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -160,7 +160,7 @@ WriteBufferFromS3::~WriteBufferFromS3() bool WriteBufferFromS3::cacheEnabled() const { - return cache && IFileCache::shouldBypassCache() == false; + return cache != nullptr; } void WriteBufferFromS3::preFinalize() diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 22c7c6af83e..d9cf6c21091 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -261,7 +261,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), ctx->compression_codec, /*reset_columns=*/ true, - ctx->blocks_are_granules_size); + ctx->blocks_are_granules_size, + global_ctx->context->getWriteSettings()); global_ctx->rows_written = 0; ctx->initial_reservation = global_ctx->space_reservation ? global_ctx->space_reservation->getSize() : 0; diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dd148147367..3dd4811b1bf 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -9,7 +9,7 @@ 1 0 22548578304 - 1 + 1 From 557edbd172aac7a97aca68d37b204ce2c3f8f749 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 24 Mar 2022 12:54:12 +0000 Subject: [PATCH 104/372] Add some improvements and fixes in schema inference --- src/Core/Settings.h | 6 + src/DataTypes/DataTypeMap.cpp | 36 ++- src/DataTypes/DataTypeMap.h | 2 + src/Formats/EscapingRuleUtils.cpp | 294 +++++++++++++++--- src/Formats/EscapingRuleUtils.h | 5 +- src/Formats/FormatFactory.cpp | 8 +- src/Formats/FormatFactory.h | 2 +- src/Formats/FormatSettings.h | 9 +- src/Formats/ReadSchemaUtils.cpp | 30 +- src/Formats/ReadSchemaUtils.h | 8 +- src/IO/ReadHelpers.cpp | 11 + src/Processors/Formats/ISchemaReader.cpp | 22 +- src/Processors/Formats/ISchemaReader.h | 9 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 10 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 70 ++++- .../Formats/Impl/ArrowColumnToCHColumn.h | 6 +- .../Formats/Impl/AvroRowInputFormat.cpp | 4 +- .../Formats/Impl/BinaryRowInputFormat.cpp | 4 +- .../Formats/Impl/CSVRowInputFormat.cpp | 12 +- .../Formats/Impl/CSVRowInputFormat.h | 3 +- .../Impl/CustomSeparatedRowInputFormat.cpp | 11 +- .../Impl/CustomSeparatedRowInputFormat.h | 3 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 4 +- .../Impl/JSONEachRowRowInputFormat.cpp | 4 +- .../Formats/Impl/MsgPackRowInputFormat.cpp | 4 +- src/Processors/Formats/Impl/NativeFormat.cpp | 2 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 8 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 8 +- .../Formats/Impl/RegexpRowInputFormat.cpp | 11 +- .../Formats/Impl/RegexpRowInputFormat.h | 3 +- .../Formats/Impl/TSKVRowInputFormat.cpp | 2 +- .../Impl/TabSeparatedRowInputFormat.cpp | 4 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 12 +- .../Formats/Impl/TemplateRowInputFormat.h | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 50 ++- .../Formats/Impl/ValuesBlockInputFormat.h | 4 +- .../RowInputFormatWithNamesAndTypes.cpp | 4 +- .../Formats/RowInputFormatWithNamesAndTypes.h | 2 +- .../02149_schema_inference.reference | 32 +- ...ma_inference_formats_with_schema.reference | 168 +++++----- ...arquet_nullable_schema_inference.reference | 40 +++ ...w_orc_parquet_nullable_schema_inference.sh | 21 ++ ...ead_null_type_to_nullable_column.reference | 1 + ...arrow_read_null_type_to_nullable_column.sh | 28 ++ ...column_names_in_shcmea_inference.reference | 8 + ...02244_column_names_in_shcmea_inference.sql | 12 + .../02245_parquet_skip_unknown_type.reference | 16 + .../02245_parquet_skip_unknown_type.sh | 18 ++ ...csv_best_effort_schema_inference.reference | 107 +++++++ ...46_tsv_csv_best_effort_schema_inference.sh | 220 +++++++++++++ 50 files changed, 1062 insertions(+), 300 deletions(-) create mode 100644 tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference create mode 100755 tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh create mode 100644 tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.reference create mode 100755 tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.sh create mode 100644 tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference create mode 100644 tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql create mode 100644 tests/queries/0_stateless/02245_parquet_skip_unknown_type.reference create mode 100755 tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh create mode 100644 tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference create mode 100755 tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ca2e9f12e66..86ea202fda7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -631,6 +631,12 @@ class IColumn; M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \ M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \ M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \ + M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ + M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ + M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Parquet", 0) \ + M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format ORC", 0) \ + M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Arrow", 0) \ + M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 41de17982aa..42ec739c33b 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -45,22 +45,7 @@ DataTypeMap::DataTypeMap(const DataTypePtr & key_type_, const DataTypePtr & valu void DataTypeMap::assertKeyType() const { - bool type_error = false; - if (key_type->getTypeId() == TypeIndex::LowCardinality) - { - const auto & low_cardinality_data_type = assert_cast(*key_type); - if (!isStringOrFixedString(*(low_cardinality_data_type.getDictionaryType()))) - type_error = true; - } - else if (!key_type->isValueRepresentedByInteger() - && !isStringOrFixedString(*key_type) - && !WhichDataType(key_type).isNothing() - && !WhichDataType(key_type).isUUID()) - { - type_error = true; - } - - if (type_error) + if (!checkKeyType(key_type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type of Map key must be a type, that can be represented by integer or String or FixedString (possibly LowCardinality) or UUID," " but {} given", key_type->getName()); @@ -102,6 +87,25 @@ bool DataTypeMap::equals(const IDataType & rhs) const return nested->equals(*rhs_map.nested); } +bool DataTypeMap::checkKeyType(DataTypePtr key_type) +{ + if (key_type->getTypeId() == TypeIndex::LowCardinality) + { + const auto & low_cardinality_data_type = assert_cast(*key_type); + if (!isStringOrFixedString(*(low_cardinality_data_type.getDictionaryType()))) + return false; + } + else if (!key_type->isValueRepresentedByInteger() + && !isStringOrFixedString(*key_type) + && !WhichDataType(key_type).isNothing() + && !WhichDataType(key_type).isUUID()) + { + return false; + } + + return true; +} + static DataTypePtr create(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 2) diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 65bdd93ca4d..479008031fe 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -48,6 +48,8 @@ public: SerializationPtr doGetDefaultSerialization() const override; + static bool checkKeyType(DataTypePtr key_type); + private: void assertKeyType() const; }; diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index b0ea10abdb6..08b34ebb0fc 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -5,12 +5,17 @@ #include #include #include +#include +#include +#include +#include +#include +#include #include #include -#include +#include #include -#include -#include + namespace DB { @@ -138,7 +143,8 @@ bool deserializeFieldByEscapingRule( serialization->deserializeTextRaw(column, buf, format_settings); break; default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Escaping rule {} is not suitable for deserialization", escapingRuleToString(escaping_rule)); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Escaping rule {} is not suitable for deserialization", escapingRuleToString(escaping_rule)); } return read; } @@ -176,7 +182,8 @@ void serializeFieldByEscapingRule( } } -void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +void writeStringByEscapingRule( + const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) { switch (escaping_rule) { @@ -249,85 +256,270 @@ String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule e return readByEscapingRule(buf, escaping_rule, format_settings); } -static bool evaluateConstantExpressionFromString(const StringRef & field, DataTypePtr & type, ContextPtr context) +static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) { - if (!context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "You must provide context to evaluate constant expression"); + if (buf.eof()) + return nullptr; - ParserExpression parser; - Expected expected; - Tokens tokens(field.data, field.data + field.size); - IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth); - ASTPtr ast; - - /// FIXME: Our parser cannot parse maps in the form of '{key : value}' that is used in text formats. - bool parsed = parser.parse(token_iterator, ast, expected); - if (!parsed || !token_iterator->isEnd()) - return false; - - try + /// Array + if (checkChar('[', buf)) { - std::pair result = evaluateConstantExpression(ast, context); - type = generalizeDataType(result.second); - return true; + skipWhitespaceIfAny(buf); + + DataTypes nested_types; + bool first = true; + while (!buf.eof() && *buf.position() != ']') + { + if (!first) + { + skipWhitespaceIfAny(buf); + if (!checkChar(',', buf)) + return nullptr; + skipWhitespaceIfAny(buf); + } + else + first = false; + + auto nested_type = determineDataTypeForSingleFieldImpl(buf); + if (!nested_type) + return nullptr; + + nested_types.push_back(nested_type); + } + + if (buf.eof()) + return nullptr; + + ++buf.position(); + + if (nested_types.empty()) + return std::make_shared(std::make_shared()); + + auto least_supertype = tryGetLeastSupertype(nested_types); + if (!least_supertype) + return nullptr; + + return std::make_shared(least_supertype); } - catch (...) + + /// Tuple + if (checkChar('(', buf)) { - return false; + skipWhitespaceIfAny(buf); + + DataTypes nested_types; + bool first = true; + while (!buf.eof() && *buf.position() != ')') + { + if (!first) + { + skipWhitespaceIfAny(buf); + if (!checkChar(',', buf)) + return nullptr; + skipWhitespaceIfAny(buf); + } + else + first = false; + + auto nested_type = determineDataTypeForSingleFieldImpl(buf); + if (!nested_type) + return nullptr; + + nested_types.push_back(nested_type); + } + + if (buf.eof() || nested_types.empty()) + return nullptr; + + ++buf.position(); + + return std::make_shared(nested_types); } + + /// Map + if (checkChar('{', buf)) + { + skipWhitespaceIfAny(buf); + + DataTypes key_types; + DataTypes value_types; + bool first = true; + while (!buf.eof() && *buf.position() != '}') + { + if (!first) + { + skipWhitespaceIfAny(buf); + if (!checkChar(',', buf)) + return nullptr; + skipWhitespaceIfAny(buf); + } + else + first = false; + + auto key_type = determineDataTypeForSingleFieldImpl(buf); + if (!key_type) + return nullptr; + + key_types.push_back(key_type); + + skipWhitespaceIfAny(buf); + if (!checkChar(':', buf)) + return nullptr; + skipWhitespaceIfAny(buf); + + auto value_type = determineDataTypeForSingleFieldImpl(buf); + if (!value_type) + return nullptr; + + value_types.push_back(value_type); + } + + if (buf.eof()) + return nullptr; + + ++buf.position(); + skipWhitespaceIfAny(buf); + + if (key_types.empty()) + return std::make_shared(std::make_shared(), std::make_shared()); + + auto key_least_supertype = tryGetLeastSupertype(key_types); + + auto value_least_supertype = tryGetLeastSupertype(value_types); + if (!key_least_supertype || !value_least_supertype) + return nullptr; + + if (!DataTypeMap::checkKeyType(key_least_supertype)) + return nullptr; + + return std::make_shared(key_least_supertype, value_least_supertype); + } + + /// String + if (*buf.position() == '\'') + { + ++buf.position(); + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\\', '\''>(buf.position(), buf.buffer().end()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\'') + break; + + if (*buf.position() == '\\') + ++buf.position(); + } + + if (buf.eof()) + return nullptr; + + ++buf.position(); + return std::make_shared(); + } + + /// Bool + if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf)) + return DataTypeFactory::instance().get("Bool"); + + /// Null + if (checkStringCaseInsensitive("NULL", buf)) + return std::make_shared(); + + Float64 tmp; + if (tryReadFloatText(tmp, buf)) + return std::make_shared(); + + return nullptr; } -DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context) +static DataTypePtr determineDataTypeForSingleField(ReadBuffer & buf) +{ + return makeNullableRecursivelyAndCheckForNothing(determineDataTypeForSingleFieldImpl(buf)); +} + +DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule) { switch (escaping_rule) { case FormatSettings::EscapingRule::Quoted: { - DataTypePtr type; - bool parsed = evaluateConstantExpressionFromString(field, type, context); - return parsed ? type : nullptr; + ReadBufferFromString buf(field); + auto type = determineDataTypeForSingleField(buf); + return buf.eof() ? type : nullptr; } case FormatSettings::EscapingRule::JSON: return getDataTypeFromJSONField(field); case FormatSettings::EscapingRule::CSV: { + if (!format_settings.csv.input_format_use_best_effort_in_schema_inference) + return makeNullable(std::make_shared()); + if (field.empty() || field == format_settings.csv.null_representation) return nullptr; if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) - return std::make_shared(); + return DataTypeFactory::instance().get("Nullable(Bool)"); - DataTypePtr type; - bool parsed; - if (field[0] == '\'' || field[0] == '"') + if (field.size() > 1 && ((field.front() == '\'' && field.back() == '\'') || (field.front() == '"' && field.back() == '"'))) { - /// Try to evaluate expression inside quotes. - parsed = evaluateConstantExpressionFromString(StringRef(field.data() + 1, field.size() - 2), type, context); - /// If it's a number in quotes we determine it as a string. - if (parsed && type && isNumber(removeNullable(type))) - return makeNullable(std::make_shared()); - } - else - parsed = evaluateConstantExpressionFromString(field, type, context); + auto s = std::string_view(field.data() + 1, field.size() - 2); - /// If we couldn't parse an expression, determine it as a string. - return parsed ? type : makeNullable(std::make_shared()); + ReadBufferFromString buf(std::string_view(field.data() + 1, field.size() - 2)); + /// Try to determine the type of value inside quotes + auto type = determineDataTypeForSingleField(buf); + + if (!type) + return nullptr; + + /// If it's a number or tuple in quotes or there is some unread data in buffer, we determine it as a string. + if (isNumber(removeNullable(type)) || isTuple(type) || !buf.eof()) + return makeNullable(std::make_shared()); + + return type; + } + + /// Case when CSV value is not in quotes. Check if it's a number, and if not, determine it's as a string. + ReadBufferFromString buf(field); + Float64 tmp; + if (tryReadFloatText(tmp, buf) && buf.eof()) + return makeNullable(std::make_shared()); + + return makeNullable(std::make_shared()); } case FormatSettings::EscapingRule::Raw: [[fallthrough]]; case FormatSettings::EscapingRule::Escaped: - /// TODO: Try to use some heuristics here to determine the type of data. - return field.empty() ? nullptr : makeNullable(std::make_shared()); + { + if (!format_settings.tsv.input_format_use_best_effort_in_schema_inference) + return makeNullable(std::make_shared()); + + if (field.empty() || field == format_settings.tsv.null_representation) + return nullptr; + + if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) + return DataTypeFactory::instance().get("Nullable(Bool)"); + + ReadBufferFromString buf(field); + auto type = determineDataTypeForSingleField(buf); + if (!buf.eof()) + return makeNullable(std::make_shared()); + + return type; + } default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the type for value with {} escaping rule", escapingRuleToString(escaping_rule)); } } -DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context) +DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule) { DataTypes data_types; data_types.reserve(fields.size()); for (const auto & field : fields) - data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule, context)); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule)); return data_types; } @@ -344,4 +536,12 @@ DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escap } } +DataTypes getDefaultDataTypeForEscapingRules(const std::vector & escaping_rules) +{ + DataTypes data_types; + for (const auto & rule : escaping_rules) + data_types.push_back(getDefaultDataTypeForEscapingRule(rule)); + return data_types; +} + } diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h index 10147b29ad6..3c7c768c003 100644 --- a/src/Formats/EscapingRuleUtils.h +++ b/src/Formats/EscapingRuleUtils.h @@ -49,9 +49,10 @@ String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule es /// expression inside quotes as a constant expression, and if it fails or /// the result is a number (we don't parse numbers in quotes) we treat it as a String. /// - For TSV and TSVRaw we treat each field as a String (TODO: try to use some tweaks and heuristics here) -DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr); -DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr); +DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule); +DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule); DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule); +DataTypes getDefaultDataTypeForEscapingRules(const std::vector & escaping_rules); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 08554cf7e07..f8636768d00 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -65,6 +65,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.format_csv_null_representation; format_settings.csv.input_format_arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; + format_settings.csv.input_format_use_best_effort_in_schema_inference = settings.input_format_csv_use_best_effort_in_schema_inference; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; @@ -94,6 +95,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; format_settings.parquet.import_nested = settings.input_format_parquet_import_nested; format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; + format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -114,6 +116,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; format_settings.tsv.null_representation = settings.format_tsv_null_representation; + format_settings.tsv.input_format_use_best_effort_in_schema_inference = settings.input_format_tsv_use_best_effort_in_schema_inference; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; @@ -123,15 +126,18 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; format_settings.arrow.allow_missing_columns = settings.input_format_arrow_allow_missing_columns; + format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference; format_settings.orc.import_nested = settings.input_format_orc_import_nested; format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; + format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_orc_skip_columns_with_unsupported_types_in_schema_inference; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.seekable_read = settings.input_format_allow_seeks; format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation; format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; + format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) @@ -366,7 +372,7 @@ SchemaReaderPtr FormatFactory::getSchemaReader( throw Exception("FormatFactory: Format " + name + " doesn't support schema inference.", ErrorCodes::LOGICAL_ERROR); auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - return schema_reader_creator(buf, format_settings, context); + return schema_reader_creator(buf, format_settings); } ExternalSchemaReaderPtr FormatFactory::getExternalSchemaReader( diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 344dabd3f4d..2f53da3bdff 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -97,7 +97,7 @@ private: /// The checker should return true if format support append. using AppendSupportChecker = std::function; - using SchemaReaderCreator = std::function; + using SchemaReaderCreator = std::function; using ExternalSchemaReaderCreator = std::function; struct Creators diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 4881c1a43c8..6ecd04536a6 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -37,6 +37,8 @@ struct FormatSettings bool seekable_read = true; UInt64 max_rows_to_read_for_schema_inference = 100; + String column_names_for_schema_inference = ""; + enum class DateTimeInputFormat { Basic, /// Default format for fast parsing: YYYY-MM-DD hh:mm:ss (ISO-8601 without fractional part and timezone) or NNNNNNNNNN unix timestamp. @@ -75,6 +77,7 @@ struct FormatSettings bool low_cardinality_as_dictionary = false; bool import_nested = false; bool allow_missing_columns = false; + bool skip_columns_with_unsupported_types_in_schema_inference = false; } arrow; struct @@ -101,6 +104,7 @@ struct FormatSettings bool input_format_arrays_as_nested_csv = false; String null_representation = "\\N"; char tuple_delimiter = ','; + bool input_format_use_best_effort_in_schema_inference = true; } csv; struct HiveText @@ -108,7 +112,7 @@ struct FormatSettings char fields_delimiter = '\x01'; char collection_items_delimiter = '\x02'; char map_keys_delimiter = '\x03'; - Names input_field_names; + Names input_field_names = {""}; } hive_text; struct Custom @@ -137,6 +141,7 @@ struct FormatSettings UInt64 row_group_size = 1000000; bool import_nested = false; bool allow_missing_columns = false; + bool skip_columns_with_unsupported_types_in_schema_inference = false; } parquet; struct Pretty @@ -203,6 +208,7 @@ struct FormatSettings bool crlf_end_of_line = false; String null_representation = "\\N"; bool input_format_enum_as_number = false; + bool input_format_use_best_effort_in_schema_inference = true; } tsv; struct @@ -217,6 +223,7 @@ struct FormatSettings bool import_nested = false; bool allow_missing_columns = false; int64_t row_batch_size = 100'000; + bool skip_columns_with_unsupported_types_in_schema_inference = false; } orc; /// For capnProto format we should determine how to diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 559fac4cfaa..18080b0e896 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -65,8 +65,11 @@ ColumnsDescription readSchemaFromFormat(const String & format_name, const std::o return readSchemaFromFormat(format_name, format_settings, read_buffer_creator, context, buf_out); } -DataTypePtr generalizeDataType(DataTypePtr type) +DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type) { + if (!type) + return nullptr; + WhichDataType which(type); if (which.isNothing()) @@ -75,16 +78,13 @@ DataTypePtr generalizeDataType(DataTypePtr type) if (which.isNullable()) { const auto * nullable_type = assert_cast(type.get()); - return generalizeDataType(nullable_type->getNestedType()); + return makeNullableRecursivelyAndCheckForNothing(nullable_type->getNestedType()); } - if (isNumber(type)) - return makeNullable(std::make_shared()); - if (which.isArray()) { const auto * array_type = assert_cast(type.get()); - auto nested_type = generalizeDataType(array_type->getNestedType()); + auto nested_type = makeNullableRecursivelyAndCheckForNothing(array_type->getNestedType()); return nested_type ? std::make_shared(nested_type) : nullptr; } @@ -94,7 +94,7 @@ DataTypePtr generalizeDataType(DataTypePtr type) DataTypes nested_types; for (const auto & element : tuple_type->getElements()) { - auto nested_type = generalizeDataType(element); + auto nested_type = makeNullableRecursivelyAndCheckForNothing(element); if (!nested_type) return nullptr; nested_types.push_back(nested_type); @@ -105,19 +105,27 @@ DataTypePtr generalizeDataType(DataTypePtr type) if (which.isMap()) { const auto * map_type = assert_cast(type.get()); - auto key_type = removeNullable(generalizeDataType(map_type->getKeyType())); - auto value_type = generalizeDataType(map_type->getValueType()); - return key_type && value_type ? std::make_shared(key_type, value_type) : nullptr; + auto key_type = makeNullableRecursivelyAndCheckForNothing(map_type->getKeyType()); + auto value_type = makeNullableRecursivelyAndCheckForNothing(map_type->getValueType()); + return key_type && value_type ? std::make_shared(removeNullable(key_type), value_type) : nullptr; } if (which.isLowCarnality()) { const auto * lc_type = assert_cast(type.get()); - auto nested_type = generalizeDataType(lc_type->getDictionaryType()); + auto nested_type = makeNullableRecursivelyAndCheckForNothing(lc_type->getDictionaryType()); return nested_type ? std::make_shared(nested_type) : nullptr; } return makeNullable(type); } +NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header) +{ + NamesAndTypesList result; + for (auto & [name, type] : header.getNamesAndTypesList()) + result.emplace_back(name, makeNullableRecursivelyAndCheckForNothing(type)); + return result; +} + } diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index 4446393a581..ea8ebbad4c0 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -29,14 +29,16 @@ ColumnsDescription readSchemaFromFormat( ContextPtr context, std::unique_ptr & buf_out); -/// Convert type to the most general type: -/// - IntN, UIntN, FloatN, Decimal -> Float64 +/// Make type Nullable recursively: /// - Type -> Nullable(type) /// - Array(Type) -> Array(Nullable(Type)) /// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN)) /// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType)) /// - LowCardinality(Type) -> LowCardinality(Nullable(Type)) /// If type is Nothing or one of the nested types is Nothing, return nullptr. -DataTypePtr generalizeDataType(DataTypePtr type); +DataTypePtr makeNullableRecursivelyAndCheckForNothing(DataTypePtr type); +/// Call makeNullableRecursivelyAndCheckForNothing for all types +/// in the block and return names and types. +NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header); } diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index e086f16be54..98a33612aa8 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1366,6 +1366,7 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf) /// - Tuples: (...) /// - Maps: {...} /// - NULL + /// - Bool: true/false /// - Number: integer, float, decimal. if (*buf.position() == '\'') @@ -1394,6 +1395,16 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf) s.append("NaN"); } } + else if (checkCharCaseInsensitive('t', buf)) + { + assertStringCaseInsensitive("rue", buf); + s.append("true"); + } + else if (checkCharCaseInsensitive('f', buf)) + { + assertStringCaseInsensitive("alse", buf); + s.append("false"); + } else { /// It's an integer, float or decimal. They all can be parsed as float. diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 096e39a2893..17db8865310 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -10,9 +11,16 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) - : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +IRowSchemaReader::IRowSchemaReader( + ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_, const DataTypes & default_types_) + : ISchemaReader(in_) + , max_rows_to_read(format_settings.max_rows_to_read_for_schema_inference), default_type(default_type_), default_types(default_types_) { + if (!format_settings.column_names_for_schema_inference.empty()) + { + /// column_names_for_schema_inference is a string in format 'column1,column2,column3,...' + boost::split(column_names, format_settings.column_names_for_schema_inference, boost::is_any_of(",")); + } } NamesAndTypesList IRowSchemaReader::readSchema() @@ -43,6 +51,8 @@ NamesAndTypesList IRowSchemaReader::readSchema() { if (default_type) data_types[i] = default_type; + else if (!default_types.empty() && i < default_types.size() && default_types[i]) + data_types[i] = default_types[i]; else throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, @@ -74,14 +84,16 @@ NamesAndTypesList IRowSchemaReader::readSchema() /// Check that we could determine the type of this column. if (!data_types[i]) { - if (!default_type) + if (default_type) + data_types[i] = default_type; + else if (!default_types.empty() && i < default_types.size() && default_types[i]) + data_types[i] = default_types[i]; + else throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", max_rows_to_read); - - data_types[i] = default_type; } result.emplace_back(column_names[i], data_types[i]); } diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 2d35809e26a..cd0b552c021 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -27,12 +27,14 @@ protected: /// Base class for schema inference for formats that read data row by row. /// It reads data row by row (up to max_rows_to_read), determines types of columns /// for each row and compare them with types from the previous rows. If some column -/// contains values with different types in different rows, the default type will be -/// used for this column or the exception will be thrown (if default type is not set). +/// contains values with different types in different rows, the default type +/// (from argument default_type_) will be used for this column or the exception +/// will be thrown (if default type is not set). If different columns have different +/// default types, you can provide them by default_types_ argument. class IRowSchemaReader : public ISchemaReader { public: - IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_ = nullptr, const DataTypes & default_types_ = {}); NamesAndTypesList readSchema() override; protected: @@ -47,6 +49,7 @@ protected: private: size_t max_rows_to_read; DataTypePtr default_type; + DataTypes default_types; std::vector column_names; }; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index cf5cfa681a1..42c68e4654b 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -3,6 +3,7 @@ #if USE_ARROW #include +#include #include #include #include @@ -167,8 +168,9 @@ NamesAndTypesList ArrowSchemaReader::readSchema() schema = createFileReader(in, format_settings, is_stopped)->schema(); } - auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, stream ? "ArrowStream" : "Arrow"); - return header.getNamesAndTypesList(); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( + *schema, stream ? "ArrowStream" : "Arrow", format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference); + return getNamesAndRecursivelyNullableTypes(header); } void registerInputFormatArrow(FormatFactory & factory) @@ -198,13 +200,13 @@ void registerArrowSchemaReader(FormatFactory & factory) { factory.registerSchemaReader( "Arrow", - [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, false, settings); }); factory.registerSchemaReader( "ArrowStream", - [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, true, settings); });} diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 14c81a0d90d..8f5bb205bef 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -26,11 +27,13 @@ #include #include #include +#include #include #include #include #include #include +#include /// UINT16 and UINT32 are processed separately, see comments in readColumnFromArrowColumn. #define FOR_ARROW_NUMERIC_TYPES(M) \ @@ -328,12 +331,17 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( const std::string & format_name, bool is_nullable, std::unordered_map> & dictionary_values, - bool read_ints_as_dates) + bool read_ints_as_dates, + bool allow_null_type, + bool skip_columns_with_unsupported_types, + bool & skipped) { if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) { - auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates); + auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped); + if (skipped) + return {}; auto nullmap_column = readByteMapFromArrowColumn(arrow_column); auto nullable_type = std::make_shared(std::move(nested_column.type)); auto nullable_column = ColumnNullable::create(nested_column.column, nullmap_column); @@ -378,7 +386,10 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::MAP: { auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped); + if (skipped) + return {}; + auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); const auto * tuple_column = assert_cast(nested_column.column.get()); @@ -390,7 +401,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::LIST: { auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped); + if (skipped) + return {}; auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); auto array_column = ColumnArray::create(nested_column.column, offsets_column); auto array_type = std::make_shared(nested_column.type); @@ -415,7 +428,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( for (int i = 0; i != arrow_struct_type->num_fields(); ++i) { auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates); + auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped); + if (skipped) + return {}; tuple_elements.emplace_back(std::move(element.column)); tuple_types.emplace_back(std::move(element.type)); tuple_names.emplace_back(std::move(element.name)); @@ -438,7 +453,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( dict_array.emplace_back(dict_chunk.dictionary()); } auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped); /// We should convert read column to ColumnUnique. auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn(); @@ -468,9 +483,33 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( # undef DISPATCH // TODO: read JSON as a string? // TODO: read UUID as a string? + case arrow::Type::NA: + { + if (allow_null_type) + { + auto type = std::make_shared(); + auto column = ColumnNothing::create(arrow_column->length()); + return {std::move(column), type, column_name}; + } + [[fallthrough]]; + } default: - throw Exception(ErrorCodes::UNKNOWN_TYPE, - "Unsupported {} type '{}' of an input column '{}'.", format_name, arrow_column->type()->name(), column_name); + { + if (skip_columns_with_unsupported_types) + { + skipped = true; + return {}; + } + + throw Exception( + ErrorCodes::UNKNOWN_TYPE, + "Unsupported {} type '{}' of an input column '{}'. If it happens during schema inference and you want to skip columns with " + "unsupported types, you can enable setting input_format_{}_skip_columns_with_unsupported_types_in_schema_inference", + format_name, + arrow_column->type()->name(), + column_name, + boost::algorithm::to_lower_copy(format_name)); + } } } @@ -484,7 +523,8 @@ static void checkStatus(const arrow::Status & status, const String & column_name throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; } -Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, const Block * hint_header) + +Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, bool skip_columns_with_unsupported_types, const Block * hint_header) { ColumnsWithTypeAndName sample_columns; std::unordered_set nested_table_names; @@ -508,9 +548,10 @@ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, arrow::ArrayVector array_vector = {arrow_array}; auto arrow_column = std::make_shared(array_vector); std::unordered_map> dict_values; - ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values, false); - - sample_columns.emplace_back(std::move(sample_column)); + bool skipped = false; + ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values, false, false, skip_columns_with_unsupported_types, skipped); + if (!skipped) + sample_columns.emplace_back(std::move(sample_column)); } return Block(std::move(sample_columns)); } @@ -544,6 +585,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & UInt64 num_rows = name_to_column_ptr.begin()->second->length(); columns_list.reserve(header.rows()); std::unordered_map nested_tables; + bool skipped = false; for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); @@ -558,7 +600,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (!nested_tables.contains(nested_table_name)) { std::shared_ptr arrow_column = name_to_column_ptr[nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values, true)}; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values, true, true, false, skipped)}; Block block(cols); nested_tables[nested_table_name] = std::make_shared(Nested::flatten(block)); } @@ -586,7 +628,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (read_from_nested) column = nested_tables[nested_table_name]->getByName(header_column.name); else - column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values, true); + column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values, true, true, false, skipped); try { diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index cf4f6bb3ff3..cc5852691e0 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -36,7 +36,11 @@ public: /// Transform arrow schema to ClickHouse header. If hint_header is provided, /// we will skip columns in schema that are not in hint_header. - static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name, const Block * hint_header = nullptr); + static Block arrowSchemaToCHHeader( + const arrow::Schema & schema, + const std::string & format_name, + bool skip_columns_with_unsupported_types = false, + const Block * hint_header = nullptr); private: const Block & header; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index a372df41344..29429650c19 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -924,12 +924,12 @@ void registerInputFormatAvro(FormatFactory & factory) void registerAvroSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Avro", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("Avro", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, false, settings); }); - factory.registerSchemaReader("AvroConfluent", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("AvroConfluent", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, true, settings); }); diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 6918220feb4..d3de2fbf494 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -95,7 +95,7 @@ void BinaryFormatReader::skipField(size_t file_column) } BinaryWithNamesAndTypesSchemaReader::BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : FormatWithNamesAndTypesSchemaReader(in_, 0, true, true, &reader), reader(in_, format_settings_) + : FormatWithNamesAndTypesSchemaReader(in_, format_settings_, true, true, &reader), reader(in_, format_settings_) { } @@ -119,7 +119,7 @@ void registerInputFormatRowBinary(FormatFactory & factory) void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("RowBinaryWithNamesAndTypes", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("RowBinaryWithNamesAndTypes", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 216ec6b295a..f246d5c0a35 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include @@ -259,16 +258,15 @@ bool CSVFormatReader::readField( } -CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_) +CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_) : FormatWithNamesAndTypesSchemaReader( in_, - format_setting_.max_rows_to_read_for_schema_inference, + format_setting_, with_names_, with_types_, &reader, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV)) , reader(in_, format_setting_) - , context(context_) { } @@ -279,7 +277,7 @@ DataTypes CSVSchemaReader::readRowAndGetDataTypes() return {}; auto fields = reader.readRow(); - return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV, context); + return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV); } @@ -382,9 +380,9 @@ void registerCSVSchemaReader(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerSchemaReader(format_name, [with_names, with_types](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + factory.registerSchemaReader(format_name, [with_names, with_types](ReadBuffer & buf, const FormatSettings & settings) { - return std::make_shared(buf, with_names, with_types, settings, context); + return std::make_shared(buf, with_names, with_types, settings); }); }; diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index ad9f6c4e492..ee45264d573 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -74,13 +74,12 @@ public: class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader { public: - CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_); + CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_); private: DataTypes readRowAndGetDataTypes() override; CSVFormatReader reader; - ContextPtr context; }; } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index d2e0d6e21a9..74c5fb1945a 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -289,17 +289,16 @@ void CustomSeparatedFormatReader::setReadBuffer(ReadBuffer & in_) } CustomSeparatedSchemaReader::CustomSeparatedSchemaReader( - ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_) + ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_) : FormatWithNamesAndTypesSchemaReader( buf, - format_setting_.max_rows_to_read_for_schema_inference, + format_setting_, with_names_, with_types_, &reader, getDefaultDataTypeForEscapingRule(format_setting_.custom.escaping_rule)) , buf(in_) , reader(buf, ignore_spaces_, updateFormatSettings(format_setting_)) - , context(context_) { } @@ -315,7 +314,7 @@ DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes() first_row = false; auto fields = reader.readRow(); - return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), context); + return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule()); } void registerInputFormatCustomSeparated(FormatFactory & factory) @@ -343,9 +342,9 @@ void registerCustomSeparatedSchemaReader(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerSchemaReader(format_name, [with_names, with_types, ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + factory.registerSchemaReader(format_name, [with_names, with_types, ignore_spaces](ReadBuffer & buf, const FormatSettings & settings) { - return std::make_shared(buf, with_names, with_types, ignore_spaces, settings, context); + return std::make_shared(buf, with_names, with_types, ignore_spaces, settings); }); }; diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index a2f4509d307..d9e62a1b8e9 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -92,14 +92,13 @@ private: class CustomSeparatedSchemaReader : public FormatWithNamesAndTypesSchemaReader { public: - CustomSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_); + CustomSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_); private: DataTypes readRowAndGetDataTypes() override; PeekableReadBuffer buf; CustomSeparatedFormatReader reader; - ContextPtr context; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index c087749d8d8..15b31eec11a 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -182,7 +182,7 @@ bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & } JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) - : FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_) + : FormatWithNamesAndTypesSchemaReader(in_, format_settings_, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_) { } @@ -231,7 +231,7 @@ void registerJSONCompactEachRowSchemaReader(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerSchemaReader(format_name, [=](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader(format_name, [=](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, with_names, with_types, json_strings, settings); }); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 549fd7a6113..19a97f50984 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -382,12 +382,12 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory void registerJSONEachRowSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("JSONEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("JSONEachRow", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_unique(buf, false, settings); }); - factory.registerSchemaReader("JSONStringsEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("JSONStringsEachRow", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_unique(buf, true, settings); }); diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 607e6f36767..36e7a56ebea 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -414,7 +414,7 @@ void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_) } MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference), buf(in_), number_of_columns(format_settings_.msgpack.number_of_columns) + : IRowSchemaReader(buf, format_settings_), buf(in_), number_of_columns(format_settings_.msgpack.number_of_columns) { if (!number_of_columns) throw Exception(ErrorCodes::BAD_ARGUMENTS, "You must specify setting input_format_msgpack_number_of_columns to extract table schema from MsgPack data"); @@ -535,7 +535,7 @@ void registerInputFormatMsgPack(FormatFactory & factory) void registerMsgPackSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("MsgPack", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("MsgPack", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index bd95cfd6376..c1dc60022f5 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -133,7 +133,7 @@ void registerOutputFormatNative(FormatFactory & factory) void registerNativeSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings &, ContextPtr) + factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings &) { return std::make_shared(buf); }); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index aa9f7874ae8..e93897edfbe 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -3,6 +3,7 @@ #if USE_ORC #include +#include #include #include #include @@ -187,8 +188,9 @@ NamesAndTypesList ORCSchemaReader::readSchema() std::shared_ptr schema; std::atomic is_stopped = 0; getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); - auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "ORC"); - return header.getNamesAndTypesList(); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( + *schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference); + return getNamesAndRecursivelyNullableTypes(header); } void registerInputFormatORC(FormatFactory & factory) @@ -209,7 +211,7 @@ void registerORCSchemaReader(FormatFactory & factory) { factory.registerSchemaReader( "ORC", - [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 548bf0138f5..f3d81822297 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -4,6 +4,7 @@ #if USE_PARQUET #include +#include #include #include #include @@ -186,8 +187,9 @@ NamesAndTypesList ParquetSchemaReader::readSchema() std::shared_ptr schema; std::atomic is_stopped = 0; getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); - auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "Parquet"); - return header.getNamesAndTypesList(); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( + *schema, "Parquet", format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference); + return getNamesAndRecursivelyNullableTypes(header); } void registerInputFormatParquet(FormatFactory & factory) @@ -208,7 +210,7 @@ void registerParquetSchemaReader(FormatFactory & factory) { factory.registerSchemaReader( "Parquet", - [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 4754b70d375..f18b6b0aaab 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -128,15 +128,14 @@ void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } -RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, ContextPtr context_) +RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : IRowSchemaReader( buf, - format_settings_.max_rows_to_read_for_schema_inference, + format_settings_, getDefaultDataTypeForEscapingRule(format_settings_.regexp.escaping_rule)) , format_settings(format_settings_) , field_extractor(format_settings) , buf(in_) - , context(context_) { } @@ -152,7 +151,7 @@ DataTypes RegexpSchemaReader::readRowAndGetDataTypes() for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i) { String field(field_extractor.getField(i)); - data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule, context)); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule)); } return data_types; @@ -203,9 +202,9 @@ void registerFileSegmentationEngineRegexp(FormatFactory & factory) void registerRegexpSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings) { - return std::make_shared(buf, settings, context); + return std::make_shared(buf, settings); }); } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 04f24bbb3e4..3cc6a3192fd 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -76,7 +76,7 @@ private: class RegexpSchemaReader : public IRowSchemaReader { public: - RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, ContextPtr context_); + RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); private: DataTypes readRowAndGetDataTypes() override; @@ -85,7 +85,6 @@ private: const FormatSettings format_settings; RegexpFieldExtractor field_extractor; PeekableReadBuffer buf; - ContextPtr context; }; } diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index f63d6fa9c46..4c50e4d9b03 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -280,7 +280,7 @@ void registerInputFormatTSKV(FormatFactory & factory) } void registerTSKVSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("TSKV", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("TSKV", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index bb844ec68ea..b6c9438a57c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -235,7 +235,7 @@ TabSeparatedSchemaReader::TabSeparatedSchemaReader( ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( in_, - format_settings_.max_rows_to_read_for_schema_inference, + format_settings_, with_names_, with_types_, &reader, @@ -280,7 +280,7 @@ void registerTSVSchemaReader(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerSchemaReader(format_name, [with_names, with_types, is_raw](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader(format_name, [with_names, with_types, is_raw](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, with_names, with_types, is_raw, settings); }); diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index 06d6ba06bcc..a1f70730b39 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -453,14 +453,12 @@ TemplateSchemaReader::TemplateSchemaReader( const ParsedTemplateFormatString & format_, const ParsedTemplateFormatString & row_format_, std::string row_between_delimiter, - const FormatSettings & format_settings_, - ContextPtr context_) - : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference) + const FormatSettings & format_settings_) + : IRowSchemaReader(buf, format_settings_, nullptr, getDefaultDataTypeForEscapingRules(row_format_.escaping_rules)) , buf(in_) , format(format_) , row_format(row_format_) , format_settings(format_settings_) - , context(context_) , format_reader(buf, ignore_spaces_, format, row_format, row_between_delimiter, format_settings) { setColumnNames(row_format.column_names); @@ -489,7 +487,7 @@ DataTypes TemplateSchemaReader::readRowAndGetDataTypes() format_settings.csv.delimiter = row_format.delimiters[i + 1].empty() ? format_settings.csv.delimiter : row_format.delimiters[i + 1].front(); field = readFieldByEscapingRule(buf, row_format.escaping_rules[i], format_settings); - data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i], context)); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i])); } format_reader.skipRowEndDelimiter(); @@ -564,12 +562,12 @@ void registerTemplateSchemaReader(FormatFactory & factory) { for (bool ignore_spaces : {false, true}) { - factory.registerSchemaReader(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + factory.registerSchemaReader(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [ignore_spaces](ReadBuffer & buf, const FormatSettings & settings) { size_t index = 0; auto idx_getter = [&](const String &) -> std::optional { return index++; }; auto row_format = fillRowFormat(settings, idx_getter, false); - return std::make_shared(buf, ignore_spaces, fillResultSetFormat(settings), row_format, settings.template_settings.row_between_delimiter, settings, context); + return std::make_shared(buf, ignore_spaces, fillResultSetFormat(settings), row_format, settings.template_settings.row_between_delimiter, settings); }); } } diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index b5ced707ace..ab7043f057e 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -116,8 +116,7 @@ public: const ParsedTemplateFormatString & format_, const ParsedTemplateFormatString & row_format_, std::string row_between_delimiter, - const FormatSettings & format_settings_, - ContextPtr context_); + const FormatSettings & format_settings_); DataTypes readRowAndGetDataTypes() override; @@ -126,7 +125,6 @@ private: const ParsedTemplateFormatString format; const ParsedTemplateFormatString row_format; FormatSettings format_settings; - ContextPtr context; TemplateFormatReader format_reader; bool first_row = true; }; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index bf8feb077ed..e8b4c69bd19 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -571,8 +572,8 @@ void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } -ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, ContextPtr context_) - : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference), buf(in_), context(context_) +ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : IRowSchemaReader(buf, format_settings_), buf(in_), format_settings(format_settings_) { } @@ -589,38 +590,25 @@ DataTypes ValuesSchemaReader::readRowAndGetDataTypes() return {}; assertChar('(', buf); - PeekableReadBufferCheckpoint checkpoint(buf); - skipToNextRow(&buf, 0, 1); - buf.makeContinuousMemoryFromCheckpointToPos(); - buf.rollbackToCheckpoint(); - - Tokens tokens(buf.position(), buf.buffer().end()); - IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth); - + skipWhitespaceIfAny(buf); DataTypes data_types; - bool finish = false; - while (!finish) + String value; + while (!buf.eof() && *buf.position() != ')') { - Expected expected; - ASTPtr ast; + if (!data_types.empty()) + { + skipWhitespaceIfAny(buf); + assertChar(',', buf); + skipWhitespaceIfAny(buf); + } - bool parsed = parser.parse(token_iterator, ast, expected); - /// Consider delimiter after value (',' or ')') as part of expression - parsed &= token_iterator->type == TokenType::Comma || token_iterator->type == TokenType::ClosingRoundBracket; - - if (!parsed) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot parse expression here: {}, token: {}", - String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), String(token_iterator.get().begin, token_iterator.get().end)); - - std::pair result = evaluateConstantExpression(ast, context); - data_types.push_back(generalizeDataType(result.second)); - - if (token_iterator->type == TokenType::ClosingRoundBracket) - finish = true; - ++token_iterator; - buf.position() = const_cast(token_iterator->begin); + readQuotedFieldIntoString(value, buf); + auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted); + data_types.push_back(std::move(type)); } + assertChar(')', buf); + skipWhitespaceIfAny(buf); if (!buf.eof() && *buf.position() == ',') ++buf.position(); @@ -642,9 +630,9 @@ void registerInputFormatValues(FormatFactory & factory) void registerValuesSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Values", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + factory.registerSchemaReader("Values", [](ReadBuffer & buf, const FormatSettings & settings) { - return std::make_shared(buf, settings, context); + return std::make_shared(buf, settings); }); } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index e1521955472..77967181566 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -97,13 +97,13 @@ private: class ValuesSchemaReader : public IRowSchemaReader { public: - ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, ContextPtr context_); + ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); private: DataTypes readRowAndGetDataTypes() override; PeekableReadBuffer buf; - ContextPtr context; + const FormatSettings format_settings; ParserExpression parser; bool first_row = true; }; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 7720b01dc74..8ab8b55c9c2 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -295,12 +295,12 @@ void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader( ReadBuffer & in_, - size_t max_rows_to_read_, + const FormatSettings & format_settings, bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, DataTypePtr default_type_) - : IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) + : IRowSchemaReader(in_, format_settings, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 25ffc8d6de2..8d24d23186b 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -124,7 +124,7 @@ class FormatWithNamesAndTypesSchemaReader : public IRowSchemaReader public: FormatWithNamesAndTypesSchemaReader( ReadBuffer & in, - size_t max_rows_to_read_, + const FormatSettings & format_settings, bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index f46e3bee101..8fca786df05 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -1,17 +1,17 @@ TSV -c1 Nullable(String) +c1 Nullable(Float64) c2 Nullable(String) -c3 Nullable(String) -c4 Nullable(String) -42 Some string [1, 2, 3, 4] (1, 2, 3) -42 abcd [] (4, 5, 6) +c3 Array(Nullable(Float64)) +c4 Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +42 Some string [1,2,3,4] (1,2,3) +42 abcd [] (4,5,6) TSVWithNames -number Nullable(String) +number Nullable(Float64) string Nullable(String) -array Nullable(String) -tuple Nullable(String) -42 Some string [1, 2, 3, 4] (1, 2, 3) -42 abcd [] (4, 5, 6) +array Array(Nullable(Float64)) +tuple Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +42 Some string [1,2,3,4] (1,2,3) +42 abcd [] (4,5,6) CSV c1 Nullable(Float64) c2 Nullable(String) @@ -74,12 +74,12 @@ s1 [] 1 \N [3] \N TSKV b Nullable(String) -c Nullable(String) -a Nullable(String) -s1 \N 1 +c Array(Nullable(Float64)) +a Nullable(Float64) +s1 [] 1 } [2] 2 -\N \N \N -\N \N \N +\N [] \N +\N [] \N \N [3] \N Values c1 Nullable(Float64) @@ -96,7 +96,7 @@ c5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(Strin 42.42 \N [1,NULL,3] (1,NULL) ([1,2],[(3,'4'),(5,'6')]) \N Some string [10] (1,2) ([],[]) Regexp -c1 Nullable(String) +c1 Nullable(Float64) c2 Nullable(String) c3 Nullable(String) 42 Some string 1 [([1, 2, 3], String 1), ([], String 1)] diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference index d3d2d86d696..b0ec4bef499 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -1,137 +1,137 @@ Arrow -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(UInt32) +int64 Nullable(Int64) +uint64 Nullable(UInt64) 0 0 0 0 0 0 0 0 -1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 -date UInt16 -date32 Date32 +date Nullable(UInt16) +date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 -str String -fixed_string String +str Nullable(String) +fixed_string Nullable(String) Str: 0 100 Str: 1 200 -array Array(UInt64) -tuple Tuple(`tuple.0` UInt64, `tuple.1` String) -map Map(String, UInt64) +array Array(Nullable(UInt64)) +tuple Tuple(Nullable(UInt64), Nullable(String)) +map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) -nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +nested1 Array(Tuple(Array(Nullable(UInt64)), Map(String, Nullable(UInt64)))) +nested2 Tuple(Tuple(Array(Array(Nullable(UInt64))), Map(UInt64, Array(Tuple(Nullable(UInt64), Nullable(String))))), Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ArrowStream -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(UInt32) +int64 Nullable(Int64) +uint64 Nullable(UInt64) 0 0 0 0 0 0 0 0 -1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 -date UInt16 -date32 Date32 +date Nullable(UInt16) +date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 -str String -fixed_string String +str Nullable(String) +fixed_string Nullable(String) Str: 0 100 Str: 1 200 -array Array(UInt64) -tuple Tuple(`tuple.0` UInt64, `tuple.1` String) -map Map(String, UInt64) +array Array(Nullable(UInt64)) +tuple Tuple(Nullable(UInt64), Nullable(String)) +map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) -nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +nested1 Array(Tuple(Array(Nullable(UInt64)), Map(String, Nullable(UInt64)))) +nested2 Tuple(Tuple(Array(Array(Nullable(UInt64))), Map(UInt64, Array(Tuple(Nullable(UInt64), Nullable(String))))), Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Parquet -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 Int64 -int64 Int64 -uint64 UInt64 +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(Int64) +int64 Nullable(Int64) +uint64 Nullable(UInt64) 0 0 0 0 0 0 0 0 -1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 -date UInt16 -date32 Date32 +date Nullable(UInt16) +date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 -str String -fixed_string String +str Nullable(String) +fixed_string Nullable(String) Str: 0 100 Str: 1 200 -array Array(UInt64) -tuple Tuple(`tuple.0` UInt64, `tuple.1` String) -map Map(String, UInt64) +array Array(Nullable(UInt64)) +tuple Tuple(Nullable(UInt64), Nullable(String)) +map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) -nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +nested1 Array(Tuple(Array(Nullable(UInt64)), Map(String, Nullable(UInt64)))) +nested2 Tuple(Tuple(Array(Array(Nullable(UInt64))), Map(UInt64, Array(Tuple(Nullable(UInt64), Nullable(String))))), Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ORC -int8 Int8 -uint8 Int8 -int16 Int16 -uint16 Int16 -int32 Int32 -uint32 Int32 -int64 Int64 -uint64 Int64 +int8 Nullable(Int8) +uint8 Nullable(Int8) +int16 Nullable(Int16) +uint16 Nullable(Int16) +int32 Nullable(Int32) +uint32 Nullable(Int32) +int64 Nullable(Int64) +uint64 Nullable(Int64) 0 0 0 0 0 0 0 0 -1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 -date Date32 -date32 Date32 +date Nullable(Date32) +date32 Nullable(Date32) 1970-01-01 1970-01-01 1970-01-02 1970-01-02 -str String -fixed_string String +str Nullable(String) +fixed_string Nullable(String) Str: 0 100 Str: 1 200 -array Array(Int64) -tuple Tuple(`tuple.0` Int64, `tuple.1` String) -map Map(String, Int64) +array Array(Nullable(Int64)) +tuple Tuple(Nullable(Int64), Nullable(String)) +map Map(String, Nullable(Int64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`nested1.0` Array(Int64), `nested1.1` Map(String, Int64))) -nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(Int64)), `nested2.0.1` Map(Int64, Array(Tuple(`nested2.0.1.0` Int64, `nested2.0.1.1` String)))), `nested2.1` Int8) +nested1 Array(Tuple(Array(Nullable(Int64)), Map(String, Nullable(Int64)))) +nested2 Tuple(Tuple(Array(Array(Nullable(Int64))), Map(Int64, Array(Tuple(Nullable(Int64), Nullable(String))))), Nullable(Int8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Native diff --git a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference new file mode 100644 index 00000000000..debc5c58936 --- /dev/null +++ b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference @@ -0,0 +1,40 @@ +Arrow +x Nullable(UInt64) +arr1 Array(Nullable(UInt64)) +arr2 Array(Array(Nullable(String))) +arr3 Array(Tuple(Nullable(String), Nullable(UInt64))) +0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] +\N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] +2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] +\N [NULL,4] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,3)] +4 [4,5] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,4)] +ArrowStream +x Nullable(UInt64) +arr1 Array(Nullable(UInt64)) +arr2 Array(Array(Nullable(String))) +arr3 Array(Tuple(Nullable(String), Nullable(UInt64))) +0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] +\N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] +2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] +\N [NULL,4] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,3)] +4 [4,5] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,4)] +Parquet +x Nullable(UInt64) +arr1 Array(Nullable(UInt64)) +arr2 Array(Array(Nullable(String))) +arr3 Array(Tuple(Nullable(String), Nullable(UInt64))) +0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] +\N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] +2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] +\N [NULL,4] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,3)] +4 [4,5] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,4)] +ORC +x Nullable(Int64) +arr1 Array(Nullable(Int64)) +arr2 Array(Array(Nullable(String))) +arr3 Array(Tuple(Nullable(String), Nullable(Int64))) +0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] +\N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] +2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] +\N [NULL,4] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,3)] +4 [4,5] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,4)] diff --git a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh new file mode 100755 index 00000000000..1b6999e3f09 --- /dev/null +++ b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02242.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +for format in Arrow ArrowStream Parquet ORC +do + echo $format + $CLICKHOUSE_CLIENT -q "select number % 2 ? NULL : number as x, [number % 2 ? NULL : number, number + 1] as arr1, [[NULL, 'String'], [NULL], []] as arr2, [(NULL, NULL), ('String', NULL), (NULL, number)] as arr3 from numbers(5) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" +done + +rm $DATA_FILE diff --git a/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.reference b/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.reference new file mode 100644 index 00000000000..f599e28b8ab --- /dev/null +++ b/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.reference @@ -0,0 +1 @@ +10 diff --git a/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.sh b/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.sh new file mode 100755 index 00000000000..cc8db7fb316 --- /dev/null +++ b/tests/queries/0_stateless/02243_arrow_read_null_type_to_nullable_column.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02243" +$CLICKHOUSE_CLIENT -q "create table test_02243 (image_path Nullable(String), + caption Nullable(String), + NSFW Nullable(String), + similarity Nullable(Float64), + LICENSE Nullable(String), + url Nullable(String), + key Nullable(UInt64), + shard_id Nullable(UInt64), + status Nullable(String), + error_message Nullable(String), + width Nullable(UInt32), + height Nullable(UInt32), + exif Nullable(String), + original_width Nullable(UInt32), + original_height Nullable(UInt32)) engine=Memory" + +cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT --stacktrace -q "insert into test_02243 format Parquet" + +$CLICKHOUSE_CLIENT -q "select count() from test_02243" +$CLICKHOUSE_CLIENT -q "drop table test_02243" diff --git a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference new file mode 100644 index 00000000000..d237caf630f --- /dev/null +++ b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.reference @@ -0,0 +1,8 @@ +x Nullable(String) +y Nullable(Float64) +x Nullable(String) +y Nullable(Float64) +x Nullable(String) +y Nullable(Float64) +x Nullable(String) +y Nullable(Float64) diff --git a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql new file mode 100644 index 00000000000..cf9f312ab0c --- /dev/null +++ b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql @@ -0,0 +1,12 @@ +insert into function file('test_02244', 'TSV', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; +desc file('test_02244', 'TSV') settings column_names_for_schema_inference='x,y'; + +insert into function file('test_02244', 'CSV', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; +desc file('test_02244', 'CSV') settings column_names_for_schema_inference='x,y'; + +insert into function file('test_02244', 'JSONCompactEachRow', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; +desc file('test_02244', 'JSONCompactEachRow') settings column_names_for_schema_inference='x,y'; + +insert into function file('test_02244', 'Values', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; +desc file('test_02244', 'Values') settings column_names_for_schema_inference='x,y'; + diff --git a/tests/queries/0_stateless/02245_parquet_skip_unknown_type.reference b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.reference new file mode 100644 index 00000000000..4f9cde534f0 --- /dev/null +++ b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.reference @@ -0,0 +1,16 @@ +OK +image_path Nullable(String) +caption Nullable(String) +NSFW Nullable(String) +similarity Nullable(Float64) +LICENSE Nullable(String) +url Nullable(String) +key Nullable(Int64) +shard_id Nullable(Int64) +status Nullable(String) +width Nullable(Int64) +height Nullable(Int64) +exif Nullable(String) +original_width Nullable(Int64) +original_height Nullable(Int64) +10 diff --git a/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh new file mode 100755 index 00000000000..005c089e434 --- /dev/null +++ b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02245.parquet +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +cp $CUR_DIR/data_parquet_bad_column/metadata_0.parquet $DATA_FILE + + +$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet)" 2>&1 | grep -qF "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1" +$CLICKHOUSE_CLIENT -q "select count(*) from file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1" + diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference new file mode 100644 index 00000000000..c245f13fdbe --- /dev/null +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -0,0 +1,107 @@ +TSV +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Nullable(Float64)) +c4 Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64)) +42 Some string [1,2,3,4] (1,2,3) +42 abcd [] (4,5,6) +c1 Nullable(String) +[({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] +[] +[({}, [], 0)] +[({}, [NULL], NULL)] +[({}, [\'String3\'], NULL)] +[({\'key3\': NULL}, []), NULL] +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +[({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] +[] +[({},[],0)] +[({},[NULL],NULL)] +[({},['String3'],NULL)] +[({'key3':NULL},[],NULL)] +c1 Nullable(Bool) +true +false +\N +c1 Array(Nullable(Bool)) +[true,NULL] +[] +[NULL] +[false] +c1 Nullable(String) +[] +c1 Nullable(String) +{} +c1 Nullable(String) +() +c1 Nullable(String) +[1, 2, 3 +c1 Nullable(String) +[(1, 2, 3 4)] +c1 Nullable(String) +[1, 2, 3 + 4] +c1 Nullable(String) +(1, 2, +c1 Nullable(String) +[1, Some trash, 42.2] +c1 Nullable(String) +[1, \'String\', {\'key\' : 2}] +c1 Nullable(String) +{\'key\' : 1, [1] : 10} +c1 Nullable(String) +{}{} +c1 Nullable(String) +[1, 2, 3 +c1 Nullable(String) +[abc, def] +c1 Array(Nullable(String)) +['abc','def'] +c1 Nullable(String) +[\'string] +c1 Nullable(String) +\'string +c1 Nullable(Float64) +42.42 +c1 Nullable(String) +42.42sometrash +c1 Nullable(String) +[42.42sometrash, 42.42] + +CSV +c1 Nullable(String) +c2 Nullable(String) +c3 Array(Nullable(Float64)) +c4 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +42 Some string [1,2,3,4] [(1,2,3)] +42\\ abcd [] [(4,5,6)] +c1 Nullable(String) +[({\'key\' : 42.42}, [\'String\', \'String2\'], 42.42), ({}, [], -42), ({\'key2\' : NULL}, [NULL], NULL)] +[] +[({}, [], 0)] +[({}, [NULL], NULL)] +[({}, [\'String3\'], NULL)] +[({\'key3\': NULL}, []), NULL] +c1 Array(Tuple(Map(String, Nullable(Float64)), Array(Nullable(String)), Nullable(Float64))) +[({'key':42.42},['String','String2'],42.42),({},[],-42),({'key2':NULL},[NULL],NULL)] +[] +[({},[],0)] +[({},[NULL],NULL)] +[({},['String3'],NULL)] +[({'key3':NULL},[],NULL)] +c1 Nullable(Bool) +true +false +\N +c1 Array(Nullable(Bool)) +[true,NULL] +[] +[NULL] +[false] +c1 Nullable(String) +(1, 2, 3) +c1 Nullable(String) +123.123 +c1 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +[(1,2,3)] +c1 Array(Tuple(Nullable(Float64), Nullable(Float64), Nullable(Float64))) +[(1,2,3)] diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh new file mode 100755 index 00000000000..6589765f739 --- /dev/null +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh @@ -0,0 +1,220 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02149.data +DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME + +touch $DATA_FILE + +echo "TSV" + +echo -e "42\tSome string\t[1, 2, 3, 4]\t(1, 2, 3) +42\tabcd\t[]\t(4, 5, 6)" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)] +[] +[({}, [], 0)] +[({}, [NULL], NULL)] +[({}, ['String3'], NULL)] +[({'key3': NULL}, []), NULL]"> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV') settings input_format_tsv_use_best_effort_in_schema_inference=false" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV') settings input_format_tsv_use_best_effort_in_schema_inference=false" + + +echo -e "[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)] +[] +[({}, [], 0)] +[({}, [NULL], NULL)] +[({}, ['String3'], NULL)] +[({'key3': NULL}, [], NULL)]"> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "true +false +\N" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[true, NULL] +[] +[NULL] +[false]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "{}" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "()" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[1, 2, 3" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[(1, 2, 3 4)]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[1, 2, 3 + 4]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "(1, 2," > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[1, Some trash, 42.2]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[1, 'String', {'key' : 2}]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "{'key' : 1, [1] : 10}" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "{}{}" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[1, 2, 3" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[abc, def]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "['abc', 'def']" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "['string]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "'string" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "42.42" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "42.42sometrash" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo -e "[42.42sometrash, 42.42]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + + +echo +echo "CSV" + +echo -e "42,Some string,'[1, 2, 3, 4]','[(1, 2, 3)]' +42\,abcd,'[]','[(4, 5, 6)]'" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)]\" +'[]' +'[({}, [], 0)]' +'[({}, [NULL], NULL)]' +\"[({}, ['String3'], NULL)]\" +\"[({'key3': NULL}, []), NULL]\""> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV') settings input_format_csv_use_best_effort_in_schema_inference=false" + +echo -e "\"[({'key' : 42.42}, ['String', 'String2'], 42.42), ({}, [], -42), ({'key2' : NULL}, [NULL], NULL)]\" +'[]' +'[({}, [], 0)]' +'[({}, [NULL], NULL)]' +\"[({}, ['String3'], NULL)]\" +\"[({'key3': NULL}, [], NULL)]\""> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "true +false +\N" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "'[true, NULL]' +'[]' +'[NULL]' +'[false]'" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + + +echo -e "'(1, 2, 3)'"> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "'123.123'"> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "'[(1, 2, 3)]'"> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "\"[(1, 2, 3)]\""> $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + + From abc020a502962239cd3cf5f0585684a16d0f7fc5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 24 Mar 2022 13:08:58 +0000 Subject: [PATCH 105/372] Clean up --- src/Formats/EscapingRuleUtils.cpp | 1 + src/Formats/EscapingRuleUtils.h | 17 +++++++++++------ src/Formats/FormatSettings.h | 2 +- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 08b34ebb0fc..b3e36e9c14a 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -429,6 +429,7 @@ static DataTypePtr determineDataTypeForSingleFieldImpl(ReadBuffer & buf) if (checkStringCaseInsensitive("NULL", buf)) return std::make_shared(); + /// Number Float64 tmp; if (tryReadFloatText(tmp, buf)) return std::make_shared(); diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h index 3c7c768c003..b4f609cd9a6 100644 --- a/src/Formats/EscapingRuleUtils.h +++ b/src/Formats/EscapingRuleUtils.h @@ -43,12 +43,17 @@ String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule es /// - For JSON escaping rule we can use JSON parser to parse a single field /// and then convert JSON type of this field to ClickHouse type. /// - For CSV escaping rule we can do the next: -/// - If the field is an unquoted string, then we could try to evaluate it -/// as a constant expression, and if it fails, treat it as a String. -/// - If the field is a string in quotes, then we can try to evaluate -/// expression inside quotes as a constant expression, and if it fails or -/// the result is a number (we don't parse numbers in quotes) we treat it as a String. -/// - For TSV and TSVRaw we treat each field as a String (TODO: try to use some tweaks and heuristics here) +/// - If the field is an unquoted string, then we try to parse it as s number, +/// and if we cannot, treat it as a String. +/// - If the field is a string in quotes, then we try to use some +/// tweaks and heuristics to determine the type inside quotes, and if we can't or +/// the result is a number or tuple (we don't parse numbers in quotes and don't +/// support tuples in CSV) we treat it as a String. +/// - If input_format_csv_use_best_effort_in_schema_inference is disabled, we +/// treat everything as a string. +/// - For TSV and TSVRaw we try to use some tweaks and heuristics to determine the type +/// of value if setting input_format_tsv_use_best_effort_in_schema_inference is enabled, +/// otherwise we treat everything as a string. DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule); DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6ecd04536a6..4b39d255110 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -112,7 +112,7 @@ struct FormatSettings char fields_delimiter = '\x01'; char collection_items_delimiter = '\x02'; char map_keys_delimiter = '\x03'; - Names input_field_names = {""}; + Names input_field_names; } hive_text; struct Custom From db58a87fcb541621c5999bd81b3763974c016069 Mon Sep 17 00:00:00 2001 From: Meena Renganathan Date: Thu, 24 Mar 2022 06:36:03 -0700 Subject: [PATCH 106/372] Updated the curl-cmake/curl_config.h to resolve the build error in MacOS --- contrib/curl-cmake/curl_config.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/curl-cmake/curl_config.h b/contrib/curl-cmake/curl_config.h index 1efdd88600f..268b2a98287 100644 --- a/contrib/curl-cmake/curl_config.h +++ b/contrib/curl-cmake/curl_config.h @@ -48,3 +48,6 @@ #define ENABLE_IPV6 #define USE_OPENSSL #define USE_THREADS_POSIX + +#define HAVE_BOOL_T 1 +#define HAVE_STDBOOL_H 1 From d6ab6de30a3594c435754f4a292f5e2f453ad577 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 24 Mar 2022 15:32:08 +0100 Subject: [PATCH 107/372] Minor changes --- src/Common/FileSegment.cpp | 3 +-- src/Disks/DiskWebServer.h | 7 +++++-- src/Disks/IDisk.h | 13 +++++------- src/Disks/IDiskRemote.h | 2 -- src/IO/WriteBufferFromS3.cpp | 1 + .../StorageSystemRemoteFilesystemCache.cpp | 14 ++++++++----- ...emote_filesystem_cache_on_insert.reference | 20 ++++++++++--------- ...2241_remote_filesystem_cache_on_insert.sql | 6 +++--- 8 files changed, 35 insertions(+), 31 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index ce1253a6f5c..1dda76ba4b5 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -161,8 +161,7 @@ String FileSegment::getDownloader() const bool FileSegment::isDownloader() const { std::lock_guard segment_lock(mutex); - LOG_TEST(log, "Checking for current downloader. Caller: {}, downloader: {}, current state: {}", getCallerId(), downloader_id, stateToString(download_state)); - return getCallerId() == downloader_id; + return getCallerIdImpl(true) == downloader_id; } FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() diff --git a/src/Disks/DiskWebServer.h b/src/Disks/DiskWebServer.h index 5dbfbe8994a..94ba32939da 100644 --- a/src/Disks/DiskWebServer.h +++ b/src/Disks/DiskWebServer.h @@ -77,8 +77,7 @@ public: UInt64 getTotalSpace() const final override { return std::numeric_limits::max(); } UInt64 getAvailableSpace() const final override { return std::numeric_limits::max(); } - - UInt64 getUnreservedSpace() const final override { return std::numeric_limits::max(); } +UInt64 getUnreservedSpace() const final override { return std::numeric_limits::max(); } /// Read-only part @@ -165,6 +164,10 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk {} is read-only", getName()); } + std::vector getRemotePaths(const String &) const override { return {}; } + + void getRemotePathsRecursive(const String &, std::vector &) override {} + /// Create part void createFile(const String &) final override {} diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 0f6e31dd0f7..81cdf47e1fb 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -205,16 +205,13 @@ public: virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); } - virtual String getCacheBasePath() const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getCacheBasePath() not implemented fro disk: {}`", getType()); - } + virtual String getCacheBasePath() const { return ""; } - /// Returnes a list of paths because for Log family engines - /// there might be multiple files in remote fs for single clickhouse file. + /// Returns a list of paths because for Log family engines there might be + /// multiple files in remote fs for single clickhouse file. virtual std::vector getRemotePaths(const String &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePaths() not implemented fro disk: {}`", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePaths() not implemented for disk: {}`", getType()); } /// For one local path there might be multiple remote paths in case of Log family engines. @@ -222,7 +219,7 @@ public: virtual void getRemotePathsRecursive(const String &, std::vector &) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented fro disk: {}`", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented for disk: {}`", getType()); } struct RemoveRequest diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index a1d6092a286..6b16a1f753c 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -68,8 +68,6 @@ public: String getCacheBasePath() const final override; - /// Returnes a list of paths because for Log family engines - /// there might be multiple files in remote fs for single clickhouse file. std::vector getRemotePaths(const String & path) const final override; void getRemotePathsRecursive(const String & path, std::vector & paths_map) override; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 9ed008907c9..dbd8e1cf743 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -34,6 +34,7 @@ const int S3_WARN_MAX_PARTS = 10000; namespace ErrorCodes { extern const int S3_ERROR; + extern const int LOGICAL_ERROR; } struct WriteBufferFromS3::UploadPartTask diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp index e37dcb74829..da5865e2043 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp @@ -17,10 +17,12 @@ NamesAndTypesList StorageSystemRemoteFilesystemCache::getNamesAndTypes() return { {"cache_base_path", std::make_shared()}, {"cache_path", std::make_shared()}, - {"file_segment_range", std::make_shared(DataTypes{std::make_shared(), std::make_shared()})}, + {"file_segment_range_begin", std::make_shared()}, + {"file_segment_range_end", std::make_shared()}, {"size", std::make_shared()}, {"state", std::make_shared()}, {"cache_hits", std::make_shared()}, + {"references", std::make_shared()}, }; } @@ -44,10 +46,12 @@ void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, res_columns[1]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset())); const auto & range = file_segment->range(); - res_columns[2]->insert(Tuple({range.left, range.right})); - res_columns[3]->insert(range.size()); - res_columns[4]->insert(FileSegment::stateToString(file_segment->state())); - res_columns[5]->insert(file_segment->hits()); + res_columns[2]->insert(range.left); + res_columns[3]->insert(range.right); + res_columns[4]->insert(range.size()); + res_columns[5]->insert(FileSegment::stateToString(file_segment->state())); + res_columns[6]->insert(file_segment->hits()); + res_columns[7]->insert(file_segment.use_count()); } } } diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 5bc2049204a..941ab9089f7 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -4,18 +4,19 @@ SET remote_fs_cache_on_write_operations=1; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 0 SELECT count() FROM system.remote_filesystem_cache; 0 INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── -file_segment_range: (0,745) -size: 746 -state: DOWNLOADED +file_segment_range_begin: 0 +file_segment_range_end: 745 +size: 746 +state: DOWNLOADED SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 7 SELECT count() FROM system.remote_filesystem_cache; @@ -32,12 +33,13 @@ SELECT count() size FROM system.remote_filesystem_cache; 7 SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── -file_segment_range: (0,1659) -size: 1660 -state: DOWNLOADED +file_segment_range_begin: 0 +file_segment_range_end: 1659 +size: 1660 +state: DOWNLOADED SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 7 SELECT count() FROM system.remote_filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 58f4adb5980..cd255d7df51 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -9,13 +9,13 @@ CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SET SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; SELECT count() FROM system.remote_filesystem_cache; INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; SELECT count() FROM system.remote_filesystem_cache; @@ -33,7 +33,7 @@ SYSTEM DROP REMOTE FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range, size, state FROM (SELECT file_segment_range, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; SELECT count() FROM system.remote_filesystem_cache; From 3b801a4093322dd048f286d410ef7f1d952a972c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 24 Mar 2022 19:16:41 +0100 Subject: [PATCH 108/372] Update src/Processors/Formats/ISchemaReader.cpp Co-authored-by: Vladimir C --- src/Processors/Formats/ISchemaReader.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 17db8865310..a80c56a0449 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -20,6 +20,13 @@ IRowSchemaReader::IRowSchemaReader( { /// column_names_for_schema_inference is a string in format 'column1,column2,column3,...' boost::split(column_names, format_settings.column_names_for_schema_inference, boost::is_any_of(",")); + for (size_t i = 0 ; i < column_names.size() ; ++i) + { + std::string col_name_trimmed = column_names[i]; + boost::trim(col_name_trimmed); + if (!col_name.empty()) + column_names[i] = col_name_trimmed; + } } } From 6a9df9d471190ddf60ec59a9385d821f2678bee0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 24 Mar 2022 19:16:47 +0100 Subject: [PATCH 109/372] Update src/Processors/Formats/ISchemaReader.cpp Co-authored-by: Vladimir C --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index a80c56a0449..a74de0447cb 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -58,7 +58,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() { if (default_type) data_types[i] = default_type; - else if (!default_types.empty() && i < default_types.size() && default_types[i]) + else if (i < default_types.size() && default_types[i]) data_types[i] = default_types[i]; else throw Exception( From 287e1a6efc6d63dce9a111821079e87bc21321f8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 24 Mar 2022 19:16:52 +0100 Subject: [PATCH 110/372] Update src/Processors/Formats/ISchemaReader.cpp Co-authored-by: Vladimir C --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index a74de0447cb..e421692743d 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -93,7 +93,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() { if (default_type) data_types[i] = default_type; - else if (!default_types.empty() && i < default_types.size() && default_types[i]) + else if (i < default_types.size() && default_types[i]) data_types[i] = default_types[i]; else throw Exception( From 1823cac89d24d906c4f5894d9dd8814bbcd694dc Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 24 Mar 2022 19:19:32 +0100 Subject: [PATCH 111/372] Update src/Formats/EscapingRuleUtils.h Co-authored-by: Vladimir C --- src/Formats/EscapingRuleUtils.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h index b4f609cd9a6..1ce04a8d1b7 100644 --- a/src/Formats/EscapingRuleUtils.h +++ b/src/Formats/EscapingRuleUtils.h @@ -43,7 +43,7 @@ String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule es /// - For JSON escaping rule we can use JSON parser to parse a single field /// and then convert JSON type of this field to ClickHouse type. /// - For CSV escaping rule we can do the next: -/// - If the field is an unquoted string, then we try to parse it as s number, +/// - If the field is an unquoted string, then we try to parse it as a number, /// and if we cannot, treat it as a String. /// - If the field is a string in quotes, then we try to use some /// tweaks and heuristics to determine the type inside quotes, and if we can't or From 83f406b722e3a76fcd0adb582153055d37a69a0e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 24 Mar 2022 15:29:29 -0400 Subject: [PATCH 112/372] optimization, INTERPOLATE without expr. list, any column is allowed except WITH FILL --- src/Core/InterpolateDescription.cpp | 63 +++------- src/Core/InterpolateDescription.h | 56 ++------- src/Interpreters/FillingRow.cpp | 93 ++++++++++----- src/Interpreters/FillingRow.h | 28 +---- src/Interpreters/InterpreterSelectQuery.cpp | 38 ++++-- src/Parsers/ExpressionElementParsers.cpp | 12 +- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 34 +++++- src/Processors/QueryPlan/FillingStep.cpp | 9 +- src/Processors/QueryPlan/FillingStep.h | 4 +- .../Transforms/FillingTransform.cpp | 110 ++++++++++-------- src/Processors/Transforms/FillingTransform.h | 6 +- .../0_stateless/02233_interpolate_1.sql | 6 +- 13 files changed, 229 insertions(+), 232 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 176909ca929..e0c3bebd45e 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,57 +7,22 @@ namespace DB { -void dumpInterpolateDescription(const InterpolateDescription & description, const Block & /*header*/, WriteBuffer & out) -{ - bool first = true; - - for (const auto & desc : description) + InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_) + : actions(actions_) { - if (!first) - out << ", "; - first = false; + for (const auto & name_type : actions->getRequiredColumnsWithTypes()) + { + columns_full_set.insert(name_type.name); + required_columns_map[name_type.name] = name_type; + } - if (desc.column.name.empty()) - out << "?"; - else - out << desc.column.name; - } -} - -void InterpolateColumnDescription::interpolate(Field & field) const -{ - if (field.isNull()) - return; - Block expr_columns; - Field column_field = convertFieldToType(field, *column.type.get()); - expr_columns.insert({column.type->createColumnConst(1, column_field), column.type, column.name}); - actions->execute(expr_columns); - field = convertFieldToType((*expr_columns.getByPosition(0).column)[0], *column.type.get()); -} - -void InterpolateColumnDescription::explain(JSONBuilder::JSONMap & map, const Block & /*header*/) const -{ - map.add("Column", column.name); -} - -std::string dumpInterpolateDescription(const InterpolateDescription & description) -{ - WriteBufferFromOwnString wb; - dumpInterpolateDescription(description, Block{}, wb); - return wb.str(); -} - -JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header) -{ - auto json_array = std::make_unique(); - for (const auto & descr : description) - { - auto json_map = std::make_unique(); - descr.explain(*json_map, header); - json_array->add(std::move(json_map)); + const Block & res_block = actions->getSampleBlock(); + size_t idx = 0; + for (const ColumnWithTypeAndName & column : res_block) + { + columns_full_set.insert(column.name); + result_columns_map[column.name] = idx++; + } } - return json_array; -} - } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index bb2b5febdc9..827586aac62 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -11,60 +11,26 @@ #include #include -class Collator; namespace DB { -namespace JSONBuilder -{ - class JSONMap; - class IItem; - using ItemPtr = std::unique_ptr; -} - -class Block; - - /// Interpolate description -struct InterpolateColumnDescription +struct InterpolateDescription { - using Signature = ExecutableFunctionExpression::Signature; + explicit InterpolateDescription(ExpressionActionsPtr actions); - ColumnWithTypeAndName column; ExpressionActionsPtr actions; + std::set columns_full_set; /// columns to add to row + std::unordered_map required_columns_map; /// input columns + std::unordered_map result_columns_map; /// result block column name -> block column index - explicit InterpolateColumnDescription(const ColumnWithTypeAndName & column_, ExpressionActionsPtr actions_) : - column(column_), actions(actions_) {} - - bool operator == (const InterpolateColumnDescription & other) const - { - return column == other.column; - } - - bool operator != (const InterpolateColumnDescription & other) const - { - return !(*this == other); - } - - void interpolate(Field & field) const; - - std::string dump() const - { - return fmt::format("{}", column.name); - } - - void explain(JSONBuilder::JSONMap & map, const Block & header) const; + /// filled externally in transform + std::unordered_map input_map; /// row index -> column name type + std::unordered_map output_map; /// result block column index -> row index + std::unordered_map reset_map; /// row index -> column type, columns not filled by fill or interpolate }; -/// Description of interpolation for several columns. -using InterpolateDescription = std::vector; - -/// Outputs user-readable description into `out`. -void dumpInterpolateDescription(const InterpolateDescription & description, const Block & header, WriteBuffer & out); - -std::string dumpInterpolateDescription(const InterpolateDescription & description); - -JSONBuilder::ItemPtr explainInterpolateDescription(const InterpolateDescription & description, const Block & header); +using InterpolateDescriptionPtr = std::shared_ptr; } diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 873c48702ae..89879f80ffc 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,30 +19,34 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_) - : sort{*this} - , interpolate{*this} - , sort_description(sort_description_) +FillingRow::FillingRow(const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) + : sort_description(sort_description_) , interpolate_description(interpolate_description_) { - row.resize(sort_description.size() + interpolate_description.size()); + std::unordered_set col_set; + for (const auto & col : sort_description) + col_set.insert(col.column_name); + if (interpolate_description) + for (const auto & name : interpolate_description->columns_full_set) + col_set.insert(name); + row.resize(col_set.size()); } bool FillingRow::operator<(const FillingRow & other) const { - for (size_t i = 0; i < sort.size(); ++i) + for (size_t i = 0; i < sort_description.size(); ++i) { - if (sort[i].isNull() || other.sort[i].isNull() || equals(sort[i], other.sort[i])) + if (row[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) continue; - return less(sort[i], other.sort[i], getDirection(i)); + return less(row[i], other.row[i], getDirection(i)); } return false; } bool FillingRow::operator==(const FillingRow & other) const { - for (size_t i = 0; i < sort.size(); ++i) - if (!equals(sort[i], other.sort[i])) + for (size_t i = 0; i < sort_description.size(); ++i) + if (!equals(row[i], other.row[i])) return false; return true; } @@ -51,52 +55,50 @@ bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; - for (size_t i = 0; i < to_row.interpolate.size(); ++i) - interpolate[i] = to_row.interpolate[i]; - /// Find position we need to increment for generating next row. - for (; pos < sort.size(); ++pos) - if (!sort[pos].isNull() && !to_row.sort[pos].isNull() && !equals(sort[pos], to_row.sort[pos])) + for (; pos < size(); ++pos) + if (!row[pos].isNull() && !to_row.row[pos].isNull() && !equals(row[pos], to_row.row[pos])) break; - if (pos == sort.size() || less(to_row.sort[pos], sort[pos], getDirection(pos))) + if (pos == size() || less(to_row.row[pos], row[pos], getDirection(pos))) return false; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. - for (size_t i = sort.size() - 1; i > pos; --i) + for (size_t i = size() - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || sort[i].isNull()) + if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) continue; - auto next_value = sort[i]; + auto next_value = row[i]; getFillDescription(i).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { - sort[i] = next_value; + row[i] = next_value; initFromDefaults(i + 1); return true; } } - auto next_value = sort[pos]; + auto next_value = row[pos]; getFillDescription(pos).step_func(next_value); - if (less(to_row.sort[pos], next_value, getDirection(pos))) + if (less(to_row.row[pos], next_value, getDirection(pos))) return false; - sort[pos] = next_value; - if (equals(sort[pos], to_row.sort[pos])) + row[pos] = next_value; + if (equals(row[pos], to_row.row[pos])) { bool is_less = false; - for (size_t i = pos + 1; i < sort.size(); ++i) + size_t i = pos + 1; + for (; i < size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) - sort[i] = fill_from; + row[i] = fill_from; else - sort[i] = to_row.sort[i]; - is_less |= less(sort[i], to_row.sort[i], getDirection(i)); + row[i] = to_row.row[i]; + is_less |= less(row[i], to_row.row[i], getDirection(i)); } return is_less; @@ -108,8 +110,39 @@ bool FillingRow::next(const FillingRow & to_row) void FillingRow::initFromDefaults(size_t from_pos) { - for (size_t i = from_pos; i < sort.size(); ++i) - sort[i] = getFillDescription(i).fill_from; + for (size_t i = from_pos; i < sort_description.size(); ++i) + row[i] = getFillDescription(i).fill_from; +} + +void FillingRow::interpolate() +{ + if (!interpolate_description) + return; + + Block block; + if (interpolate_description->input_map.size()) + { + for (const auto & [row_pos, name_type] : interpolate_description->input_map) + { + if (row[row_pos].isNull()) + return; + block.insert({name_type.type->createColumnConst(1, row[row_pos]), name_type.type, name_type.name}); + } + } + else /// INTERPOLATE contains only constants + { + /// dirty hack - we need at least one column with one row to execute actions on block + DataTypePtr dt = std::make_shared(); + block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + } + + interpolate_description->actions->execute(block); + + for (auto [col_pos, row_pos] : interpolate_description->output_map) + block.getByPosition(col_pos).column->get(0, row[row_pos]); + + for (const auto & [row_pos, col_type] : interpolate_description->reset_map) + row[row_pos] = col_type->getDefault(); } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 10adb6b72a8..04e0b6a0a79 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,25 +18,7 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - struct - { - FillingRow & filling_row; - - Field & operator[](size_t index) { return filling_row.row[index]; } - const Field & operator[](size_t index) const { return filling_row.row[index]; } - size_t size() const { return filling_row.sort_description.size(); } - } sort; - - struct - { - FillingRow & filling_row; - - Field & operator[](size_t index) { return filling_row.row[filling_row.sort_description.size() + index]; } - const Field & operator[](size_t index) const { return filling_row.row[filling_row.sort_description.size() + index]; } - size_t size() const { return filling_row.interpolate_description.size(); } - } interpolate; -public: - FillingRow(const SortDescription & sort_description, const InterpolateDescription & interpolate_description); + FillingRow(const SortDescription & sort_description, InterpolateDescriptionPtr interpolate_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -45,18 +27,20 @@ public: Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } - size_t size() const { return row.size(); } + size_t size() const { return sort_description.size(); } + size_t row_size() const { return row.size(); } bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } - InterpolateColumnDescription & getInterpolateDescription(size_t index) { return interpolate_description[index]; } + + void interpolate(); private: Row row; SortDescription sort_description; - InterpolateDescription interpolate_description; + InterpolateDescriptionPtr interpolate_description; }; void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4bef4c66ade..611087e1e99 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -833,13 +833,12 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { - InterpolateDescription interpolate_descr; + InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - interpolate_descr.reserve(query.interpolate()->children.size()); - + std::map> position_map; for (const auto & elem : query.interpolate()->children) { auto interpolate = elem->as(); @@ -847,14 +846,31 @@ static InterpolateDescription getInterpolateDescription(const ASTSelectQuery & q if (!block_column) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); - ColumnWithTypeAndName column = block_column->cloneEmpty(); - auto syntax_result = TreeRewriter(context).analyze(interpolate.expr, block.getNamesAndTypesList()); - ExpressionAnalyzer analyzer(interpolate.expr, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + size_t position = block.getPositionByName(interpolate.column->getColumnName()); + position_map[position] = { + ColumnWithTypeAndName(block_column->type, block_column->name), + interpolate.expr->clone() + }; - interpolate_descr.emplace_back(column, actions); } + + ColumnsWithTypeAndName columns; + ASTPtr exprs = std::make_shared(); + for (auto & p : position_map) + { + columns.emplace_back(std::move(p.second.first)); + exprs->children.emplace_back(std::move(p.second.second)); + } + + auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); + ExpressionAnalyzer analyzer(exprs, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); + ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getActionsDAG().getResultColumns(), + columns, ActionsDAG::MatchColumnsMode::Position, true); + ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); + + interpolate_descr = std::make_shared(std::make_shared(mergeDAG)); } return interpolate_descr; @@ -2527,8 +2543,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescription interpolate_descr = getInterpolateDescription(query, source_header, context); - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), std::move(interpolate_descr)); + InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, context); + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dbcfeb8e8b5..a643cfb83ea 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2325,12 +2325,14 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (!ident_p.parse(pos, ident, expected)) return false; - if (!as.ignore(pos, expected)) - return false; - ASTPtr expr; - if (!element_p.parse(pos, expr, expected)) - return false; + if (as.ignore(pos, expected)) + { + if (!element_p.parse(pos, expr, expected)) + return false; + } + else + expr = ident; auto elem = std::make_shared(); elem->column = ident; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1396e86df39..e79bdc495c1 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -759,7 +759,7 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & bool ParserInterpolateExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), true) .parse(pos, node, expected); } diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 6cc00da1db0..65c89bd6f9b 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include namespace DB @@ -250,12 +252,32 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (s_interpolate.ignore(pos, expected)) { - if (!open_bracket.ignore(pos, expected)) - return false; - if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) - return false; - if (!close_bracket.ignore(pos, expected)) - return false; + if (open_bracket.ignore(pos, expected)) + { + if (!interpolate_list.parse(pos, interpolate_expression_list, expected)) + return false; + if (!close_bracket.ignore(pos, expected)) + return false; + } else + interpolate_expression_list = std::make_shared(); + + if (interpolate_expression_list->children.empty()) + { + std::unordered_map columns; + for (const auto & elem : select_expression_list->children) + columns[elem->getColumnName()] = elem; + for (const auto & elem : order_expression_list->children) + if (elem->as()->with_fill) + columns.erase(elem->as()->children.front()->getColumnName()); + + for (const auto & [column, ast] : columns) + { + auto elem = std::make_shared(); + elem->column = ast; + elem->expr = ast; + interpolate_expression_list->children.push_back(elem); + } + } } } } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 47540bb2725..cfff191b464 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -28,9 +28,9 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_) - : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_/*, interpolate_description_*/), getTraits()) - , sort_description(std::move(sort_description_)), interpolate_description(std::move(interpolate_description_)) +FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_) + : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) + , sort_description(std::move(sort_description_)), interpolate_description(interpolate_description_) { if (!input_stream_.has_single_port) throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR); @@ -50,14 +50,11 @@ void FillingStep::describeActions(FormatSettings & settings) const settings.out << String(settings.offset, ' '); dumpSortDescription(sort_description, input_streams.front().header, settings.out); settings.out << '\n'; - dumpInterpolateDescription(interpolate_description, input_streams.front().header, settings.out); - settings.out << '\n'; } void FillingStep::describeActions(JSONBuilder::JSONMap & map) const { map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - map.add("Interpolate Description", explainInterpolateDescription(interpolate_description, input_streams.front().header)); } } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 252f1ba3fd5..0393b2c525b 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -10,7 +10,7 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescription interpolate_description_); + FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "Filling"; } @@ -23,7 +23,7 @@ public: private: SortDescription sort_description; - InterpolateDescription interpolate_description; + InterpolateDescriptionPtr interpolate_description; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c3991e393b9..85d31ecda4c 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -140,7 +140,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_) + const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_) : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) , sort_description(sort_description_) , interpolate_description(interpolate_description_) @@ -154,6 +154,19 @@ FillingTransform::FillingTransform( std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { + if (interpolate_description && interpolate_description->columns_full_set.count(sort_description[i].column_name)) + { + if (interpolate_description->result_columns_map.find(sort_description[i].column_name) != + interpolate_description->result_columns_map.end()) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", + sort_description[i].column_name); + + if (const auto & p = interpolate_description->required_columns_map.find(sort_description[i].column_name); + p != interpolate_description->required_columns_map.end()) + interpolate_description->input_map[fill_column_positions.size()] = p->second; + } + size_t block_position = header_.getPositionByName(sort_description[i].column_name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); @@ -174,41 +187,40 @@ FillingTransform::FillingTransform( } } - for (const auto & descr : interpolate_description) - { - size_t block_position = header_.getPositionByName(descr.column.name); - is_fill_column[block_position] = true; - fill_column_positions.push_back(block_position); - - /// Check column-expression compatibility - auto column = descr.column; - auto exp_type = descr.actions->getActionsDAG().getResultColumns()[0].type; - auto exp_column = exp_type->createColumn(); - exp_column->insertDefault(); - - try - { - if (auto exp_field = (*exp_column)[0]; convertFieldToType(exp_field, *column.type).isNull()) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", - exp_type->getName(), column.name, column.type->getName()); - } - catch (const Exception &) - { - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Incompatible types of INTERPOLATE expression type {} with column '{}' of type {}", - exp_type->getName(), column.name, column.type->getName()); - } - } - std::set unique_positions; for (auto pos : fill_column_positions) if (!unique_positions.insert(pos).second) throw Exception("Multiple WITH FILL for identical expressions is not supported in ORDER BY", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - for (size_t i = 0; i < header_.columns(); ++i) - if (!is_fill_column[i]) - other_column_positions.push_back(i); + size_t idx = 0; + for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) + { + if (!is_fill_column[idx]) + { + if (interpolate_description && interpolate_description->columns_full_set.count(column.name)) + { + if ( + const auto & p = interpolate_description->required_columns_map.find(column.name); + p != interpolate_description->required_columns_map.end() + ) + interpolate_description->input_map[fill_column_positions.size()] = p->second; + + if ( + const auto & p = interpolate_description->result_columns_map.find(column.name); + p != interpolate_description->result_columns_map.end() + ) + interpolate_description->output_map[p->second] = fill_column_positions.size(); + else + interpolate_description->reset_map[fill_column_positions.size()] = column.type; + + is_fill_column[idx] = true; + fill_column_positions.push_back(idx); + } + else + other_column_positions.push_back(idx); + } + ++idx; + } } IProcessor::Status FillingTransform::prepare() @@ -217,8 +229,8 @@ IProcessor::Status FillingTransform::prepare() { should_insert_first = next_row < filling_row || first; - for (size_t i = 0, size = filling_row.sort.size(); i < size; ++i) - next_row.sort[i] = filling_row.getFillDescription(i).fill_to; + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; if (first || filling_row < next_row) { @@ -277,9 +289,11 @@ void FillingTransform::transform(Chunk & chunk) init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions); init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions); + bool first_block = first; + if (first) { - for (size_t i = 0; i < filling_row.sort.size(); ++i) + for (size_t i = 0; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[0]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; @@ -300,7 +314,8 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - for (size_t i = 0; i < filling_row.sort.size(); ++i) + size_t i = 0; + for (; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -311,28 +326,25 @@ void FillingTransform::transform(Chunk & chunk) next_row[i] = fill_to; } + if (row_ind > 0) + for (; i < filling_row.row_size(); ++i) + filling_row[i] = (*old_fill_columns[i])[row_ind-1]; + /// A case, when at previous step row was initialized from defaults 'fill_from' values /// and probably we need to insert it to block. if (should_insert_first && filling_row < next_row) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); - /// Update interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); - /// Insert generated filling row to block, while it is less than current row in block. - while (filling_row.next(next_row)) + if (first_block) { - /// Update interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - filling_row.getInterpolateDescription(i - filling_row.sort.size()).interpolate(next_row[i]); - - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + first_block = false; + while (filling_row.next(next_row)) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); } - - /// Reset interpolate fields - for (size_t i = filling_row.sort.size(); i < filling_row.size(); ++i) - next_row[i] = (*old_fill_columns[i])[row_ind]; + else + for (filling_row.interpolate(); filling_row.next(next_row); filling_row.interpolate()) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 1a8a6c4919c..9f233dcafc6 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -14,13 +14,13 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, const InterpolateDescription & interpolate_description_, bool on_totals_); + FillingTransform(const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_); String getName() const override { return "FillingTransform"; } Status prepare() override; - static Block transformHeader(Block header, const SortDescription & sort_description/*, const InterpolateDescription & interpolate_description_*/); + static Block transformHeader(Block header, const SortDescription & sort_description); protected: void transform(Chunk & Chunk) override; @@ -29,7 +29,7 @@ private: void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; const SortDescription sort_description; /// Contains only columns with WITH FILL. - const InterpolateDescription interpolate_description; /// Contains INTERPOLATE columns + const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns const bool on_totals; /// FillingTransform does nothing on totals. FillingRow filling_row; /// Current row, which is used to fill gaps. diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index dacfbd068ae..c65be99bbe5 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -21,12 +21,12 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with incompatible const - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } # Test INTERPOLATE with incompatible expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } # Test INTERPOLATE with column from WITH FILL expression - should produce error SELECT n, source, inter FROM ( @@ -36,4 +36,4 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } From ae92963b15ec6b48408e994fd21bd2898935c209 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 25 Mar 2022 11:30:25 +0100 Subject: [PATCH 113/372] Fix build error in Formats/ISchemaReader.cpp --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index e421692743d..392b636e073 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -24,7 +24,7 @@ IRowSchemaReader::IRowSchemaReader( { std::string col_name_trimmed = column_names[i]; boost::trim(col_name_trimmed); - if (!col_name.empty()) + if (!col_name_trimmed.empty()) column_names[i] = col_name_trimmed; } } From 18ab49e788f3ed8a0c20634e31ba652129c0d24e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 25 Mar 2022 11:59:50 +0100 Subject: [PATCH 114/372] Check all logs for crashes, logical errors, etc in backward compatibility check --- docker/test/stress/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 3cef5b008db..e56afcbfd7a 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -348,13 +348,13 @@ then rm -f /test_output/tmp # OOM - zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \ + zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \ && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Logical errors echo "Check for Logical errors in server log:" - zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_logical_errors.txt \ + zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_logical_errors.txt \ && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv @@ -362,13 +362,13 @@ then [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt # Crash - zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \ + zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \ && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" - zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_fatal_messages.txt \ + zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_fatal_messages.txt \ && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 6fb3c3be043b7006e0ef63b62f22cb7620853145 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 25 Mar 2022 12:02:21 +0000 Subject: [PATCH 115/372] Fix comments and build --- src/Processors/Formats/ISchemaReader.cpp | 35 ++++++++++++++----- src/Processors/Formats/ISchemaReader.h | 5 ++- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 2 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 2 +- 4 files changed, 33 insertions(+), 11 deletions(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 392b636e073..0709b8f4d75 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -12,24 +12,35 @@ namespace ErrorCodes } IRowSchemaReader::IRowSchemaReader( - ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_, const DataTypes & default_types_) + ReadBuffer & in_, const FormatSettings & format_settings) : ISchemaReader(in_) - , max_rows_to_read(format_settings.max_rows_to_read_for_schema_inference), default_type(default_type_), default_types(default_types_) + , max_rows_to_read(format_settings.max_rows_to_read_for_schema_inference) { if (!format_settings.column_names_for_schema_inference.empty()) { /// column_names_for_schema_inference is a string in format 'column1,column2,column3,...' boost::split(column_names, format_settings.column_names_for_schema_inference, boost::is_any_of(",")); - for (size_t i = 0 ; i < column_names.size() ; ++i) + for (auto & column_name : column_names) { - std::string col_name_trimmed = column_names[i]; - boost::trim(col_name_trimmed); + std::string col_name_trimmed = boost::trim_copy(column_name); if (!col_name_trimmed.empty()) - column_names[i] = col_name_trimmed; + column_name = col_name_trimmed; } } } +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_) + : IRowSchemaReader(in_, format_settings) +{ + default_type = default_type_; +} + +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_) + : IRowSchemaReader(in_, format_settings) +{ + default_types = default_types_; +} + NamesAndTypesList IRowSchemaReader::readSchema() { DataTypes data_types = readRowAndGetDataTypes(); @@ -63,7 +74,11 @@ NamesAndTypesList IRowSchemaReader::readSchema() else throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName()); + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", + new_data_types[i]->getName(), + i + 1, + row, + data_types[i]->getName()); } } } @@ -146,7 +161,11 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() else throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName()); + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", + type->getName(), + name, + row, + new_type->getName()); } } } diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index cd0b552c021..1716b78f1b4 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -34,7 +34,10 @@ protected: class IRowSchemaReader : public ISchemaReader { public: - IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_ = nullptr, const DataTypes & default_types_ = {}); + IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); + IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, DataTypePtr default_type_); + IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, const DataTypes & default_types_); + NamesAndTypesList readSchema() override; protected: diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index c17c86d51ea..bb58e851ff8 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -684,7 +684,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema & schema) const { std::vector missing_columns; - auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name, &header, case_insensitive_matching); + auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name, false, &header, case_insensitive_matching); auto flatten_block_from_arrow = Nested::flatten(block_from_arrow); for (size_t i = 0, columns = header.columns(); i < columns; ++i) diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index a1f70730b39..df4d49b172c 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -454,7 +454,7 @@ TemplateSchemaReader::TemplateSchemaReader( const ParsedTemplateFormatString & row_format_, std::string row_between_delimiter, const FormatSettings & format_settings_) - : IRowSchemaReader(buf, format_settings_, nullptr, getDefaultDataTypeForEscapingRules(row_format_.escaping_rules)) + : IRowSchemaReader(buf, format_settings_, getDefaultDataTypeForEscapingRules(row_format_.escaping_rules)) , buf(in_) , format(format_) , row_format(row_format_) From eee89491500e75a741ba3407cd2df8d583fe156e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 25 Mar 2022 23:33:46 +0800 Subject: [PATCH 116/372] fix code --- src/Storages/Hive/HiveFile.cpp | 6 +++--- src/Storages/Hive/StorageHive.cpp | 2 -- src/TableFunctions/Hive/TableFunctionHive.cpp | 1 + src/TableFunctions/Hive/TableFunctionHive.h | 1 + 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 2862e654ddc..d1765da6b28 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -147,7 +147,7 @@ void HiveOrcFile::prepareColumnMapping() bool HiveOrcFile::hasMinMaxIndex() const { - return !storage_settings->enable_orc_file_minmax_index; + return storage_settings->enable_orc_file_minmax_index; } @@ -198,7 +198,7 @@ void HiveOrcFile::loadMinMaxIndex() bool HiveOrcFile::hasSubMinMaxIndex() const { - return !storage_settings->enable_orc_stripe_minmax_index; + return storage_settings->enable_orc_stripe_minmax_index; } @@ -228,7 +228,7 @@ void HiveOrcFile::loadSubMinMaxIndex() bool HiveParquetFile::hasSubMinMaxIndex() const { - return !storage_settings->enable_parquet_rowgroup_minmax_index; + return storage_settings->enable_parquet_rowgroup_minmax_index; } void HiveParquetFile::prepareReader() diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 7507f60b9b3..4296df3d7b1 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -7,8 +7,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index e7de55181c3..d29d65c2b8b 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -1,4 +1,5 @@ #include + #if USE_HIVE #include #include diff --git a/src/TableFunctions/Hive/TableFunctionHive.h b/src/TableFunctions/Hive/TableFunctionHive.h index 0973bdda329..2a8e47a90da 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.h +++ b/src/TableFunctions/Hive/TableFunctionHive.h @@ -1,5 +1,6 @@ #pragma once #include + #if USE_HIVE #include #include From 141c290344cca3e7db082e89822ecde71edc14a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 25 Mar 2022 18:31:15 +0100 Subject: [PATCH 117/372] Fix build --- src/Common/tests/gtest_lru_file_cache.cpp | 12 ++++++++--- src/Disks/tests/gtest_disk_encrypted.cpp | 20 +++++++++---------- .../StorageSystemRemoteFilesystemCache.cpp | 2 +- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Common/tests/gtest_lru_file_cache.cpp b/src/Common/tests/gtest_lru_file_cache.cpp index d5a76f9daad..77f4467fa36 100644 --- a/src/Common/tests/gtest_lru_file_cache.cpp +++ b/src/Common/tests/gtest_lru_file_cache.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -102,7 +103,10 @@ TEST(LRUFileCache, get) query_context->setCurrentQueryId("query_id"); DB::CurrentThread::QueryScope query_scope_holder(query_context); - auto cache = DB::LRUFileCache(cache_base_path, 30, 5); + DB::FileCacheSettings settings; + settings.max_size = 30; + settings.max_elements = 5; + auto cache = DB::LRUFileCache(cache_base_path, settings); cache.initialize(); auto key = cache.hash("key1"); @@ -472,7 +476,7 @@ TEST(LRUFileCache, get) { /// Test LRUCache::restore(). - auto cache2 = DB::LRUFileCache(cache_base_path, 30, 5); + auto cache2 = DB::LRUFileCache(cache_base_path, settings); cache2.initialize(); ASSERT_EQ(cache2.getStat().downloaded_size, 5); @@ -491,7 +495,9 @@ TEST(LRUFileCache, get) { /// Test max file segment size - auto cache2 = DB::LRUFileCache(caches_dir / "cache2", 30, 5, /* max_file_segment_size */10); + auto settings2 = settings; + settings.max_file_segment_size = 10; + auto cache2 = DB::LRUFileCache(caches_dir / "cache2", settings2); cache2.initialize(); auto holder1 = cache2.getOrSet(key, 0, 25); /// Get [0, 24] diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index fd3cc1acbe5..da041437951 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -96,7 +96,7 @@ TEST_F(DiskEncryptedTest, WriteAndRead) /// Write a file. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); writeString(std::string_view{"Some text"}, *buf); } @@ -122,7 +122,7 @@ TEST_F(DiskEncryptedTest, Append) /// Write a file (we use the append mode). { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); writeString(std::string_view{"Some text"}, *buf); } @@ -132,7 +132,7 @@ TEST_F(DiskEncryptedTest, Append) /// Append the file. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); writeString(std::string_view{" Another text"}, *buf); } @@ -148,7 +148,7 @@ TEST_F(DiskEncryptedTest, Truncate) /// Write a file (we use the append mode). { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); writeString(std::string_view{"Some text"}, *buf); } @@ -178,7 +178,7 @@ TEST_F(DiskEncryptedTest, ZeroFileSize) /// Write nothing to a file. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); } EXPECT_EQ(encrypted_disk->getFileSize("a.txt"), 0); @@ -187,7 +187,7 @@ TEST_F(DiskEncryptedTest, ZeroFileSize) /// Append the file with nothing. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); } EXPECT_EQ(encrypted_disk->getFileSize("a.txt"), 0); @@ -211,7 +211,7 @@ TEST_F(DiskEncryptedTest, AnotherFolder) /// Write a file. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); writeString(std::string_view{"Some text"}, *buf); } @@ -231,11 +231,11 @@ TEST_F(DiskEncryptedTest, RandomIV) /// Write two files with the same contents. { - auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); writeString(std::string_view{"Some text"}, *buf); } { - auto buf = encrypted_disk->writeFile("b.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + auto buf = encrypted_disk->writeFile("b.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); writeString(std::string_view{"Some text"}, *buf); } @@ -277,7 +277,7 @@ TEST_F(DiskEncryptedTest, RemoveFileDuringWriting) std::thread t1{[&] { for (size_t i = 0; i != n; ++i) - encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + encrypted_disk->writeFile("a.txt", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); }}; std::thread t2{[&] diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp index da5865e2043..c0d8ffc67bf 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp @@ -37,7 +37,7 @@ void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, for (const auto & [cache_base_path, cache_data] : caches) { - auto & cache = cache_data.cache; + const auto & cache = cache_data.cache; auto holder = cache->getAll(); for (const auto & file_segment : holder.file_segments) From 05ffa45907c65150f0de7041e49f2af76af21a7f Mon Sep 17 00:00:00 2001 From: Meena Renganathan Date: Fri, 25 Mar 2022 16:22:34 -0700 Subject: [PATCH 118/372] Further updates to the CMakeLists.txt to fix the build error in MacOS --- contrib/curl-cmake/CMakeLists.txt | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index b1e1a0ded8a..fdd73612549 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -153,6 +153,22 @@ set (SRCS add_library (_curl ${SRCS}) +set(CURL_LIBS "") +if(APPLE) + + find_library(SYSTEMCONFIGURATION_FRAMEWORK "SystemConfiguration") + if(NOT SYSTEMCONFIGURATION_FRAMEWORK) + message(FATAL_ERROR "SystemConfiguration framework not found") + endif() + set(CURL_LIBS ${CURL_LIBS} "-framework SystemConfiguration") + + find_library(COREFOUNDATION_FRAMEWORK "CoreFoundation") + if(NOT COREFOUNDATION_FRAMEWORK) + message(FATAL_ERROR "CoreFoundation framework not found") + endif() + set(CURL_LIBS ${CURL_LIBS} "-framework CoreFoundation") +endif() + target_compile_definitions (_curl PRIVATE HAVE_CONFIG_H BUILDING_LIBCURL @@ -166,7 +182,8 @@ target_include_directories (_curl SYSTEM PUBLIC . # curl_config.h ) -target_link_libraries (_curl PRIVATE OpenSSL::SSL) +set(CURL_LIBS ${CURL_LIBS} OpenSSL::SSL) +target_link_libraries (_curl PRIVATE ${CURL_LIBS}) # The library is large - avoid bloat (XXX: is it?) target_compile_options (_curl PRIVATE -g0) From 3bfd911ce2af3166efb7533a3a7fc33d577c70fd Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 26 Mar 2022 16:11:45 -0300 Subject: [PATCH 119/372] test for crash _join_with_nullable_lowcardinality --- ...th_nullable_lowcardinality_crash.reference | 2 ++ ...oin_with_nullable_lowcardinality_crash.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.reference create mode 100644 tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.reference b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.reference new file mode 100644 index 00000000000..12c61d9c54e --- /dev/null +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.reference @@ -0,0 +1,2 @@ +usa + diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql new file mode 100644 index 00000000000..abc2ee41402 --- /dev/null +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -0,0 +1,20 @@ +drop table if exists with_nullable; +drop table if exists without_nullable; + +CREATE TABLE with_nullable +( timestamp UInt32, + country LowCardinality(Nullable(String)) ) ENGINE = Memory; + +CREATE TABLE without_nullable +( timestamp UInt32, + country LowCardinality(String)) ENGINE = Memory; + +insert into with_nullable values(0,'f'),(0,'usa'); +insert into without_nullable values(0,'usa'),(0,'us2a'); + +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; + +drop table with_nullable; +drop table without_nullable; + From 5a4694f340ff82b41bbc2651cf74b733388088c4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 27 Mar 2022 13:33:22 -0400 Subject: [PATCH 120/372] major refactoring, simplified, optimized, bugs fixed --- src/Core/InterpolateDescription.cpp | 14 +- src/Core/InterpolateDescription.h | 9 +- src/Core/SortDescription.h | 7 +- src/Interpreters/FillingRow.cpp | 58 ++---- src/Interpreters/FillingRow.h | 11 +- src/Interpreters/InterpreterSelectQuery.cpp | 23 ++- .../Transforms/FillingTransform.cpp | 179 ++++++++++++------ src/Processors/Transforms/FillingTransform.h | 6 +- .../0_stateless/02233_interpolate_1.reference | 52 ++++- .../0_stateless/02233_interpolate_1.sql | 16 +- 10 files changed, 221 insertions(+), 154 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index e0c3bebd45e..817f66aa4bc 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -11,18 +11,10 @@ namespace DB : actions(actions_) { for (const auto & name_type : actions->getRequiredColumnsWithTypes()) - { - columns_full_set.insert(name_type.name); - required_columns_map[name_type.name] = name_type; - } + required_columns_map[name_type.name] = name_type.type; - const Block & res_block = actions->getSampleBlock(); - size_t idx = 0; - for (const ColumnWithTypeAndName & column : res_block) - { - columns_full_set.insert(column.name); - result_columns_map[column.name] = idx++; - } + for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) + result_columns_map.insert(column.name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 827586aac62..26ad4d1a11c 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -21,14 +21,11 @@ struct InterpolateDescription explicit InterpolateDescription(ExpressionActionsPtr actions); ExpressionActionsPtr actions; - std::set columns_full_set; /// columns to add to row - std::unordered_map required_columns_map; /// input columns - std::unordered_map result_columns_map; /// result block column name -> block column index + std::unordered_map required_columns_map; /// input columns + std::unordered_set result_columns_map; /// result block columns /// filled externally in transform - std::unordered_map input_map; /// row index -> column name type - std::unordered_map output_map; /// result block column index -> row index - std::unordered_map reset_map; /// row index -> column type, columns not filled by fill or interpolate + std::vector> input_positions; /// positions in result columns required for actions }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index db15f3a54db..ae5e9196d24 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include class Collator; @@ -24,6 +25,8 @@ class Block; struct FillColumnDescription { + DataTypePtr type; + /// All missed values in range [FROM, TO) will be filled /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM @@ -50,14 +53,14 @@ struct SortColumnDescription explicit SortColumnDescription( size_t column_number_, int direction_ = 1, int nulls_direction_ = 1, const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) + bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) , with_fill(with_fill_), fill_description(fill_description_) {} explicit SortColumnDescription( const std::string & column_name_, int direction_ = 1, int nulls_direction_ = 1, const std::shared_ptr & collator_ = nullptr, - bool with_fill_ = false, const FillColumnDescription & fill_description_ = {}) + bool with_fill_ = false, const FillColumnDescription & fill_description_ = FillColumnDescription{}) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 89879f80ffc..f5bbf036dd4 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -19,24 +19,17 @@ bool equals(const Field & lhs, const Field & rhs) } -FillingRow::FillingRow(const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) +FillingRow::FillingRow(const SortDescription & sort_description_) : sort_description(sort_description_) - , interpolate_description(interpolate_description_) { - std::unordered_set col_set; - for (const auto & col : sort_description) - col_set.insert(col.column_name); - if (interpolate_description) - for (const auto & name : interpolate_description->columns_full_set) - col_set.insert(name); - row.resize(col_set.size()); + row.resize(sort_description.size()); } bool FillingRow::operator<(const FillingRow & other) const { for (size_t i = 0; i < sort_description.size(); ++i) { - if (row[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) + if ((*this)[i].isNull() || other.row[i].isNull() || equals(row[i], other.row[i])) continue; return less(row[i], other.row[i], getDirection(i)); } @@ -114,39 +107,8 @@ void FillingRow::initFromDefaults(size_t from_pos) row[i] = getFillDescription(i).fill_from; } -void FillingRow::interpolate() -{ - if (!interpolate_description) - return; - - Block block; - if (interpolate_description->input_map.size()) - { - for (const auto & [row_pos, name_type] : interpolate_description->input_map) - { - if (row[row_pos].isNull()) - return; - block.insert({name_type.type->createColumnConst(1, row[row_pos]), name_type.type, name_type.name}); - } - } - else /// INTERPOLATE contains only constants - { - /// dirty hack - we need at least one column with one row to execute actions on block - DataTypePtr dt = std::make_shared(); - block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); - } - - interpolate_description->actions->execute(block); - - for (auto [col_pos, row_pos] : interpolate_description->output_map) - block.getByPosition(col_pos).column->get(0, row[row_pos]); - - for (const auto & [row_pos, col_type] : interpolate_description->reset_map) - row[row_pos] = col_type->getDefault(); -} - - -void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row) +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns, + const FillingRow & filling_row, const Block & interpolate_block) { for (size_t i = 0; i < filling_columns.size(); ++i) { @@ -156,6 +118,16 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & oth filling_columns[i]->insert(filling_row[i]); } + if(size_t size = interpolate_block.columns()) + { + Columns columns = interpolate_block.getColumns(); + for (size_t i = 0; i < size; ++i) + interpolate_columns[i]->insertFrom(*columns[i], 0); + } + else + for (const auto & interpolate_column : interpolate_columns) + interpolate_column->insertDefault(); + for (const auto & other_column : other_columns) other_column->insertDefault(); } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 04e0b6a0a79..626bee70896 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,7 +18,7 @@ bool equals(const Field & lhs, const Field & rhs); class FillingRow { public: - FillingRow(const SortDescription & sort_description, InterpolateDescriptionPtr interpolate_description); + explicit FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. bool next(const FillingRow & to_row); @@ -27,23 +27,20 @@ public: Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } - size_t size() const { return sort_description.size(); } - size_t row_size() const { return row.size(); } + size_t size() const { return row.size(); } bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } - void interpolate(); - private: Row row; SortDescription sort_description; - InterpolateDescriptionPtr interpolate_description; }; -void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns, + const FillingRow & filling_row, const Block & interpolate_block); void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ab09d8cd71d..b35bf31c81f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -763,9 +763,11 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, DataTypePtr type, ContextPtr context) { FillColumnDescription descr; + descr.type = type; + if (order_by_elem.fill_from) descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); if (order_by_elem.fill_to) @@ -809,7 +811,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, Block block, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -824,7 +826,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP if (order_by_elem.with_fill) { - FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); + FillColumnDescription fill_desc = + getWithFillDescription(order_by_elem, block.getByName(order_by_elem.children.front()->getColumnName()).type, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -1398,7 +1401,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query, analysis_result.order_by_elements_actions, - getSortDescription(query, context), + getSortDescription(query, source_header, context), query_info.syntax_analyzer_result); } } @@ -2387,7 +2390,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, context); + SortDescription output_order_descr = getSortDescription(query, source_header, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2425,7 +2428,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, source_header,context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2529,7 +2532,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr = getSortDescription(query, source_header, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2579,7 +2582,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, context); + order_descr = getSortDescription(query, source_header, context); } auto limit = std::make_unique( diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 088e85ca9d9..f71c4f4f511 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -139,12 +139,12 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_/*, interpolate_description_*/), true) + : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) , sort_description(sort_description_) , interpolate_description(interpolate_description_) , on_totals(on_totals_) - , filling_row(sort_description_, interpolate_description_) - , next_row(sort_description_, interpolate_description_) + , filling_row(sort_description_) + , next_row(sort_description_) { if (on_totals) return; @@ -152,18 +152,10 @@ FillingTransform::FillingTransform( std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->columns_full_set.count(sort_description[i].column_name)) - { - if (interpolate_description->result_columns_map.find(sort_description[i].column_name) != - interpolate_description->result_columns_map.end()) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", - sort_description[i].column_name); - - if (const auto & p = interpolate_description->required_columns_map.find(sort_description[i].column_name); - p != interpolate_description->required_columns_map.end()) - interpolate_description->input_map[fill_column_positions.size()] = p->second; - } + if (interpolate_description && interpolate_description->result_columns_map.count(sort_description[i].column_name)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", + sort_description[i].column_name); size_t block_position = header_.getPositionByName(sort_description[i].column_name); is_fill_column[block_position] = true; @@ -193,27 +185,15 @@ FillingTransform::FillingTransform( size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) { + if (interpolate_description) + if (const auto & p = interpolate_description->required_columns_map.find(column.name); + p != interpolate_description->required_columns_map.end()) + interpolate_description->input_positions.emplace_back(idx, NameAndTypePair(column.name, p->second)); + if (!is_fill_column[idx]) { - if (interpolate_description && interpolate_description->columns_full_set.count(column.name)) - { - if ( - const auto & p = interpolate_description->required_columns_map.find(column.name); - p != interpolate_description->required_columns_map.end() - ) - interpolate_description->input_map[fill_column_positions.size()] = p->second; - - if ( - const auto & p = interpolate_description->result_columns_map.find(column.name); - p != interpolate_description->result_columns_map.end() - ) - interpolate_description->output_map[p->second] = fill_column_positions.size(); - else - interpolate_description->reset_map[fill_column_positions.size()] = column.type; - - is_fill_column[idx] = true; - fill_column_positions.push_back(idx); - } + if (interpolate_description && interpolate_description->result_columns_map.count(column.name)) + interpolate_column_positions.push_back(idx); else other_column_positions.push_back(idx); } @@ -247,37 +227,85 @@ void FillingTransform::transform(Chunk & chunk) return; Columns old_fill_columns; + Columns old_interpolate_columns; Columns old_other_columns; MutableColumns res_fill_columns; + MutableColumns res_interpolate_columns; MutableColumns res_other_columns; - auto init_columns_by_positions = [](const Columns & old_columns, Columns & new_columns, - MutableColumns & new_mutable_columns, const Positions & positions) + std::vector> res_map; + res_map.resize(input.getHeader().columns()); + + auto init_columns_by_positions = [&res_map](const Columns & old_columns, Columns & new_columns, + MutableColumns & new_mutable_columns, const Positions & positions) { for (size_t pos : positions) { auto old_column = old_columns[pos]->convertToFullColumnIfConst(); new_columns.push_back(old_column); + res_map[pos] = {&new_mutable_columns, new_mutable_columns.size()}; new_mutable_columns.push_back(old_column->cloneEmpty()->assumeMutable()); } }; + Block interpolate_block; + + auto interpolate = [&]() { + if (interpolate_description) + { + interpolate_block.clear(); + + if (interpolate_description->input_positions.size()) + { + for (const auto & [col_pos, name_type] : interpolate_description->input_positions) + { + MutableColumnPtr column = name_type.type->createColumn(); + auto [res_columns, pos] = res_map[col_pos]; + size_t size = (*res_columns)[pos]->size(); + if (size == 0) + { + if (last_row.size() > col_pos && last_row[col_pos]->size()) + column->insertFrom(*last_row[col_pos], 0); + else + column->insertDefault(); + } + else + column->insertFrom(*(*res_columns)[pos], size - 1); + + interpolate_block.insert({std::move(column), name_type.type, name_type.name}); + } + } + else /// all INTERPOLATE expressions are constants + { + /// dirty hack - we need at least one column with one row to execute actions on block + DataTypePtr dt = std::make_shared(); + interpolate_block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + } + + interpolate_description->actions->execute(interpolate_block); + } + }; + if (generate_suffix) { const auto & empty_columns = input.getHeader().getColumns(); init_columns_by_positions(empty_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(empty_columns, old_interpolate_columns, res_interpolate_columns, interpolate_column_positions); init_columns_by_positions(empty_columns, old_other_columns, res_other_columns, other_column_positions); if (first) filling_row.initFromDefaults(); if (should_insert_first && filling_row < next_row) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + { + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } - while (filling_row.next(next_row)) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + for (interpolate(); filling_row.next(next_row); interpolate()) + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - setResultColumns(chunk, res_fill_columns, res_other_columns); + setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); return; } @@ -285,10 +313,9 @@ void FillingTransform::transform(Chunk & chunk) auto old_columns = chunk.detachColumns(); init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(old_columns, old_interpolate_columns, res_interpolate_columns, interpolate_column_positions); init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions); - bool first_block = first; - if (first) { for (size_t i = 0; i < filling_row.size(); ++i) @@ -300,7 +327,10 @@ void FillingTransform::transform(Chunk & chunk) { filling_row.initFromDefaults(i); if (less(fill_from, current_value, filling_row.getDirection(i))) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + { + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } break; } filling_row[i] = current_value; @@ -312,8 +342,7 @@ void FillingTransform::transform(Chunk & chunk) { should_insert_first = next_row < filling_row; - size_t i = 0; - for (; i < filling_row.size(); ++i) + for (size_t i = 0; i < filling_row.size(); ++i) { auto current_value = (*old_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; @@ -324,45 +353,71 @@ void FillingTransform::transform(Chunk & chunk) next_row[i] = fill_to; } - if (row_ind > 0) - for (; i < filling_row.row_size(); ++i) - filling_row[i] = (*old_fill_columns[i])[row_ind-1]; - /// A case, when at previous step row was initialized from defaults 'fill_from' values /// and probably we need to insert it to block. if (should_insert_first && filling_row < next_row) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); - - /// Insert generated filling row to block, while it is less than current row in block. - if (first_block) { - first_block = false; - while (filling_row.next(next_row)) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - else - for (filling_row.interpolate(); filling_row.next(next_row); filling_row.interpolate()) - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + for (interpolate(); filling_row.next(next_row); interpolate()) + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); + copyRowFromColumns(res_interpolate_columns, old_interpolate_columns, row_ind); copyRowFromColumns(res_other_columns, old_other_columns, row_ind); } - setResultColumns(chunk, res_fill_columns, res_other_columns); + saveLastRow(res_fill_columns, res_interpolate_columns, res_other_columns); + setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); } -void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const +void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns) const { - MutableColumns result_columns(fill_columns.size() + other_columns.size()); + MutableColumns result_columns(fill_columns.size() + interpolate_columns.size() + other_columns.size()); /// fill_columns always non-empty. size_t num_rows = fill_columns[0]->size(); for (size_t i = 0, size = fill_columns.size(); i < size; ++i) result_columns[fill_column_positions[i]] = std::move(fill_columns[i]); + for (size_t i = 0, size = interpolate_columns.size(); i < size; ++i) + result_columns[interpolate_column_positions[i]] = std::move(interpolate_columns[i]); for (size_t i = 0, size = other_columns.size(); i < size; ++i) result_columns[other_column_positions[i]] = std::move(other_columns[i]); chunk.setColumns(std::move(result_columns), num_rows); } +void FillingTransform::saveLastRow(const MutableColumns & fill_columns, const MutableColumns & interpolate_columns, const MutableColumns & other_columns) +{ + last_row.clear(); + last_row.resize(fill_columns.size() + interpolate_columns.size() + other_columns.size()); + + size_t num_rows = fill_columns[0]->size(); + if (num_rows == 0) + return; + + for (size_t i = 0, size = fill_columns.size(); i < size; ++i) + { + auto column = fill_columns[i]->cloneEmpty(); + column->insertFrom(*fill_columns[i], num_rows - 1); + last_row[fill_column_positions[i]] = std::move(column); + } + + for (size_t i = 0, size = interpolate_columns.size(); i < size; ++i) + { + auto column = interpolate_columns[i]->cloneEmpty(); + column->insertFrom(*interpolate_columns[i], num_rows - 1); + last_row[interpolate_column_positions[i]] = std::move(column); + } + + for (size_t i = 0, size = other_columns.size(); i < size; ++i) + { + auto column = other_columns[i]->cloneEmpty(); + column->insertFrom(*other_columns[i], num_rows - 1); + last_row[other_column_positions[i]] = std::move(column); + } +} + } diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 9f233dcafc6..90f8d61ceb6 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -26,7 +26,8 @@ protected: void transform(Chunk & Chunk) override; private: - void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; + void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns) const; + void saveLastRow(const MutableColumns & fill_columns, const MutableColumns & interpolate_columns, const MutableColumns & other_columns); const SortDescription sort_description; /// Contains only columns with WITH FILL. const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns @@ -37,10 +38,13 @@ private: using Positions = std::vector; Positions fill_column_positions; + Positions interpolate_column_positions; Positions other_column_positions; bool first = true; bool generate_suffix = false; + Columns last_row; + /// Determines should we insert filling row before start generating next rows. bool should_insert_first = false; }; diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index 45377d486f9..c7774fe05ff 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -10,9 +10,20 @@ 4.5 0 5 0 5.5 0 +6 0 +6.5 0 7 original 7 -0 0 -0.5 0 +7.5 0 +8 0 +8.5 0 +9 0 +9.5 0 +10 0 +10.5 0 +11 0 +11.5 0 +0 42 +0.5 42 1 original 1 1.5 42 2 42 @@ -23,7 +34,18 @@ 4.5 42 5 42 5.5 42 +6 42 +6.5 42 7 original 7 +7.5 42 +8 42 +8.5 42 +9 42 +9.5 42 +10 42 +10.5 42 +11 42 +11.5 42 0 0 0.5 0 1 original 1 @@ -36,9 +58,20 @@ 4.5 4 5 4 5.5 4 +6 4 +6.5 4 7 original 7 -0 0 -0.5 0 +7.5 7 +8 7 +8.5 7 +9 7 +9.5 7 +10 7 +10.5 7 +11 7 +11.5 7 +0 1 +0.5 2 1 original 1 1.5 2 2 3 @@ -49,4 +82,15 @@ 4.5 5 5 6 5.5 7 +6 8 +6.5 9 7 original 7 +7.5 8 +8 9 +8.5 10 +9 11 +9.5 12 +10 13 +10.5 14 +11 15 +11.5 16 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index c65be99bbe5..a355e903dae 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -1,39 +1,39 @@ # Test WITH FILL without INTERPOLATE SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5; +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5; # Test INTERPOLATE with const SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 42); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS 42); # Test INTERPOLATE with field value SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter); # Test INTERPOLATE with expression SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter + 1); # Test INTERPOLATE with incompatible const - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS 'inter'); -- { serverError 6 } # Test INTERPOLATE with incompatible expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS inter||'inter'); -- { serverError 44 } # Test INTERPOLATE with column from WITH FILL expression - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (n AS n); -- { serverError 475 } # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } From bbfe8a2ca7bcd52aee0f138b59db4ad96b0b623f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Mar 2022 15:28:17 +0000 Subject: [PATCH 121/372] fix possible loss of subcolumns in type Object --- src/DataTypes/DataTypeTuple.cpp | 2 +- .../0_stateless/01825_type_json_9.reference | 1 + tests/queries/0_stateless/01825_type_json_9.sql | 16 ++++++++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01825_type_json_9.reference create mode 100644 tests/queries/0_stateless/01825_type_json_9.sql diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index a5e9868cf89..abf53a4baf1 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -206,7 +206,7 @@ bool DataTypeTuple::equals(const IDataType & rhs) const return false; for (size_t i = 0; i < size; ++i) - if (!elems[i]->equals(*rhs_tuple.elems[i])) + if (!elems[i]->equals(*rhs_tuple.elems[i]) || names[i] != rhs_tuple.names[i]) return false; return true; diff --git a/tests/queries/0_stateless/01825_type_json_9.reference b/tests/queries/0_stateless/01825_type_json_9.reference new file mode 100644 index 00000000000..a426b09a100 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_9.reference @@ -0,0 +1 @@ +Tuple(foo Int8, k1 Int8, k2 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_9.sql b/tests/queries/0_stateless/01825_type_json_9.sql new file mode 100644 index 00000000000..8fa4b335578 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_9.sql @@ -0,0 +1,16 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json; + +SET allow_experimental_object_type = 1; + +CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}}; +INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}}; + +OPTIMIZE TABLE t_json FINAL; + +SELECT any(toTypeName(obj)) from t_json; + +DROP TABLE IF EXISTS t_json; From 4baccf9d810526fbbec238f4de84e1721767cb93 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 28 Mar 2022 20:24:18 +0200 Subject: [PATCH 122/372] fix formatting --- tests/integration/test_transactions/test.py | 133 ++++++++++++-------- 1 file changed, 79 insertions(+), 54 deletions(-) diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 66b5f13213a..8983e70b4cb 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -2,7 +2,12 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/transactions.xml"], stay_alive=True, with_zookeeper=True) +node = cluster.add_instance( + "node", + main_configs=["configs/transactions.xml"], + stay_alive=True, + with_zookeeper=True, +) @pytest.fixture(scope="module") @@ -15,81 +20,101 @@ def start_cluster(): def tx(session, query): - params = {'session_id': 'session_{}'.format(session)} + params = {"session_id": "session_{}".format(session)} return node.http_query(None, data=query, params=params) def test_rollback_unfinished_on_restart(start_cluster): - node.query('create table mt (n int, m int) engine=MergeTree order by n partition by n % 2') - node.query('insert into mt values (1, 10), (2, 20)') + node.query( + "create table mt (n int, m int) engine=MergeTree order by n partition by n % 2" + ) + node.query("insert into mt values (1, 10), (2, 20)") tid0 = "(1,1,'00000000-0000-0000-0000-000000000000')" # it will hold a snapshot and avoid parts cleanup - tx(0, 'begin transaction') + tx(0, "begin transaction") - tx(4, 'begin transaction') + tx(4, "begin transaction") - tx(1, 'begin transaction') - tid1 = tx(1, 'select transactionID()').strip() + tx(1, "begin transaction") + tid1 = tx(1, "select transactionID()").strip() tx(1, "alter table mt drop partition id '1'") - tx(1, 'commit') + tx(1, "commit") - tx(1, 'begin transaction') - tid2 = tx(1, 'select transactionID()').strip() - tx(1, 'insert into mt values (3, 30), (4, 40)') - tx(1, 'commit') + tx(1, "begin transaction") + tid2 = tx(1, "select transactionID()").strip() + tx(1, "insert into mt values (3, 30), (4, 40)") + tx(1, "commit") - node.query('system flush logs') - csn1 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid1)).strip() - csn2 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid2)).strip() + node.query("system flush logs") + csn1 = node.query( + "select csn from system.transactions_info_log where type='Commit' and tid={}".format( + tid1 + ) + ).strip() + csn2 = node.query( + "select csn from system.transactions_info_log where type='Commit' and tid={}".format( + tid2 + ) + ).strip() # insert a part before starting mutation and check that it will not be mutated - tx(4, 'insert into mt values (9, 90)') + tx(4, "insert into mt values (9, 90)") # check that uncommitted mutation will be rolled back on restart - tx(1, 'begin transaction') - tid3 = tx(1, 'select transactionID()').strip() - tx(1, 'insert into mt values (5, 50)') + tx(1, "begin transaction") + tid3 = tx(1, "select transactionID()").strip() + tx(1, "insert into mt values (5, 50)") tx(1, "alter table mt update m = m+n in partition id '1' where 1") # check that uncommitted merge will be rolled back on restart - tx(2, 'begin transaction') - tid4 = tx(2, 'select transactionID()').strip() - tx(2, "optimize table mt partition id '0' final settings optimize_throw_if_noop = 1") + tx(2, "begin transaction") + tid4 = tx(2, "select transactionID()").strip() + tx( + 2, + "optimize table mt partition id '0' final settings optimize_throw_if_noop = 1", + ) # check that uncommitted insert will be rolled back on restart - tx(3, 'begin transaction') - tid5 = tx(3, 'select transactionID()').strip() - tx(3, 'insert into mt values (6, 70)') + tx(3, "begin transaction") + tid5 = tx(3, "select transactionID()").strip() + tx(3, "insert into mt values (6, 70)") - tid6 = tx(4, 'select transactionID()').strip() - tx(4, 'commit') - node.query('system flush logs') - csn6 = node.query("select csn from system.transactions_info_log where type='Commit' and tid={}".format(tid6)).strip() + tid6 = tx(4, "select transactionID()").strip() + tx(4, "commit") + node.query("system flush logs") + csn6 = node.query( + "select csn from system.transactions_info_log where type='Commit' and tid={}".format( + tid6 + ) + ).strip() node.restart_clickhouse(kill=True) - assert node.query('select *, _part from mt order by n') == '2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n9\t90\t1_5_5_0\n' - res = node.query("select name, active, creation_tid, 'csn' || toString(creation_csn) || '_', removal_tid, 'csn' || toString(removal_csn) || '_' from system.parts where table='mt' order by name") - res = res.replace(tid0, 'tid0') - res = res.replace(tid1, 'tid1').replace('csn' + csn1 + '_', 'csn_1') - res = res.replace(tid2, 'tid2').replace('csn' + csn2 + '_', 'csn_2') - res = res.replace(tid3, 'tid3') - res = res.replace(tid4, 'tid4') - res = res.replace(tid5, 'tid5') - res = res.replace(tid6, 'tid6').replace('csn' + csn6 + '_', 'csn_6') - assert res == "0_2_2_0\t1\ttid0\tcsn1_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "0_2_4_1\t0\ttid4\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "0_8_8_0\t0\ttid5\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n" \ - "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "1_6_6_0\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" \ - "1_6_6_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" - - - - - + assert ( + node.query("select *, _part from mt order by n") + == "2\t20\t0_2_2_0\n3\t30\t1_3_3_0\n4\t40\t0_4_4_0\n9\t90\t1_5_5_0\n" + ) + res = node.query( + "select name, active, creation_tid, 'csn' || toString(creation_csn) || '_', removal_tid, 'csn' || toString(removal_csn) || '_' from system.parts where table='mt' order by name" + ) + res = res.replace(tid0, "tid0") + res = res.replace(tid1, "tid1").replace("csn" + csn1 + "_", "csn_1") + res = res.replace(tid2, "tid2").replace("csn" + csn2 + "_", "csn_2") + res = res.replace(tid3, "tid3") + res = res.replace(tid4, "tid4") + res = res.replace(tid5, "tid5") + res = res.replace(tid6, "tid6").replace("csn" + csn6 + "_", "csn_6") + assert ( + res + == "0_2_2_0\t1\ttid0\tcsn1_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "0_2_4_1\t0\ttid4\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "0_8_8_0\t0\ttid5\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n" + "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "1_6_6_0\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + "1_6_6_0_7\t0\ttid3\tcsn18446744073709551615_\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" + ) From 6cbdc6af005f87e8b638c7c3f862cf1aea464a22 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Mar 2022 18:44:53 +0000 Subject: [PATCH 123/372] remove obsolete parameter --- .../DataTypeLowCardinalityHelpers.cpp | 2 +- src/DataTypes/DataTypeTuple.cpp | 21 +++++++------------ src/DataTypes/DataTypeTuple.h | 6 +----- src/Functions/FunctionsConversion.h | 3 +-- src/Functions/tuple.cpp | 21 ++----------------- .../02008_tuple_to_name_value_pairs.sql | 2 +- 6 files changed, 13 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index 41ba81814d0..21ab25b6da3 100644 --- a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -36,7 +36,7 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) element = recursiveRemoveLowCardinality(element); if (tuple_type->haveExplicitNames()) - return std::make_shared(elements, tuple_type->getElementNames(), tuple_type->serializeNames()); + return std::make_shared(elements, tuple_type->getElementNames()); else return std::make_shared(elements); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index abf53a4baf1..908e0184b8d 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -64,8 +64,8 @@ static std::optional checkTupleNames(const Strings & names) return {}; } -DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, bool serialize_names_) - : elems(elems_), names(names_), have_explicit_names(true), serialize_names(serialize_names_) +DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_) + : elems(elems_), names(names_), have_explicit_names(true) { size_t size = elems.size(); if (names.size() != size) @@ -75,11 +75,6 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, b throw std::move(*exception); } -bool DataTypeTuple::canBeCreatedWithNames(const Strings & names) -{ - return checkTupleNames(names) == std::nullopt; -} - std::string DataTypeTuple::doGetName() const { size_t size = elems.size(); @@ -91,7 +86,7 @@ std::string DataTypeTuple::doGetName() const if (i != 0) s << ", "; - if (have_explicit_names && serialize_names) + if (have_explicit_names) s << backQuoteIfNeed(names[i]) << ' '; s << elems[i]->getName(); @@ -265,31 +260,29 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const SerializationPtr DataTypeTuple::doGetDefaultSerialization() const { SerializationTuple::ElementSerializations serializations(elems.size()); - bool use_explicit_names = have_explicit_names && serialize_names; for (size_t i = 0; i < elems.size(); ++i) { - String elem_name = use_explicit_names ? names[i] : toString(i + 1); + String elem_name = have_explicit_names ? names[i] : toString(i + 1); auto serialization = elems[i]->getDefaultSerialization(); serializations[i] = std::make_shared(serialization, elem_name); } - return std::make_shared(std::move(serializations), use_explicit_names); + return std::make_shared(std::move(serializations), have_explicit_names); } SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) const { SerializationTuple::ElementSerializations serializations(elems.size()); const auto & info_tuple = assert_cast(info); - bool use_explicit_names = have_explicit_names && serialize_names; for (size_t i = 0; i < elems.size(); ++i) { - String elem_name = use_explicit_names ? names[i] : toString(i + 1); + String elem_name = have_explicit_names ? names[i] : toString(i + 1); auto serialization = elems[i]->getSerialization(*info_tuple.getElementInfo(i)); serializations[i] = std::make_shared(serialization, elem_name); } - return std::make_shared(std::move(serializations), use_explicit_names); + return std::make_shared(std::move(serializations), have_explicit_names); } MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index db122aae5df..009a2284a0a 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -22,14 +22,11 @@ private: DataTypes elems; Strings names; bool have_explicit_names; - bool serialize_names = true; public: static constexpr bool is_parametric = true; explicit DataTypeTuple(const DataTypes & elems); - DataTypeTuple(const DataTypes & elems, const Strings & names, bool serialize_names_ = true); - - static bool canBeCreatedWithNames(const Strings & names); + DataTypeTuple(const DataTypes & elems, const Strings & names); TypeIndex getTypeId() const override { return TypeIndex::Tuple; } std::string doGetName() const override; @@ -66,7 +63,6 @@ public: String getNameByPosition(size_t i) const; bool haveExplicitNames() const { return have_explicit_names; } - bool serializeNames() const { return serialize_names; } }; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e098378f51a..587efa9f217 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2957,8 +2957,7 @@ private: /// For named tuples allow conversions for tuples with /// different sets of elements. If element exists in @to_type /// and doesn't exist in @to_type it will be filled by default values. - if (from_type->haveExplicitNames() && from_type->serializeNames() - && to_type->haveExplicitNames() && to_type->serializeNames()) + if (from_type->haveExplicitNames() && to_type->haveExplicitNames()) { const auto & from_names = from_type->getElementNames(); std::unordered_map from_positions; diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 8e8b18e335d..6d5c53c0770 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -54,29 +54,12 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - DataTypes types; - Strings names; - - for (const auto & argument : arguments) - { - types.emplace_back(argument.type); - names.emplace_back(argument.name); - } - - /// Create named tuple if possible. We don't print tuple element names - /// because they are bad anyway -- aliases are not used, e.g. tuple(1 a) - /// will have element name '1' and not 'a'. If we ever change this, and - /// add the ability to access tuple elements by name, like tuple(1 a).a, - /// we should probably enable printing for better discoverability. - if (DataTypeTuple::canBeCreatedWithNames(names)) - return std::make_shared(types, names, false /*print names*/); - - return std::make_shared(types); + return std::make_shared(arguments); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override diff --git a/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql index 9204975b579..59987a86590 100644 --- a/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql +++ b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test02008; CREATE TABLE test02008 ( col Tuple( a Tuple(key1 int, key2 int), - b Tuple(key1 int, key3 int) + b Tuple(key1 int, key2 int) ) ) ENGINE=Memory(); INSERT INTO test02008 VALUES (tuple(tuple(1, 2), tuple(3, 4))); From fddeecdd699c049e07b60010e57a005a500eb884 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 28 Mar 2022 21:59:43 +0200 Subject: [PATCH 124/372] Fix fast test --- .../0_stateless/02244_column_names_in_shcmea_inference.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql index cf9f312ab0c..4733690b225 100644 --- a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql +++ b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + insert into function file('test_02244', 'TSV', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; desc file('test_02244', 'TSV') settings column_names_for_schema_inference='x,y'; From 615efa1381b2afe824cb993b8ad26c2e9d94bbf2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 28 Mar 2022 19:15:53 -0400 Subject: [PATCH 125/372] aliases processing fixed --- src/Core/InterpolateDescription.cpp | 16 ++++- src/Core/InterpolateDescription.h | 6 +- src/Core/SortDescription.h | 2 - src/Interpreters/FillingRow.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 58 +++++++------------ src/Parsers/ASTInterpolateElement.cpp | 3 +- src/Parsers/ASTInterpolateElement.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ParserSelectQuery.cpp | 11 +++- .../Transforms/FillingTransform.cpp | 7 ++- 10 files changed, 55 insertions(+), 54 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 817f66aa4bc..5a19c51cc9f 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,14 +7,24 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_) + InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_, const Aliases & aliases) : actions(actions_) { for (const auto & name_type : actions->getRequiredColumnsWithTypes()) - required_columns_map[name_type.name] = name_type.type; + { + if (const auto & p = aliases.find(name_type.name); p != aliases.end()) + required_columns_map[p->second->getColumnName()] = name_type; + else + required_columns_map[name_type.name] = name_type; + } for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) - result_columns_map.insert(column.name); + { + if (const auto & p = aliases.find(column.name); p != aliases.end()) + result_columns_map.insert(p->second->getColumnName()); + else + result_columns_map.insert(column.name); + } } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 26ad4d1a11c..553c236b116 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -18,10 +19,11 @@ namespace DB /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ExpressionActionsPtr actions); + explicit InterpolateDescription(ExpressionActionsPtr actions, const Aliases & aliases); ExpressionActionsPtr actions; - std::unordered_map required_columns_map; /// input columns + + std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_map; /// result block columns /// filled externally in transform diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index ae5e9196d24..ffdff0ecd4c 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -25,8 +25,6 @@ class Block; struct FillColumnDescription { - DataTypePtr type; - /// All missed values in range [FROM, TO) will be filled /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index f5bbf036dd4..9679923f10b 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -118,7 +118,7 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & int filling_columns[i]->insert(filling_row[i]); } - if(size_t size = interpolate_block.columns()) + if (size_t size = interpolate_block.columns()) { Columns columns = interpolate_block.getColumns(); for (size_t i = 0; i < size; ++i) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b35bf31c81f..a2926347c24 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -763,10 +763,9 @@ static std::pair> getWithFillStep(const ASTPt throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, DataTypePtr type, ContextPtr context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) { FillColumnDescription descr; - descr.type = type; if (order_by_elem.fill_from) descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); @@ -811,7 +810,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or return descr; } -static SortDescription getSortDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -823,11 +822,9 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, Block bl std::shared_ptr collator; if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); - if (order_by_elem.with_fill) { - FillColumnDescription fill_desc = - getWithFillDescription(order_by_elem, block.getByName(order_by_elem.children.front()->getColumnName()).type, context); + FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc); } else @@ -837,34 +834,23 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, Block bl return order_descr; } -static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, const Aliases & aliases, ContextPtr context) { InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - std::map> position_map; - for (const auto & elem : query.interpolate()->children) - { - auto interpolate = elem->as(); - ColumnWithTypeAndName *block_column = block.findByName(interpolate.column->getColumnName()); - if (!block_column) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Missing column '{}' as an INTERPOLATE expression target", interpolate.column->getColumnName()); - - size_t position = block.getPositionByName(interpolate.column->getColumnName()); - position_map[position] = { - ColumnWithTypeAndName(block_column->type, block_column->name), - interpolate.expr->clone() - }; - - } - ColumnsWithTypeAndName columns; ASTPtr exprs = std::make_shared(); - for (auto & p : position_map) + for (const auto & elem : query.interpolate()->children) { - columns.emplace_back(std::move(p.second.first)); - exprs->children.emplace_back(std::move(p.second.second)); + const auto & interpolate = elem->as(); + ColumnWithTypeAndName *block_column = block.findByName(interpolate.column); + if (!block_column) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + + columns.emplace_back(block_column->type, block_column->name); + exprs->children.emplace_back(interpolate.expr->clone()); } auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); @@ -874,7 +860,7 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); - interpolate_descr = std::make_shared(std::make_shared(mergeDAG)); + interpolate_descr = std::make_shared(std::make_shared(mergeDAG), aliases); } return interpolate_descr; @@ -1401,7 +1387,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query, analysis_result.order_by_elements_actions, - getSortDescription(query, source_header, context), + getSortDescription(query, context), query_info.syntax_analyzer_result); } } @@ -2390,7 +2376,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, source_header, context); + SortDescription output_order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); if (input_sorting_info) @@ -2428,7 +2414,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, source_header,context); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); executeMergeSorted(query_plan, order_descr, limit, description); @@ -2532,7 +2518,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, source_header, context); + SortDescription order_descr = getSortDescription(query, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2543,7 +2529,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, context); + InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, context); auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } @@ -2582,7 +2568,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan) { if (!query.orderBy()) throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, source_header, context); + order_descr = getSortDescription(query, context); } auto limit = std::make_unique( diff --git a/src/Parsers/ASTInterpolateElement.cpp b/src/Parsers/ASTInterpolateElement.cpp index 3da94f772f2..15531962eb9 100644 --- a/src/Parsers/ASTInterpolateElement.cpp +++ b/src/Parsers/ASTInterpolateElement.cpp @@ -9,8 +9,7 @@ namespace DB void ASTInterpolateElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - column->formatImpl(settings, state, frame); - settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + settings.ostr << column << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); expr->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index e23b10cd434..ffb492787c9 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -9,7 +9,7 @@ namespace DB class ASTInterpolateElement : public IAST { public: - ASTPtr column; + String column; ASTPtr expr; String getID(char) const override { return "InterpolateElement"; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index a643cfb83ea..8eda7bba54a 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2335,7 +2335,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex expr = ident; auto elem = std::make_shared(); - elem->column = ident; + elem->column = ident->getColumnName(); elem->expr = expr; node = elem; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 65c89bd6f9b..a232451fdb9 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -265,7 +266,11 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { std::unordered_map columns; for (const auto & elem : select_expression_list->children) - columns[elem->getColumnName()] = elem; + { + std::string alias = elem->tryGetAlias(); + columns[alias.empty() ? elem->getColumnName() : alias] = elem; + } + for (const auto & elem : order_expression_list->children) if (elem->as()->with_fill) columns.erase(elem->as()->children.front()->getColumnName()); @@ -273,8 +278,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) for (const auto & [column, ast] : columns) { auto elem = std::make_shared(); - elem->column = ast; - elem->expr = ast; + elem->column = column; + elem->expr = std::make_shared(column); interpolate_expression_list->children.push_back(elem); } } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index f71c4f4f511..e2af77c9790 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -188,7 +188,7 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - interpolate_description->input_positions.emplace_back(idx, NameAndTypePair(column.name, p->second)); + interpolate_description->input_positions.emplace_back(idx, p->second); if (!is_fill_column[idx]) { @@ -250,7 +250,8 @@ void FillingTransform::transform(Chunk & chunk) Block interpolate_block; - auto interpolate = [&]() { + auto interpolate = [&]() + { if (interpolate_description) { interpolate_block.clear(); @@ -267,7 +268,7 @@ void FillingTransform::transform(Chunk & chunk) if (last_row.size() > col_pos && last_row[col_pos]->size()) column->insertFrom(*last_row[col_pos], 0); else - column->insertDefault(); + column->insertDefault(); } else column->insertFrom(*(*res_columns)[pos], size - 1); From 95565b45d640b5e1b1afb53a1b7692ed7d880fa7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Mar 2022 13:33:17 +0200 Subject: [PATCH 126/372] Fix --- programs/local/LocalServer.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 26d42a11315..0e02453dcd9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -455,16 +455,13 @@ void LocalServer::processConfig() auto logging = (config().has("logger.console") || config().has("logger.level") || config().has("log-level") + || config().has("send_logs_level") || config().has("logger.log")); - auto file_logging = config().has("server_logs_file"); - if (is_interactive && logging && !file_logging) - throw Exception("For interactive mode logging is allowed only with --server_logs_file option", - ErrorCodes::BAD_ARGUMENTS); + auto level = Poco::Logger::parseLevel(config().getString("log-level", config().getString("send_logs_level", "trace"))); - if (file_logging) + if (config().has("server_logs_file")) { - auto level = Poco::Logger::parseLevel(config().getString("log-level", "trace")); Poco::Logger::root().setLevel(level); Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::SimpleFileChannel(server_logs_file))); } @@ -472,6 +469,7 @@ void LocalServer::processConfig() { // force enable logging config().setString("logger", "logger"); + Poco::Logger::root().setLevel(level); // sensitive data rules are not used here buildLoggers(config(), logger(), "clickhouse-local"); } @@ -712,6 +710,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp config().setString("logger.log", options["logger.log"].as()); if (options.count("logger.level")) config().setString("logger.level", options["logger.level"].as()); + if (options.count("send_logs_level")) + config().setString("send_logs_level", options["send_logs_level"].as()); } } From 8edf6e74487a59b6e21cb679509dd8fd95ca1d59 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 29 Mar 2022 14:15:19 +0200 Subject: [PATCH 127/372] Mark test 02242_optimize_to_subcolumns_no_storage as backward incompatible for version 22.3.2.1 --- .../0_stateless/02242_optimize_to_subcolumns_no_storage.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql b/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql index e6e4663c5aa..8f8485eb58f 100644 --- a/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql +++ b/tests/queries/0_stateless/02242_optimize_to_subcolumns_no_storage.sql @@ -1,3 +1,4 @@ +-- Tags: no-backward-compatibility-check:22.3.2.1 SET optimize_functions_to_subcolumns = 1; SELECT count(*) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 WHERE (n1.number = n2.number) AND (n2.number = n3.number); From 97f5033ea98d7f1e06a21210ad9460a121300653 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Mar 2022 13:07:37 +0000 Subject: [PATCH 128/372] Fix tests --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 6 +++++- .../02166_arrow_dictionary_inference.reference | 2 +- .../0_stateless/02166_arrow_dictionary_inference.sh | 2 +- .../02211_shcema_inference_from_stdin.reference | 6 +++--- .../02240_tskv_schema_inference_bug.reference | 10 +++++----- .../02244_column_names_in_shcmea_inference.sql | 2 +- 6 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index bb58e851ff8..c792d828e44 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -553,7 +553,11 @@ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( auto arrow_column = std::make_shared(array_vector); std::unordered_map> dict_values; bool skipped = false; - ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values, false, false, skip_columns_with_unsupported_types, skipped); + bool allow_null_type = false; + if (hint_header && hint_header->has(field->name()) && hint_header->getByName(field->name()).type->isNullable()) + allow_null_type = true; + ColumnWithTypeAndName sample_column = readColumnFromArrowColumn( + arrow_column, field->name(), format_name, false, dict_values, false, allow_null_type, skip_columns_with_unsupported_types, skipped); if (!skipped) sample_columns.emplace_back(std::move(sample_column)); } diff --git a/tests/queries/0_stateless/02166_arrow_dictionary_inference.reference b/tests/queries/0_stateless/02166_arrow_dictionary_inference.reference index 46f448cfba7..20f3368e446 100644 --- a/tests/queries/0_stateless/02166_arrow_dictionary_inference.reference +++ b/tests/queries/0_stateless/02166_arrow_dictionary_inference.reference @@ -1 +1 @@ -x LowCardinality(UInt64) +x LowCardinality(Nullable(UInt64)) diff --git a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh index e560dc10d2c..7d313b571d9 100755 --- a/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh +++ b/tests/queries/0_stateless/02166_arrow_dictionary_inference.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1" +$CLICKHOUSE_CLIENT -q "insert into table function file('arrow.dict', 'Arrow', 'x LowCardinality(UInt64)') select number from numbers(10) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "desc file('arrow.dict', 'Arrow')" diff --git a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference index d176e0ee1ed..6920aa16198 100644 --- a/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference +++ b/tests/queries/0_stateless/02211_shcema_inference_from_stdin.reference @@ -9,7 +9,7 @@ x Nullable(Float64) 7 8 9 -c1 Nullable(String) -c2 Nullable(String) -c3 Nullable(String) +c1 Nullable(Float64) +c2 Nullable(Float64) +c3 Nullable(Float64) 1 2 3 diff --git a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference index a8abc33648e..7950770cfd4 100644 --- a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference +++ b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference @@ -1,8 +1,8 @@ b Nullable(String) -c Nullable(String) -a Nullable(String) -s1 \N 1 +c Array(Nullable(Float64)) +a Nullable(Float64) +s1 [] 1 } [2] 2 -\N \N \N -\N \N \N +\N [] \N +\N [] \N \N [3] \N diff --git a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql index 4733690b225..af56856f0be 100644 --- a/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql +++ b/tests/queries/0_stateless/02244_column_names_in_shcmea_inference.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into function file('test_02244', 'TSV', 'x String, y UInt32') select 'Hello, world!', 42 settings engine_file_truncate_on_insert=1; desc file('test_02244', 'TSV') settings column_names_for_schema_inference='x,y'; From a2fd09e0314f83d9e64ea758c973ea56f39d740c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 29 Mar 2022 16:34:07 +0200 Subject: [PATCH 129/372] Fix style --- src/Formats/EscapingRuleUtils.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index b3e36e9c14a..870202faf72 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -23,7 +23,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; } FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule) From 666ef3586c5d200a277b2a806b289f9781360628 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 29 Mar 2022 16:42:01 +0200 Subject: [PATCH 130/372] Fix typos check --- utils/check-style/codespell-ignore-words.list | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index d3a7586647c..7aabaff17c5 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -10,3 +10,4 @@ ths offsett numer ue +alse From 421b1e5a815a583ead854a4254cb02a0e6703799 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Mar 2022 17:33:02 +0200 Subject: [PATCH 131/372] Update --- src/Common/FileSegment.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 6 +++++- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 3 ++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 1dda76ba4b5..7d341d9bbb9 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -161,7 +161,7 @@ String FileSegment::getDownloader() const bool FileSegment::isDownloader() const { std::lock_guard segment_lock(mutex); - return getCallerIdImpl(true) == downloader_id; + return getCallerId() == downloader_id; } FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 91f448a2ea7..d106260e6b3 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.remote_fs_enable_cache; + // auto global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; + // bool with_cache = cache && settings.remote_fs_enable_cache && global_context; + bool with_cache = false; auto remote_file_reader_creator = [=, this]() { @@ -50,7 +52,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S if (with_cache) { if (IFileCache::isReadOnly()) + { settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; + } return std::make_shared( path, cache, remote_file_reader_creator, settings, read_until_position ? read_until_position : file_size); diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index bdb012a6376..4a3e0ba27a2 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -74,7 +74,8 @@ std::future ThreadPoolRemoteFSReader::submit(Reques ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::RemoteFSReadBytes, bytes_read); - if (running_group) + /// Query id cound be attached artificially. + if (running_group || (CurrentThread::isInitialized() && CurrentThread::getQueryId().size != 0)) thread_status.detachQuery(); return Result{ .size = bytes_read, .offset = offset }; From 3fc36627b3f2ff4b736bfecd9e20d0dc56792252 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Mar 2022 17:37:31 +0000 Subject: [PATCH 132/372] Allow to infer and parse bools as numbers in JSON input formats --- src/Core/Settings.h | 1 + .../Serializations/SerializationNumber.cpp | 4 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/JSONEachRowUtils.cpp | 3 +- src/Processors/Formats/ISchemaReader.cpp | 34 +++++++++++--- src/Processors/Formats/ISchemaReader.h | 6 ++- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 10 +++- .../Impl/JSONEachRowRowInputFormat.cpp | 4 +- .../RowInputFormatWithNamesAndTypes.cpp | 5 +- .../Formats/RowInputFormatWithNamesAndTypes.h | 3 +- ...02247_read_bools_as_numbers_json.reference | 18 +++++++ .../02247_read_bools_as_numbers_json.sh | 47 +++++++++++++++++++ 13 files changed, 120 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference create mode 100755 tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f81b61ea648..1b6fcd5ccb1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -633,6 +633,7 @@ class IColumn; M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \ M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \ M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \ + M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 4b6b79151bc..14c53dd5956 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -43,7 +43,7 @@ void SerializationNumber::serializeTextJSON(const IColumn & column, size_t ro } template -void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { bool has_quote = false; if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. @@ -67,7 +67,7 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & static constexpr bool is_uint8 = std::is_same_v; static constexpr bool is_int8 = std::is_same_v; - if (is_uint8 || is_int8) + if (settings.json.read_bools_as_numbers || is_uint8 || is_int8) { // extra conditions to parse true/false strings into 1/0 if (istr.eof()) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 3aa82cb79b4..d4e47f11d0d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -88,6 +88,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; + format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bd0a84d9ded..435579ad2f9 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -130,6 +130,7 @@ struct FormatSettings bool escape_forward_slashes = true; bool named_tuples_as_objects = false; bool serialize_as_strings = false; + bool read_bools_as_numbers = true; } json; struct diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index fb1ddb479f2..a56a27ec491 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -118,7 +119,7 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field) return nullptr; if (field.isBool()) - return makeNullable(std::make_shared()); + return DataTypeFactory::instance().get("Nullable(Bool)"); if (field.isInt64() || field.isUInt64() || field.isDouble()) return makeNullable(std::make_shared()); diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 096e39a2893..0fd44755445 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -10,8 +11,8 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) - : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_) { } @@ -39,9 +40,18 @@ NamesAndTypesList IRowSchemaReader::readSchema() data_types[i] = new_data_types[i]; /// If the new type and the previous type for this column are different, /// we will use default type if we have it or throw an exception. - else if (data_types[i]->getName() != new_data_types[i]->getName()) + else if (!data_types[i]->equals(*new_data_types[i])) { - if (default_type) + /// Check if we have Bool and Number and if allow_bools_as_numbers + /// is true make the result type Number + auto not_nullable_type = removeNullable(data_types[i]); + auto not_nullable_new_type = removeNullable(new_data_types[i]); + if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) + && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) { + if (isBool(not_nullable_type)) + data_types[i] = new_data_types[i]; + } + else if (default_type) data_types[i] = default_type; else throw Exception( @@ -89,8 +99,8 @@ NamesAndTypesList IRowSchemaReader::readSchema() return result; } -IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) - : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_) { } @@ -122,7 +132,17 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() /// we will use default type if we have it or throw an exception. else if (new_type && type->getName() != new_type->getName()) { - if (default_type) + /// Check if we have Bool and Number and if allow_bools_as_numbers + /// is true make the result type Number + auto not_nullable_type = removeNullable(type); + auto not_nullable_new_type = removeNullable(new_type); + if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) + && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) + { + if (isBool(not_nullable_type)) + type = new_type; + } + else if (default_type) type = default_type; else throw Exception( diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 36cf0656119..19673c3a651 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -36,7 +36,7 @@ protected: class IRowSchemaReader : public ISchemaReader { public: - IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; protected: @@ -51,6 +51,7 @@ protected: private: size_t max_rows_to_read; DataTypePtr default_type; + bool allow_bools_as_numbers; std::vector column_names; }; @@ -62,7 +63,7 @@ private: class IRowWithNamesSchemaReader : public ISchemaReader { public: - IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr, bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; bool hasStrictOrderOfColumns() const override { return false; } @@ -76,6 +77,7 @@ protected: private: size_t max_rows_to_read; DataTypePtr default_type; + bool allow_bools_as_numbers; }; /// Base class for schema inference for formats that don't need any data to diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index c087749d8d8..0496e3e41a8 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -182,7 +182,15 @@ bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & } JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) - : FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_) + : FormatWithNamesAndTypesSchemaReader( + in_, + format_settings_.max_rows_to_read_for_schema_inference, + with_names_, + with_types_, + &reader, + nullptr, + format_settings_.json.read_bools_as_numbers) + , reader(in_, yield_strings_, format_settings_) { } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 549fd7a6113..e132e4ebb9c 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -307,7 +307,9 @@ void JSONEachRowRowInputFormat::readSuffix() } JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings) - : IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference), json_strings(json_strings_) + : IRowWithNamesSchemaReader( + in_, format_settings.max_rows_to_read_for_schema_inference, nullptr, format_settings.json.read_bools_as_numbers) + , json_strings(json_strings_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 7720b01dc74..0157c54a5b3 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -299,8 +299,9 @@ FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader( bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, - DataTypePtr default_type_) - : IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) + DataTypePtr default_type_, + bool allow_bools_as_numbers_) + : IRowSchemaReader(in_, max_rows_to_read_, default_type_, allow_bools_as_numbers_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 25ffc8d6de2..8fbd426112c 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -128,7 +128,8 @@ public: bool with_names_, bool with_types_, FormatWithNamesAndTypesReader * format_reader_, - DataTypePtr default_type_ = nullptr); + DataTypePtr default_type_ = nullptr, + bool allow_bools_as_numbers_ = false); NamesAndTypesList readSchema() override; diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference new file mode 100644 index 00000000000..a7609bdd86b --- /dev/null +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.reference @@ -0,0 +1,18 @@ +x Nullable(Bool) +true +false +x Nullable(Float64) +42.42 +0 +x Nullable(Float64) +1 +0.42 +c1 Nullable(Bool) +true +false +c1 Nullable(Float64) +42.42 +0 +c1 Nullable(Float64) +1 +0.42 diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh new file mode 100755 index 00000000000..10f050ea6d1 --- /dev/null +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02247.data +DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME + +touch $DATA_FILE + +echo -e '{"x" : true} +{"x" : false}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"x" : 42.42} +{"x" : false}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"x" : true} +{"x" : 0.42}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + + +echo -e '[true] +[false]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e '[42.42] +[false]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e '[true] +[0.42]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + + +rm $DATA_FILE From 000f3043e763bc6f6a79522df430a45cea392c9d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Mar 2022 17:40:07 +0000 Subject: [PATCH 133/372] Make better --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 0fd44755445..796cdccbe8f 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -130,7 +130,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() type = new_type; /// If the new type and the previous type for this column are different, /// we will use default type if we have it or throw an exception. - else if (new_type && type->getName() != new_type->getName()) + else if (new_type && type->equals(*new_type)) { /// Check if we have Bool and Number and if allow_bools_as_numbers /// is true make the result type Number From f0e0d977524bfe9accb4ac2b134b1ad2e619e98f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Mar 2022 19:49:42 +0200 Subject: [PATCH 134/372] Fix lask of query id in merges --- src/Common/FileCache.cpp | 2 +- src/Common/FileCache.h | 2 +- src/Common/FileSegment.cpp | 3 +- src/Common/ThreadStatus.h | 5 ++++ src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 9 +----- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 28 ++++++++++++++++--- src/Disks/IO/ThreadPoolRemoteFSReader.h | 1 + src/Disks/S3/DiskS3.cpp | 6 ++++ src/IO/WriteBufferFromS3.cpp | 2 +- ...emote_filesystem_cache_on_insert.reference | 4 +-- 11 files changed, 44 insertions(+), 20 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 37a8ac78e98..f0d2bba33d4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -57,7 +57,7 @@ String IFileCache::getPathInLocalCache(const Key & key) return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str; } -bool IFileCache::isReadOnly() +bool IFileCache::shouldBypassCache() { return !CurrentThread::isInitialized() || !CurrentThread::get().getQueryContext() diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 90632a54edd..3a444b1c201 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -44,7 +44,7 @@ public: virtual void tryRemoveAll() = 0; /// If cache can be used as read only. (For merges, for example). - static bool isReadOnly(); + static bool shouldBypassCache(); /// Cache capacity in bytes. size_t capacity() const { return max_size; } diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 7d341d9bbb9..ae74e9b4c22 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -89,8 +89,7 @@ String FileSegment::getCallerId() String FileSegment::getCallerIdImpl(bool allow_non_strict_checking) { - /// Cache is read only, if it is read operation (which can potentially do cache writes), but there is no query attached. - if (IFileCache::isReadOnly()) + if (IFileCache::shouldBypassCache()) { /// getCallerId() can be called from completeImpl(), which can be called from complete(). /// complete() is called from destructor of CachedReadBufferFromRemoteFS when there is no query id anymore. diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index f3920474111..3d7ec08cdaf 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -216,6 +216,11 @@ public: return query_context.lock(); } + auto getGlobalContext() const + { + return global_context.lock(); + } + void disableProfiling() { assert(!query_profiler_real && !query_profiler_cpu); diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 84490d84801..a810c7b9f66 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -774,7 +774,7 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() void CachedReadBufferFromRemoteFS::assertCacheAllowed() const { - if (IFileCache::isReadOnly() && !settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + if (IFileCache::shouldBypassCache() && !settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache used when not allowed"); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index d106260e6b3..8f91804bbbe 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,9 +38,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - // auto global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; - // bool with_cache = cache && settings.remote_fs_enable_cache && global_context; - bool with_cache = false; + bool with_cache = cache && settings.remote_fs_enable_cache && !IFileCache::shouldBypassCache(); auto remote_file_reader_creator = [=, this]() { @@ -51,11 +49,6 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S if (with_cache) { - if (IFileCache::isReadOnly()) - { - settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; - } - return std::make_shared( path, cache, remote_file_reader_creator, settings, read_until_position ? read_until_position : file_size); } diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 4a3e0ba27a2..7e82dc627d5 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -1,5 +1,6 @@ #include "ThreadPoolRemoteFSReader.h" +#include #include #include #include @@ -50,18 +51,37 @@ std::future ThreadPoolRemoteFSReader::submit(Reques if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); + if (!query_context) + { + if (!shared_query_context) + { + ContextPtr global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; + if (global_context) + { + shared_query_context = Context::createCopy(global_context); + shared_query_context->makeQueryContext(); + } + } + + if (shared_query_context) + { + shared_query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); + query_context = shared_query_context; + } + } + auto task = std::make_shared>([request, running_group, query_context] { ThreadStatus thread_status; - /// Save query context if any, because cache implementation needs it. - if (query_context) - thread_status.attachQueryContext(query_context); - /// To be able to pass ProfileEvents. if (running_group) thread_status.attachQuery(running_group); + /// Save query context if any, because cache implementation needs it. + if (query_context) + thread_status.attachQueryContext(query_context); + setThreadName("VFSRead"); CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h index b2d5f11724a..a2a1e77c834 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.h +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.h @@ -15,6 +15,7 @@ class ThreadPoolRemoteFSReader : public IAsynchronousReader private: ThreadPool pool; + ContextMutablePtr shared_query_context; public: ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 23510cfee93..517972da876 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -228,7 +229,12 @@ std::unique_ptr DiskS3::readFile(const String & path, co ReadSettings disk_read_settings{read_settings}; if (cache) + { + if (IFileCache::shouldBypassCache()) + disk_read_settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; + disk_read_settings.remote_fs_cache = cache; + } auto s3_impl = std::make_unique( path, settings->client, bucket, metadata, diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index dbd8e1cf743..0eee7366775 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -161,7 +161,7 @@ WriteBufferFromS3::~WriteBufferFromS3() bool WriteBufferFromS3::cacheEnabled() const { - return cache != nullptr; + return cache != nullptr && !IFileCache::shouldBypassCache(); } void WriteBufferFromS3::preFinalize() diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 941ab9089f7..1f470d5644f 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -55,8 +55,8 @@ SELECT count() FROM system.remote_filesystem_cache; 21 OPTIMIZE TABLE test FINAL; SELECT count() FROM system.remote_filesystem_cache; -24 +27 SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; SELECT count() FROM system.remote_filesystem_cache; -25 +28 From ce9131f905e58b7b523db24d022a69272b25f40e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Mar 2022 19:57:55 +0200 Subject: [PATCH 135/372] Fix unit test --- src/Common/tests/gtest_lru_file_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_lru_file_cache.cpp b/src/Common/tests/gtest_lru_file_cache.cpp index 77f4467fa36..8a45e881487 100644 --- a/src/Common/tests/gtest_lru_file_cache.cpp +++ b/src/Common/tests/gtest_lru_file_cache.cpp @@ -496,7 +496,7 @@ TEST(LRUFileCache, get) /// Test max file segment size auto settings2 = settings; - settings.max_file_segment_size = 10; + settings2.max_file_segment_size = 10; auto cache2 = DB::LRUFileCache(caches_dir / "cache2", settings2); cache2.initialize(); From 0af6fdb5765e49c8c3fbebae3e6d031c26a772e6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 30 Mar 2022 11:28:21 +0800 Subject: [PATCH 136/372] fix building --- src/Storages/Hive/HiveFile.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index d1765da6b28..407d9602b61 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -135,8 +135,8 @@ void HiveOrcFile::prepareReader() void HiveOrcFile::prepareColumnMapping() { const orc::Type & type = reader->GetRawORCReader()->getType(); - size_t size = type.getSubtypeCount(); - for (size_t pos = 0; pos < size; pos++) + size_t count = type.getSubtypeCount(); + for (size_t pos = 0; pos < count; pos++) { /// Column names in hive is case-insensitive. String column{type.getFieldName(pos)}; @@ -156,9 +156,9 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c if (!statistics) return nullptr; - size_t size = index_names_and_types.size(); + size_t range_num = index_names_and_types.size(); auto idx = std::make_unique(); - idx->hyperrectangle.resize(size); + idx->hyperrectangle.resize(range_num); size_t i = 0; for (const auto & name_type : index_names_and_types) From 08e1fdc6b8a4d6fb74f76d29861e97a1accdf26e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 30 Mar 2022 16:19:16 +0800 Subject: [PATCH 137/372] Fixed :a elimate AMBIGUOUS_COLUMN_NAME exception elimate AMBIGUOUS_COLUMN_NAME exception in hdfs/s3 cluster when run a query with join --- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 2 +- tests/integration/test_s3_cluster/test.py | 19 +++++++++++++++++++ tests/integration/test_storage_hdfs/test.py | 10 ++++++++++ 4 files changed, 31 insertions(+), 2 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index b039caa4330..3e02fd46596 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -93,7 +93,7 @@ Pipe StorageHDFSCluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.query), + queryToString(query_info.original_query), header, context, /*throttler=*/nullptr, diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 3c031d545fe..9e8fbfca04c 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -134,7 +134,7 @@ Pipe StorageS3Cluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.query), + queryToString(query_info.original_query), header, context, /*throttler=*/nullptr, diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 561d3e3ed28..be0715e069e 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -153,3 +153,22 @@ def test_wrong_cluster(started_cluster): ) assert "not found" in error + + +def test_ambiguous_join(started_cluster): + node = started_cluster.instances["s0_0_0"] + result = node.query( + """ + SELECT l.name, r.value from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as l + JOIN s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as r + ON l.name = r.name + """ + ) + assert "AMBIGUOUS_COLUMN_NAME" not in result + \ No newline at end of file diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 7f340424ccf..85f1f79dee0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -553,6 +553,16 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 +def test_cluster_join(started_cluster): + result = node1.query( + ''' + SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l + JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r + ON l.id = r.id + ''' + ) + assert "AMBIGUOUS_COLUMN_NAME" not in result + if __name__ == "__main__": cluster.start() From 98ad3f4a911f10ef58cf102c6055652ad40d66f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Mar 2022 11:54:42 +0200 Subject: [PATCH 138/372] Fix tests, rename some remote mentions --- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 8 +++--- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 4 +-- src/Disks/S3/DiskS3.cpp | 1 - ...e.cpp => StorageSystemFilesystemCache.cpp} | 9 +++---- ...Cache.h => StorageSystemFilesystemCache.h} | 10 ++++---- src/Storages/System/attachSystemTables.cpp | 4 +-- .../0_stateless/02226_s3_with_cache.reference | 2 ++ .../0_stateless/02226_s3_with_cache.sql | 25 +++++++++++++++++++ ...0_system_remote_filesystem_cache.reference | 21 ++++++++-------- .../02240_system_remote_filesystem_cache.sql | 13 +++++----- 10 files changed, 60 insertions(+), 37 deletions(-) rename src/Storages/System/{StorageSystemRemoteFilesystemCache.cpp => StorageSystemFilesystemCache.cpp} (80%) rename src/Storages/System/{StorageSystemRemoteFilesystemCache.h => StorageSystemFilesystemCache.h} (64%) diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index a810c7b9f66..0bd06e44496 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -380,14 +380,12 @@ bool CachedReadBufferFromRemoteFS::completeFileSegmentAndGetNext() if (current_file_segment_it == file_segments_holder->file_segments.end()) return false; - file_segment = *current_file_segment_it; - - implementation_buffer = getImplementationBuffer(file_segment); + implementation_buffer = getImplementationBuffer(*current_file_segment_it); if (read_type == ReadType::CACHED) - file_segment->hit(); + (*current_file_segment_it)->hit(); - LOG_TEST(log, "New segment: {}", file_segment->range().toString()); + LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString()); return true; } diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 7e82dc627d5..83a5d8b276a 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -94,9 +94,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::RemoteFSReadBytes, bytes_read); - /// Query id cound be attached artificially. - if (running_group || (CurrentThread::isInitialized() && CurrentThread::getQueryId().size != 0)) - thread_status.detachQuery(); + thread_status.detachQuery(/* if_not_detached */true); return Result{ .size = bytes_read, .offset = offset }; }); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 517972da876..e682adb1487 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -272,7 +272,6 @@ std::unique_ptr DiskS3::writeFile(const String & path, LOG_TRACE(log, "{} to file by path: {}. S3 path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + blob_name); - bool cache_on_insert = fs::path(path).extension() != ".tmp" && write_settings.remote_fs_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp similarity index 80% rename from src/Storages/System/StorageSystemRemoteFilesystemCache.cpp rename to src/Storages/System/StorageSystemFilesystemCache.cpp index c0d8ffc67bf..08a62c47f27 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -1,4 +1,4 @@ -#include "StorageSystemRemoteFilesystemCache.h" +#include "StorageSystemFilesystemCache.h" #include #include #include @@ -6,13 +6,12 @@ #include #include #include -#include namespace DB { -NamesAndTypesList StorageSystemRemoteFilesystemCache::getNamesAndTypes() +NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() { return { {"cache_base_path", std::make_shared()}, @@ -26,12 +25,12 @@ NamesAndTypesList StorageSystemRemoteFilesystemCache::getNamesAndTypes() }; } -StorageSystemRemoteFilesystemCache::StorageSystemRemoteFilesystemCache(const StorageID & table_id_) +StorageSystemFilesystemCache::StorageSystemFilesystemCache(const StorageID & table_id_) : IStorageSystemOneBlock(table_id_) { } -void StorageSystemRemoteFilesystemCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { auto caches = FileCacheFactory::instance().getAll(); diff --git a/src/Storages/System/StorageSystemRemoteFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h similarity index 64% rename from src/Storages/System/StorageSystemRemoteFilesystemCache.h rename to src/Storages/System/StorageSystemFilesystemCache.h index b4ace8a7fe8..cfd938a50ce 100644 --- a/src/Storages/System/StorageSystemRemoteFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -27,17 +27,17 @@ namespace DB * FORMAT Vertical */ -class StorageSystemRemoteFilesystemCache final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemFilesystemCache final : public shared_ptr_helper, + public IStorageSystemOneBlock { - friend struct shared_ptr_helper; + friend struct shared_ptr_helper; public: - std::string getName() const override { return "SystemRemoteFilesystemCache"; } + std::string getName() const override { return "SystemFilesystemCache"; } static NamesAndTypesList getNamesAndTypes(); protected: - explicit StorageSystemRemoteFilesystemCache(const StorageID & table_id_); + explicit StorageSystemFilesystemCache(const StorageID & table_id_); void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index db30c265dc2..f84f26a5a78 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,7 +68,7 @@ #include #include #include -#include +#include #include #ifdef OS_LINUX @@ -161,7 +161,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "replicated_fetches"); attach(context, system_database, "part_moves_between_shards"); attach(context, system_database, "asynchronous_inserts"); - attach(context, system_database, "remote_filesystem_cache"); + attach(context, system_database, "filesystem_cache"); attach(context, system_database, "remote_data_paths"); if (has_zookeeper) diff --git a/tests/queries/0_stateless/02226_s3_with_cache.reference b/tests/queries/0_stateless/02226_s3_with_cache.reference index 214addac2d6..4041f51b3f9 100644 --- a/tests/queries/0_stateless/02226_s3_with_cache.reference +++ b/tests/queries/0_stateless/02226_s3_with_cache.reference @@ -1,2 +1,4 @@ SELECT 1, * FROM test LIMIT 10 FORMAT Null; 1 0 1 SELECT 2, * FROM test LIMIT 10 FORMAT Null; 0 1 0 +0 +SELECT 3, * FROM test LIMIT 10 FORMAT Null; 1 1 0 diff --git a/tests/queries/0_stateless/02226_s3_with_cache.sql b/tests/queries/0_stateless/02226_s3_with_cache.sql index b3126a419df..5b0d4ff3e44 100644 --- a/tests/queries/0_stateless/02226_s3_with_cache.sql +++ b/tests/queries/0_stateless/02226_s3_with_cache.sql @@ -1,7 +1,9 @@ -- Tags: no-parallel, no-fasttest, long SET max_memory_usage='20G'; +SET remote_fs_cache_on_write_operations = 0; +DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 10000; @@ -41,4 +43,27 @@ SET remote_filesystem_read_method='threadpool'; SELECT * FROM test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; +SET remote_fs_cache_on_write_operations = 1; + +TRUNCATE TABLE test; +SELECT count() FROM test; + +SYSTEM DROP REMOTE FILESYSTEM CACHE; + +INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 10000; + +SELECT 3, * FROM test LIMIT 10 FORMAT Null; + +SYSTEM FLUSH LOGS; +SELECT query, + ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read, + ProfileEvents['RemoteFSCacheReadBytes'] > 0 as remote_fs_cache_read, + ProfileEvents['RemoteFSCacheDownloadBytes'] > 0 as remote_fs_read_and_download +FROM system.query_log +WHERE query LIKE 'SELECT 3, * FROM test LIMIT%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; + DROP TABLE test; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference index db07f1dd577..59c4d43d8ae 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -1,18 +1,19 @@ --- Tags: no-parallel - -- { echo } SYSTEM DROP REMOTE FILESYSTEM CACHE; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +SET remote_fs_cache_on_write_operations=0; DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; -./disks/s3/data_cache/ (0,745) 746 +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; +./disks/s3/data_cache/ 0 0 1 +./disks/s3/data_cache/ 0 79 80 +./disks/s3/data_cache/ 0 745 746 SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; -./disks/s3/data_cache/ (0,745) 746 +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +./disks/s3/data_cache/ 0 745 746 SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index fb6dd8d61b4..24ea62aabf8 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -3,15 +3,16 @@ -- { echo } SYSTEM DROP REMOTE FILESYSTEM CACHE; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +SET remote_fs_cache_on_write_operations=0; DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range, size FROM system.remote_filesystem_cache; +SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; From 8c05a3dffc662b2d638c0bbcbb4993b3e175149e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 30 Mar 2022 15:00:32 +0200 Subject: [PATCH 139/372] Update 01825_type_json_parallel_insert.sql --- tests/queries/0_stateless/01825_type_json_parallel_insert.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_type_json_parallel_insert.sql b/tests/queries/0_stateless/01825_type_json_parallel_insert.sql index f54004a6630..93d1eecfbd7 100644 --- a/tests/queries/0_stateless/01825_type_json_parallel_insert.sql +++ b/tests/queries/0_stateless/01825_type_json_parallel_insert.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-backward-compatibility-check:22.3.2.1 DROP TABLE IF EXISTS t_json_parallel; SET allow_experimental_object_type = 1, max_insert_threads = 20, max_threads = 20; From 265fbaaa5aadf4e65e3569e840550690b3025586 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 30 Mar 2022 13:23:43 +0000 Subject: [PATCH 140/372] Fix tests --- .../02245_s3_schema_desc.reference | 24 +++++++++---------- .../0_stateless/02245_s3_schema_desc.sql | 17 ++++++------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.reference b/tests/queries/0_stateless/02245_s3_schema_desc.reference index a5b0f81a2c7..e039680d933 100644 --- a/tests/queries/0_stateless/02245_s3_schema_desc.reference +++ b/tests/queries/0_stateless/02245_s3_schema_desc.reference @@ -1,21 +1,21 @@ -c1 Nullable(String) -c2 Nullable(String) -c3 Nullable(String) -c1 Nullable(String) -c2 Nullable(String) -c3 Nullable(String) +c1 Nullable(Float64) +c2 Nullable(Float64) +c3 Nullable(Float64) +c1 Nullable(Float64) +c2 Nullable(Float64) +c3 Nullable(Float64) c1 UInt64 c2 UInt64 c3 UInt64 -c1 Nullable(String) -c2 Nullable(String) -c3 Nullable(String) +c1 Nullable(Float64) +c2 Nullable(Float64) +c3 Nullable(Float64) c1 UInt64 c2 UInt64 c3 UInt64 -c1 Nullable(String) -c2 Nullable(String) -c3 Nullable(String) +c1 Nullable(Float64) +c2 Nullable(Float64) +c3 Nullable(Float64) c1 UInt64 c2 UInt64 c3 UInt64 diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.sql b/tests/queries/0_stateless/02245_s3_schema_desc.sql index 4ab870e1379..72800f3ab38 100644 --- a/tests/queries/0_stateless/02245_s3_schema_desc.sql +++ b/tests/queries/0_stateless/02245_s3_schema_desc.sql @@ -1,13 +1,14 @@ -- Tags: no-fasttest -- Tag no-fasttest: Depends on AWS -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); -desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); +desc file('data_minio/{a,b,c}.tsv'); +desc file('data_minio/{a,b,c}.tsv', 'TSV'); +desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc file('data_minio/{a,b,c}.tsv'); +desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); +desc file('data_minio/{a,b,c}.tsv', 'TSV'); +desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); + SELECT * FROM s3(decodeURLComponent(NULL), [NULL]); --{serverError 170} From 0fc92fe2aaf615b49eed9fbaf688422dbf6c589c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Mar 2022 13:47:44 +0200 Subject: [PATCH 141/372] Get rid of all "remote" mentions --- docker/test/stateful/run.sh | 2 +- src/Common/FileCache.cpp | 1 + src/Common/FileCache.h | 1 - src/Core/Settings.h | 7 +-- src/Disks/DiskCacheWrapper.cpp | 6 +-- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 6 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 4 +- src/IO/ReadSettings.h | 6 +-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 7 +-- src/Interpreters/InterpreterSystemQuery.cpp | 8 ++-- src/Parsers/ASTSystemQuery.h | 4 +- .../System/StorageSystemFilesystemCache.h | 2 +- tests/config/users.d/s3_cache.xml | 8 ++++ ...605_adaptive_granularity_block_borders.sql | 2 +- .../01641_memory_tracking_insert_optimize.sql | 2 +- .../0_stateless/01926_order_by_desc_limit.sql | 2 +- .../0_stateless/02226_s3_with_cache.sql | 6 +-- ...0_system_remote_filesystem_cache.reference | 10 ++-- .../02240_system_remote_filesystem_cache.sql | 10 ++-- ...emote_filesystem_cache_on_insert.reference | 46 +++++++++---------- ...2241_remote_filesystem_cache_on_insert.sql | 46 +++++++++---------- 23 files changed, 100 insertions(+), 90 deletions(-) create mode 100644 tests/config/users.d/s3_cache.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 77dc61e6cd0..e91acaa0b2a 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -96,7 +96,7 @@ else clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" + clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0" fi clickhouse-client --query "SHOW TABLES FROM test" diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index f0d2bba33d4..0eb53f71bc4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -720,6 +720,7 @@ std::vector LRUFileCache::tryGetCachePaths(const Key & key) std::lock_guard cache_lock(mutex); std::vector cache_paths; + const auto & cells_by_offset = files[key]; for (const auto & [offset, cell] : cells_by_offset) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 3a444b1c201..e8280fba08a 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -43,7 +43,6 @@ public: virtual void tryRemoveAll() = 0; - /// If cache can be used as read only. (For merges, for example). static bool shouldBypassCache(); /// Cache capacity in bytes. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86dc16f6220..8891a49a7de 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -555,9 +555,10 @@ class IColumn; \ M(UInt64, remote_fs_read_max_backoff_ms, 10000, "Max wait time when trying to read data for remote disk", 0) \ M(UInt64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ - M(Bool, remote_fs_enable_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ - M(UInt64, remote_fs_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ - M(Bool, remote_fs_cache_on_write_operations, true, "Write into cache on INSERT query To actually work this setting requires be added to disk config too", 0) \ + M(Bool, enable_filesystem_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ + M(UInt64, filesystem_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ + M(Bool, enable_filesystem_cache_on_write_operations, false, "Write into cache on write operations. To actually work this setting requires be added to disk config too", 0) \ + M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache, false, "", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 568fbf160c0..178caa0c496 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -150,7 +150,7 @@ DiskCacheWrapper::readFile( /// Note: enabling `threadpool` read requires to call setReadUntilEnd(). current_read_settings.remote_fs_method = RemoteFSReadMethod::read; /// Disable data cache. - current_read_settings.remote_fs_enable_cache = false; + current_read_settings.enable_filesystem_cache = false; if (metadata->status == DOWNLOADING) { @@ -169,7 +169,7 @@ DiskCacheWrapper::readFile( auto src_buffer = DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); WriteSettings write_settings; - write_settings.remote_fs_cache_on_write_operations = false; + write_settings.enable_filesystem_cache_on_write_operations = false; auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite, write_settings); copyData(*src_buffer, *dst_buffer); @@ -206,7 +206,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode return DiskDecorator::writeFile(path, buf_size, mode, settings); WriteSettings current_settings = settings; - current_settings.remote_fs_cache_on_write_operations = false; + current_settings.enable_filesystem_cache_on_write_operations = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 0bd06e44496..6aa5f71139a 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -120,12 +120,12 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getReadBufferForFileSegment( auto range = file_segment->range(); /// Each wait() call has a timeout of 1 second. - size_t wait_download_max_tries = settings.remote_fs_cache_max_wait_sec; + size_t wait_download_max_tries = settings.filesystem_cache_max_wait_sec; size_t wait_download_tries = 0; auto download_state = file_segment->state(); - if (settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) { if (download_state == FileSegment::State::DOWNLOADED) { @@ -772,7 +772,7 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() void CachedReadBufferFromRemoteFS::assertCacheAllowed() const { - if (IFileCache::shouldBypassCache() && !settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + if (IFileCache::shouldBypassCache() && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache used when not allowed"); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8f91804bbbe..abbcd5c8add 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.remote_fs_enable_cache && !IFileCache::shouldBypassCache(); + bool with_cache = cache && settings.enable_filesystem_cache && !IFileCache::shouldBypassCache(); auto remote_file_reader_creator = [=, this]() { diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index e682adb1487..07d27f67d1e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -231,7 +231,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co if (cache) { if (IFileCache::shouldBypassCache()) - disk_read_settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; + disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; disk_read_settings.remote_fs_cache = cache; } @@ -273,7 +273,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + blob_name); bool cache_on_insert = fs::path(path).extension() != ".tmp" - && write_settings.remote_fs_cache_on_write_operations + && write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; auto s3_buffer = std::make_unique( diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 936de1673b4..92346615a7a 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,9 +77,9 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; - bool remote_fs_enable_cache = true; - size_t remote_fs_cache_max_wait_sec = 1; - bool remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = false; + bool enable_filesystem_cache = true; + size_t filesystem_cache_max_wait_sec = 1; + bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; size_t remote_read_min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 81a6705cbab..af26452e8e6 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -5,7 +5,7 @@ namespace DB struct WriteSettings { - bool remote_fs_cache_on_write_operations = false; + bool enable_filesystem_cache_on_write_operations = false; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a3169f435e4..83236781418 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3179,8 +3179,9 @@ ReadSettings Context::getReadSettings() const res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms; res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; - res.remote_fs_enable_cache = settings.remote_fs_enable_cache; - res.remote_fs_cache_max_wait_sec = settings.remote_fs_cache_max_wait_sec; + res.enable_filesystem_cache = settings.enable_filesystem_cache; + res.filesystem_cache_max_wait_sec = settings.filesystem_cache_max_wait_sec; + res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; res.remote_read_min_bytes_for_seek = settings.remote_read_min_bytes_for_seek; @@ -3203,7 +3204,7 @@ WriteSettings Context::getWriteSettings() const { WriteSettings res; - res.remote_fs_cache_on_write_operations = settings.remote_fs_cache_on_write_operations; + res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; return res; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 5f030159b2e..353e5393a03 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -298,9 +298,9 @@ BlockIO InterpreterSystemQuery::execute() cache->reset(); break; #endif - case Type::DROP_REMOTE_FILESYSTEM_CACHE: + case Type::DROP_FILESYSTEM_CACHE: { - if (query.remote_filesystem_cache_path.empty()) + if (query.filesystem_cache_path.empty()) { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) @@ -308,7 +308,7 @@ BlockIO InterpreterSystemQuery::execute() } else { - auto cache = FileCacheFactory::instance().get(query.remote_filesystem_cache_path); + auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path); cache->tryRemoveAll(); } break; @@ -775,7 +775,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_UNCOMPRESSED_CACHE: case Type::DROP_INDEX_MARK_CACHE: case Type::DROP_INDEX_UNCOMPRESSED_CACHE: - case Type::DROP_REMOTE_FILESYSTEM_CACHE: + case Type::DROP_FILESYSTEM_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 7113698789f..600525f9abe 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -28,7 +28,7 @@ public: #if USE_EMBEDDED_COMPILER DROP_COMPILED_EXPRESSION_CACHE, #endif - DROP_REMOTE_FILESYSTEM_CACHE, + DROP_FILESYSTEM_CACHE, STOP_LISTEN_QUERIES, START_LISTEN_QUERIES, RESTART_REPLICAS, @@ -89,7 +89,7 @@ public: String volume; String disk; UInt64 seconds{}; - String remote_filesystem_cache_path; + String filesystem_cache_path; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Storages/System/StorageSystemFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h index cfd938a50ce..0f0bd81e760 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -23,7 +23,7 @@ namespace DB * remote_path * FROM system.remote_data_paths * ) AS data_paths - * INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path + * INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path * FORMAT Vertical */ diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml new file mode 100644 index 00000000000..69b24ecbbc4 --- /dev/null +++ b/tests/config/users.d/s3_cache.xml @@ -0,0 +1,8 @@ + + + + 1 + 1 + + + diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 750809da338..7654be4eb29 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -22,7 +22,7 @@ OPTIMIZE TABLE adaptive_table FINAL; SELECT marks FROM system.parts WHERE table = 'adaptive_table' and database=currentDatabase() and active; -SET remote_fs_enable_cache = 0; +SET enable_filesystem_cache = 0; -- If we have computed granularity incorrectly than we will exceed this limit. SET max_memory_usage='30M'; diff --git a/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql b/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql index 7ec3153886c..36b6c97460c 100644 --- a/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql +++ b/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql @@ -3,7 +3,7 @@ drop table if exists data_01641; -- Disable cache for s3 storage tests because it increases memory usage. -set remote_fs_enable_cache=0; +set enable_filesystem_cache=0; set remote_filesystem_read_method='read'; create table data_01641 (key Int, value String) engine=MergeTree order by (key, repeat(value, 40)) settings old_parts_lifetime=0, min_bytes_for_wide_part=0; diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index 9f65cf73252..86468b4fcd6 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS order_by_desc; -SET remote_fs_enable_cache=0; +SET enable_filesystem_cache=0; CREATE TABLE order_by_desc (u UInt32, s String) ENGINE MergeTree ORDER BY u PARTITION BY u % 100 diff --git a/tests/queries/0_stateless/02226_s3_with_cache.sql b/tests/queries/0_stateless/02226_s3_with_cache.sql index 5b0d4ff3e44..d470f2ef140 100644 --- a/tests/queries/0_stateless/02226_s3_with_cache.sql +++ b/tests/queries/0_stateless/02226_s3_with_cache.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel, no-fasttest, long SET max_memory_usage='20G'; -SET remote_fs_cache_on_write_operations = 0; +SET enable_filesystem_cache_on_write_operations = 0; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; @@ -43,12 +43,12 @@ SET remote_filesystem_read_method='threadpool'; SELECT * FROM test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; -SET remote_fs_cache_on_write_operations = 1; +SET enable_filesystem_cache_on_write_operations = 1; TRUNCATE TABLE test; SELECT count() FROM test; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 10000; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference index 59c4d43d8ae..20cb2329604 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -1,19 +1,19 @@ -- { echo } -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SET remote_fs_cache_on_write_operations=0; +SYSTEM DROP FILESYSTEM CACHE; +SET enable_filesystem_cache_on_write_operations=0; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; ./disks/s3/data_cache/ 0 0 1 ./disks/s3/data_cache/ 0 79 80 ./disks/s3/data_cache/ 0 745 746 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; ./disks/s3/data_cache/ 0 745 746 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index 24ea62aabf8..a889bea0fcf 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -2,17 +2,17 @@ -- { echo } -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SET remote_fs_cache_on_write_operations=0; +SYSTEM DROP FILESYSTEM CACHE; +SET enable_filesystem_cache_on_write_operations=0; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 1f470d5644f..e83b5551821 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -1,62 +1,62 @@ -- { echo } -SET remote_fs_cache_on_write_operations=1; +SET enable_filesystem_cache_on_write_operations=1; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SYSTEM DROP FILESYSTEM CACHE; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 0 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 0 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 745 size: 746 state: DOWNLOADED -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 0 SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 2 SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 2 -SELECT count() size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.filesystem_cache; 7 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; -SELECT count() FROM system.remote_filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=0; +SELECT count() FROM system.filesystem_cache; 7 INSERT INTO test SELECT number, toString(number) FROM numbers(100); INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 21 OPTIMIZE TABLE test FINAL; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 27 SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 28 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index cd255d7df51..745af904c5f 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -2,51 +2,51 @@ -- { echo } -SET remote_fs_cache_on_write_operations=1; +SET enable_filesystem_cache_on_write_operations=1; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.filesystem_cache; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; -SELECT count() size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.filesystem_cache; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SELECT count() FROM system.filesystem_cache; -SELECT count() FROM system.remote_filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=0; +SELECT count() FROM system.filesystem_cache; INSERT INTO test SELECT number, toString(number) FROM numbers(100); INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; OPTIMIZE TABLE test FINAL; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; From 996d57b918519bb9a71503fd1a525ce7090234bb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Mar 2022 17:42:47 +0200 Subject: [PATCH 142/372] try fix build with gcc --- src/Interpreters/TransactionLog.h | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index af741f7e76e..86584a74c68 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -24,25 +24,22 @@ public: if (likely(ptr)) return *ptr; - std::lock_guard lock{instance_mutex}; - if (!instance_holder.has_value()) - { - instance_holder.emplace(); - instance_raw_ptr = &instance_holder.value(); - } - return instance_holder.value(); + return createInstanceOrThrow(); } static void shutdownIfAny() { std::lock_guard lock{instance_mutex}; - if (instance_holder.has_value()) + if (instance_holder) instance_holder->shutdown(); } private: + static Derived & createInstanceOrThrow(); + static inline std::atomic instance_raw_ptr; - static inline std::optional instance_holder; + /// It was supposed to be std::optional, but gcc fails to compile it for some reason + static inline std::shared_ptr instance_holder; static inline std::mutex instance_mutex; }; @@ -180,4 +177,16 @@ private: ThreadFromGlobalPool updating_thread; }; +template +Derived & SingletonHelper::createInstanceOrThrow() +{ + std::lock_guard lock{instance_mutex}; + if (!instance_holder) + { + instance_holder = std::make_shared(); + instance_raw_ptr = instance_holder.get(); + } + return *instance_holder; +} + } From 738966b6b6bf1683787906c692308e8f870bc040 Mon Sep 17 00:00:00 2001 From: shuchaome Date: Thu, 31 Mar 2022 00:03:45 +0800 Subject: [PATCH 143/372] fix filebuffer pos in RemoteReadBuffer --- src/Storages/Cache/ExternalDataSourceCache.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 18607c16ffa..17966d49c74 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -94,6 +94,8 @@ bool RemoteReadBuffer::nextImpl() return status; } + //file_buffer::pos should increase correspondingly when RemoteReadBuffer is consumed, otherwise start_offset will be incorrect. + local_file_holder->file_buffer->position() = local_file_holder->file_buffer->buffer().begin() + BufferBase::offset(); auto start_offset = local_file_holder->file_buffer->getPosition(); auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size(); local_file_holder->file_cache_controller->value().waitMoreData(start_offset, end_offset); From 6a1e116c467b9dfe055f3b01a4c3656507ae0c63 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 30 Mar 2022 16:34:19 -0400 Subject: [PATCH 144/372] refactoring --- src/Core/InterpolateDescription.cpp | 10 ++--- src/Core/InterpolateDescription.h | 9 ++--- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++-- src/Processors/QueryPlan/FillingStep.cpp | 2 +- .../Transforms/FillingTransform.cpp | 40 ++++++++++++------- src/Processors/Transforms/FillingTransform.h | 2 + 6 files changed, 41 insertions(+), 30 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 5a19c51cc9f..4147d254033 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -7,10 +7,10 @@ namespace DB { - InterpolateDescription::InterpolateDescription(ExpressionActionsPtr actions_, const Aliases & aliases) + InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) : actions(actions_) { - for (const auto & name_type : actions->getRequiredColumnsWithTypes()) + for (const auto & name_type : actions->getRequiredColumns()) { if (const auto & p = aliases.find(name_type.name); p != aliases.end()) required_columns_map[p->second->getColumnName()] = name_type; @@ -18,12 +18,12 @@ namespace DB required_columns_map[name_type.name] = name_type; } - for (const ColumnWithTypeAndName & column : actions->getSampleBlock()) + for (const ColumnWithTypeAndName & column : actions->getResultColumns()) { if (const auto & p = aliases.find(column.name); p != aliases.end()) - result_columns_map.insert(p->second->getColumnName()); + result_columns_set.insert(p->second->getColumnName()); else - result_columns_map.insert(column.name); + result_columns_set.insert(column.name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 553c236b116..b19674eb590 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -19,15 +19,12 @@ namespace DB /// Interpolate description struct InterpolateDescription { - explicit InterpolateDescription(ExpressionActionsPtr actions, const Aliases & aliases); + explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases); - ExpressionActionsPtr actions; + ActionsDAGPtr actions; std::unordered_map required_columns_map; /// input column name -> {alias, type} - std::unordered_set result_columns_map; /// result block columns - - /// filled externally in transform - std::vector> input_positions; /// positions in result columns required for actions + std::unordered_set result_columns_set; /// result block columns }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a2926347c24..6a9fc2cef43 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -855,12 +855,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(exprs, syntax_result, context); - ExpressionActionsPtr actions = analyzer.getActions(true, true, CompileExpressions::yes); - ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getActionsDAG().getResultColumns(), + ActionsDAGPtr actions = analyzer.getActionsDAG(true); + ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getResultColumns(), columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->getActionsDAG().clone()), std::move(*convDAG)); + ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->clone()), std::move(*convDAG)); - interpolate_descr = std::make_shared(std::make_shared(mergeDAG), aliases); + interpolate_descr = std::make_shared(mergeDAG, aliases); } return interpolate_descr; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index cfff191b464..fe8abcd3750 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -41,7 +41,7 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; - return std::make_shared(header, sort_description, interpolate_description, on_totals); + return std::make_shared(header, sort_description, std::move(interpolate_description), on_totals); }); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index e2af77c9790..b8dcf1f0bf9 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -149,10 +149,13 @@ FillingTransform::FillingTransform( if (on_totals) return; + if (interpolate_description) + interpolate_actions = std::make_shared(interpolate_description->actions); + std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->result_columns_map.count(sort_description[i].column_name)) + if (interpolate_description && interpolate_description->result_columns_set.count(sort_description[i].column_name)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", sort_description[i].column_name); @@ -188,11 +191,11 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - interpolate_description->input_positions.emplace_back(idx, p->second); + input_positions.emplace_back(idx, p->second); if (!is_fill_column[idx]) { - if (interpolate_description && interpolate_description->result_columns_map.count(column.name)) + if (interpolate_description && interpolate_description->result_columns_set.count(column.name)) interpolate_column_positions.push_back(idx); else other_column_positions.push_back(idx); @@ -256,34 +259,35 @@ void FillingTransform::transform(Chunk & chunk) { interpolate_block.clear(); - if (interpolate_description->input_positions.size()) + if (input_positions.size()) { - for (const auto & [col_pos, name_type] : interpolate_description->input_positions) + /// populate calculation block with required columns with values from previous row + for (const auto & [col_pos, name_type] : input_positions) { MutableColumnPtr column = name_type.type->createColumn(); auto [res_columns, pos] = res_map[col_pos]; size_t size = (*res_columns)[pos]->size(); - if (size == 0) + if (size == 0) /// this is the first row in current chunk { + /// take value from last row of previous chunk if exists, else use default if (last_row.size() > col_pos && last_row[col_pos]->size()) column->insertFrom(*last_row[col_pos], 0); else column->insertDefault(); } - else + else /// take value from previous row of current chunk column->insertFrom(*(*res_columns)[pos], size - 1); interpolate_block.insert({std::move(column), name_type.type, name_type.name}); } + interpolate_actions->execute(interpolate_block); } else /// all INTERPOLATE expressions are constants { - /// dirty hack - we need at least one column with one row to execute actions on block - DataTypePtr dt = std::make_shared(); - interpolate_block.insert({dt->createColumnConst(1, dt->getDefault()), dt, "dummy"}); + /// LOL :) + size_t n = 1; + interpolate_actions->execute(interpolate_block, n); } - - interpolate_description->actions->execute(interpolate_block); } }; @@ -303,8 +307,12 @@ void FillingTransform::transform(Chunk & chunk) insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - for (interpolate(); filling_row.next(next_row); interpolate()) + interpolate(); + while (filling_row.next(next_row)) + { insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + interpolate(); + } setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); return; @@ -362,8 +370,12 @@ void FillingTransform::transform(Chunk & chunk) insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); } - for (interpolate(); filling_row.next(next_row); interpolate()) + interpolate(); + while (filling_row.next(next_row)) + { insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + interpolate(); + } copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, old_interpolate_columns, row_ind); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 90f8d61ceb6..5e1068e18e2 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -40,6 +40,8 @@ private: Positions fill_column_positions; Positions interpolate_column_positions; Positions other_column_positions; + std::vector> input_positions; /// positions in result columns required for actions + ExpressionActionsPtr interpolate_actions; bool first = true; bool generate_suffix = false; From d0e5b3571cf6c0455e60312177694f13ef8bac57 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 31 Mar 2022 09:28:07 +0800 Subject: [PATCH 145/372] fixed code style --- tests/ci/clickhouse_helper.py | 2 +- tests/integration/test_s3_cluster/test.py | 6 +++--- tests/integration/test_storage_hdfs/test.py | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 218aaca8b91..d52b6262a78 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -15,7 +15,7 @@ class ClickHouseHelper: self.url = url self.auth = { "X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"), - "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password") + "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"), } @staticmethod diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index be0715e069e..93708acd49c 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -149,7 +149,8 @@ def test_wrong_cluster(started_cluster): SELECT count(*) from s3Cluster( 'non_existent_cluster', 'http://minio1:9001/root/data/{clickhouse,database}/*', - 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')""" + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + """ ) assert "not found" in error @@ -158,7 +159,7 @@ def test_wrong_cluster(started_cluster): def test_ambiguous_join(started_cluster): node = started_cluster.instances["s0_0_0"] result = node.query( - """ + """ SELECT l.name, r.value from s3Cluster( 'cluster_simple', 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', @@ -171,4 +172,3 @@ def test_ambiguous_join(started_cluster): """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result - \ No newline at end of file diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 85f1f79dee0..b6dcff6d174 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -555,11 +555,11 @@ def test_insert_select_schema_inference(started_cluster): def test_cluster_join(started_cluster): result = node1.query( - ''' + """ SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r ON l.id = r.id - ''' + """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result From 564a77c462eda1b1e432008d758f09b8d82a1a26 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 31 Mar 2022 12:49:23 +0200 Subject: [PATCH 146/372] Fix build --- src/Formats/EscapingRuleUtils.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 870202faf72..e9d7e464cce 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -466,8 +466,6 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe if (field.size() > 1 && ((field.front() == '\'' && field.back() == '\'') || (field.front() == '"' && field.back() == '"'))) { - auto s = std::string_view(field.data() + 1, field.size() - 2); - ReadBufferFromString buf(std::string_view(field.data() + 1, field.size() - 2)); /// Try to determine the type of value inside quotes auto type = determineDataTypeForSingleField(buf); From 4db5043ed45fffe512d716bd056a3dc5ae76a1d2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 31 Mar 2022 12:50:53 +0200 Subject: [PATCH 147/372] Fix test --- .../queries/0_stateless/02245_s3_schema_desc.sql | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.sql b/tests/queries/0_stateless/02245_s3_schema_desc.sql index 72800f3ab38..2cd362ff233 100644 --- a/tests/queries/0_stateless/02245_s3_schema_desc.sql +++ b/tests/queries/0_stateless/02245_s3_schema_desc.sql @@ -1,14 +1,14 @@ -- Tags: no-fasttest -- Tag no-fasttest: Depends on AWS -desc file('data_minio/{a,b,c}.tsv'); -desc file('data_minio/{a,b,c}.tsv', 'TSV'); -desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); -desc file('data_minio/{a,b,c}.tsv'); -desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); -desc file('data_minio/{a,b,c}.tsv', 'TSV'); -desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); -desc file('data_minio/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); SELECT * FROM s3(decodeURLComponent(NULL), [NULL]); --{serverError 170} From 74275da7eedfba63aaf954e546aab9f2d1bd27c5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Mar 2022 10:52:34 +0000 Subject: [PATCH 148/372] Make better --- src/Processors/Formats/ISchemaReader.cpp | 129 ++++++++++------------- 1 file changed, 58 insertions(+), 71 deletions(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 796cdccbe8f..615ebfb03be 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -11,6 +11,60 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } +static void chooseResultType( + DataTypePtr & type, + const DataTypePtr & new_type, + bool allow_bools_as_numbers, + const DataTypePtr & default_type, + const String & column_name, + size_t row) +{ + if (!type) + type = new_type; + + /// If the new type and the previous type for this column are different, + /// we will use default type if we have it or throw an exception. + if (new_type && type->equals(*new_type)) + { + /// Check if we have Bool and Number and if allow_bools_as_numbers + /// is true make the result type Number + auto not_nullable_type = removeNullable(type); + auto not_nullable_new_type = removeNullable(new_type); + if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) + && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) + { + if (isBool(not_nullable_type)) + type = new_type; + } + else if (default_type) + type = default_type; + else + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", + type->getName(), + column_name, + row, + new_type->getName()); + } +} + +static void checkTypeAndAppend(NamesAndTypesList & result, DataTypePtr & type, const String & name, const DataTypePtr & default_type, size_t max_rows_to_read) +{ + if (!type) + { + if (!default_type) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " + "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", + max_rows_to_read); + + type = default_type; + } + result.emplace_back(name, type); +} + IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_, bool allow_bools_as_numbers_) : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_), allow_bools_as_numbers(allow_bools_as_numbers_) { @@ -35,29 +89,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() if (!new_data_types[i]) continue; - /// If we couldn't determine the type of column yet, just set the new type. - if (!data_types[i]) - data_types[i] = new_data_types[i]; - /// If the new type and the previous type for this column are different, - /// we will use default type if we have it or throw an exception. - else if (!data_types[i]->equals(*new_data_types[i])) - { - /// Check if we have Bool and Number and if allow_bools_as_numbers - /// is true make the result type Number - auto not_nullable_type = removeNullable(data_types[i]); - auto not_nullable_new_type = removeNullable(new_data_types[i]); - if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) - && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) { - if (isBool(not_nullable_type)) - data_types[i] = new_data_types[i]; - } - else if (default_type) - data_types[i] = default_type; - else - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName()); - } + chooseResultType(data_types[i], new_data_types[i], allow_bools_as_numbers, default_type, std::to_string(i + 1), row); } } @@ -82,18 +114,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() for (size_t i = 0; i != data_types.size(); ++i) { /// Check that we could determine the type of this column. - if (!data_types[i]) - { - if (!default_type) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " - "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", - max_rows_to_read); - - data_types[i] = default_type; - } - result.emplace_back(column_names[i], data_types[i]); + checkTypeAndAppend(result, data_types[i], column_names[i], default_type, max_rows_to_read); } return result; @@ -125,30 +146,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() } auto & type = it->second; - /// If we couldn't determine the type of column yet, just set the new type. - if (!type) - type = new_type; - /// If the new type and the previous type for this column are different, - /// we will use default type if we have it or throw an exception. - else if (new_type && type->equals(*new_type)) - { - /// Check if we have Bool and Number and if allow_bools_as_numbers - /// is true make the result type Number - auto not_nullable_type = removeNullable(type); - auto not_nullable_new_type = removeNullable(new_type); - if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) - && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) - { - if (isBool(not_nullable_type)) - type = new_type; - } - else if (default_type) - type = default_type; - else - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName()); - } + chooseResultType(type, new_type, allow_bools_as_numbers, default_type, name, row); } } @@ -160,18 +158,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() for (auto & [name, type] : names_and_types) { /// Check that we could determine the type of this column. - if (!type) - { - if (!default_type) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " - "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", - max_rows_to_read); - - type = default_type; - } - result.emplace_back(name, type); + checkTypeAndAppend(result, type, name, default_type, max_rows_to_read); } return result; From d272356324a647e53fd44e01527c63402b6b80fe Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Mar 2022 10:55:09 +0000 Subject: [PATCH 149/372] Minor code improvement --- src/Processors/Formats/ISchemaReader.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 615ebfb03be..567a5b05096 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -30,8 +30,9 @@ static void chooseResultType( /// is true make the result type Number auto not_nullable_type = removeNullable(type); auto not_nullable_new_type = removeNullable(new_type); - if (allow_bools_as_numbers && (isBool(not_nullable_type) || isBool(not_nullable_new_type)) - && (isNumber(not_nullable_type) || isNumber(not_nullable_new_type))) + bool bool_type_presents = isBool(not_nullable_type) || isBool(not_nullable_new_type); + bool number_type_presents = isNumber(not_nullable_type) || isNumber(not_nullable_new_type); + if (allow_bools_as_numbers && bool_type_presents && number_type_presents) { if (isBool(not_nullable_type)) type = new_type; From a41c73fcf6d90e381691ab6f2bb60809196e6bf8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Mar 2022 11:25:52 +0000 Subject: [PATCH 150/372] Fix tests --- .../0_stateless/01801_s3_cluster.reference | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01801_s3_cluster.reference b/tests/queries/0_stateless/01801_s3_cluster.reference index 31c97f14fa3..0448ff3933b 100644 --- a/tests/queries/0_stateless/01801_s3_cluster.reference +++ b/tests/queries/0_stateless/01801_s3_cluster.reference @@ -2,30 +2,6 @@ 0 0 0 0 0 0 1 2 3 -10 11 12 -13 14 15 -16 17 18 -20 21 22 -23 24 25 -26 27 28 -4 5 6 -7 8 9 -0 0 0 -0 0 0 -0 0 0 -1 2 3 -10 11 12 -13 14 15 -16 17 18 -20 21 22 -23 24 25 -26 27 28 -4 5 6 -7 8 9 -0 0 0 -0 0 0 -0 0 0 -1 2 3 4 5 6 7 8 9 10 11 12 @@ -38,14 +14,26 @@ 0 0 0 0 0 0 1 2 3 +4 5 6 +7 8 9 10 11 12 13 14 15 16 17 18 20 21 22 23 24 25 26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 4 5 6 7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 0 0 0 0 0 0 0 0 0 @@ -62,14 +50,26 @@ 0 0 0 0 0 0 1 2 3 +4 5 6 +7 8 9 10 11 12 13 14 15 16 17 18 20 21 22 23 24 25 26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 4 5 6 7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 0 0 0 0 0 0 0 0 0 From d166bb51153f630b9581902f8120ee0247b9d792 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 31 Mar 2022 13:47:15 +0200 Subject: [PATCH 151/372] Update 02245_format_string_stack_overflow.sql --- tests/queries/0_stateless/02245_format_string_stack_overflow.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql index 1ee3606d3a6..9376b12aa1e 100644 --- a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql +++ b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql @@ -1 +1,2 @@ +-- Tags: no-backward-compatibility-check:22.3.2.2 select format('{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}', toString(number)) str from numbers(1); From 461ecd56a49437263154bea359023a95cdbe5f6a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 31 Mar 2022 13:47:38 +0200 Subject: [PATCH 152/372] fix race when database is Ordinary --- src/Interpreters/MergeTreeTransaction.cpp | 20 ++++++++++++++++++++ src/Interpreters/MergeTreeTransaction.h | 2 ++ 2 files changed, 22 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 4d2ea266e50..dd235d23bfe 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -13,6 +13,16 @@ namespace ErrorCodes 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_) @@ -106,14 +116,19 @@ void MergeTreeTransaction::addNewPartAndRemoveCovered(const StoragePtr & storage 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(); @@ -121,6 +136,8 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP 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); } @@ -129,9 +146,12 @@ void MergeTreeTransaction::removeOldPart(const StoragePtr & storage, const DataP 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); } diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index fde11654a34..301434ac712 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -66,6 +67,7 @@ private: /// Lists of changes made by transaction std::unordered_set storages; + std::vector table_read_locks_for_ordinary_db; DataPartsVector creating_parts; DataPartsVector removing_parts; using RunningMutationsList = std::vector>; From b5682c1f02eaaf15be2f85977417fc8f1b16c904 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 08:33:50 -0400 Subject: [PATCH 153/372] minor refactoring --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- src/Processors/Transforms/FillingTransform.cpp | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6a9fc2cef43..f788cc98c72 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -856,11 +856,11 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery auto syntax_result = TreeRewriter(context).analyze(exprs, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(exprs, syntax_result, context); ActionsDAGPtr actions = analyzer.getActionsDAG(true); - ActionsDAGPtr convDAG = ActionsDAG::makeConvertingActions(actions->getResultColumns(), + ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), columns, ActionsDAG::MatchColumnsMode::Position, true); - ActionsDAGPtr mergeDAG = ActionsDAG::merge(std::move(*actions->clone()), std::move(*convDAG)); + ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); - interpolate_descr = std::make_shared(mergeDAG, aliases); + interpolate_descr = std::make_shared(merge_dag, aliases); } return interpolate_descr; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index b8dcf1f0bf9..99fdd6b19f0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -259,7 +259,7 @@ void FillingTransform::transform(Chunk & chunk) { interpolate_block.clear(); - if (input_positions.size()) + if (!input_positions.empty()) { /// populate calculation block with required columns with values from previous row for (const auto & [col_pos, name_type] : input_positions) @@ -270,7 +270,7 @@ void FillingTransform::transform(Chunk & chunk) if (size == 0) /// this is the first row in current chunk { /// take value from last row of previous chunk if exists, else use default - if (last_row.size() > col_pos && last_row[col_pos]->size()) + if (last_row.size() > col_pos && !last_row[col_pos]->empty()) column->insertFrom(*last_row[col_pos], 0); else column->insertDefault(); @@ -284,7 +284,6 @@ void FillingTransform::transform(Chunk & chunk) } else /// all INTERPOLATE expressions are constants { - /// LOL :) size_t n = 1; interpolate_actions->execute(interpolate_block, n); } From 74ec5eb1cc97381edece677a0d364d10a7503d9b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Mar 2022 14:35:40 +0200 Subject: [PATCH 154/372] Fix checks --- tests/config/install.sh | 1 + ...2240_system_remote_filesystem_cache.reference | 16 ++++++++-------- .../02240_system_remote_filesystem_cache.sql | 8 ++++---- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index c499ffa88f7..2125f515734 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -49,6 +49,7 @@ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/s3_cache/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference index 20cb2329604..a26133180e4 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -6,14 +6,14 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; -./disks/s3/data_cache/ 0 0 1 -./disks/s3/data_cache/ 0 79 80 -./disks/s3/data_cache/ 0 745 746 +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; +0 0 1 +0 79 80 +0 745 746 SYSTEM DROP FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; -./disks/s3/data_cache/ 0 745 746 +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +0 745 746 SYSTEM DROP FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index a889bea0fcf..cc5fd259ce8 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -9,10 +9,10 @@ CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SET INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; SYSTEM DROP FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SYSTEM DROP FILESYSTEM CACHE; -SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; From 252d66e80d99fcc1e6c1b0e6e9df191d99469e9e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 31 Mar 2022 16:08:37 +0200 Subject: [PATCH 155/372] Update src/Processors/Formats/ISchemaReader.cpp Co-authored-by: Antonio Andelic --- src/Processors/Formats/ISchemaReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 567a5b05096..1a8a9a0ab12 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -24,7 +24,7 @@ static void chooseResultType( /// If the new type and the previous type for this column are different, /// we will use default type if we have it or throw an exception. - if (new_type && type->equals(*new_type)) + if (new_type && !type->equals(*new_type)) { /// Check if we have Bool and Number and if allow_bools_as_numbers /// is true make the result type Number From a15996315e5b01b7a09128310ff1e76182940fab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 11:51:13 -0400 Subject: [PATCH 156/372] bugfix - columns order tracking --- src/Core/InterpolateDescription.cpp | 10 ++++++---- src/Core/InterpolateDescription.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 4 ++++ src/Processors/Transforms/FillingTransform.cpp | 12 ++++++------ 4 files changed, 17 insertions(+), 10 deletions(-) diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index 4147d254033..e7b74716b79 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -20,10 +20,12 @@ namespace DB for (const ColumnWithTypeAndName & column : actions->getResultColumns()) { - if (const auto & p = aliases.find(column.name); p != aliases.end()) - result_columns_set.insert(p->second->getColumnName()); - else - result_columns_set.insert(column.name); + std::string name = column.name; + if (const auto & p = aliases.find(name); p != aliases.end()) + name = p->second->getColumnName(); + + result_columns_set.insert(name); + result_columns_order.push_back(name); } } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index b19674eb590..8aabce1470e 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -25,6 +25,7 @@ struct InterpolateDescription std::unordered_map required_columns_map; /// input column name -> {alias, type} std::unordered_set result_columns_set; /// result block columns + std::vector result_columns_order; /// result block columns order }; using InterpolateDescriptionPtr = std::shared_ptr; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f788cc98c72..6bcedc5b4f7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -839,6 +839,7 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { + std::unordered_set col_set; ColumnsWithTypeAndName columns; ASTPtr exprs = std::make_shared(); for (const auto & elem : query.interpolate()->children) @@ -848,6 +849,9 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery if (!block_column) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + if (!col_set.insert(block_column->name).second) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Duplicate INTERPOLATE column '{}'", interpolate.column); columns.emplace_back(block_column->type, block_column->name); exprs->children.emplace_back(interpolate.expr->clone()); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 99fdd6b19f0..c13bdfe2d84 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -193,15 +193,15 @@ FillingTransform::FillingTransform( p != interpolate_description->required_columns_map.end()) input_positions.emplace_back(idx, p->second); - if (!is_fill_column[idx]) - { - if (interpolate_description && interpolate_description->result_columns_set.count(column.name)) - interpolate_column_positions.push_back(idx); - else + if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.count(column.name))) other_column_positions.push_back(idx); - } + ++idx; } + + if(interpolate_description) + for (const auto & name : interpolate_description->result_columns_order) + interpolate_column_positions.push_back(header_.getPositionByName(name)); } IProcessor::Status FillingTransform::prepare() From 538373a79b51cd8ec7f2ad9701a067cc9f6eae75 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 31 Mar 2022 12:13:49 -0400 Subject: [PATCH 157/372] style fix --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c13bdfe2d84..c7bcd03ec08 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -199,7 +199,7 @@ FillingTransform::FillingTransform( ++idx; } - if(interpolate_description) + if (interpolate_description) for (const auto & name : interpolate_description->result_columns_order) interpolate_column_positions.push_back(header_.getPositionByName(name)); } From 603dcbb1b74fa2695abd9ece6c8ed49470527aa7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Mar 2022 15:27:48 +0200 Subject: [PATCH 158/372] Extend test --- src/IO/WriteBufferFromS3.cpp | 1 + tests/config/install.sh | 2 +- ...41_remote_filesystem_cache_on_insert.reference | 15 ++++++++++++++- .../02241_remote_filesystem_cache_on_insert.sql | 14 +++++++++++++- 4 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 0eee7366775..b5e61724ede 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -100,6 +100,7 @@ void WriteBufferFromS3::nextImpl() auto cache_key = cache->hash(blob_name); auto file_segments_holder = cache->setDownloading(cache_key, current_download_offset, size); + current_download_offset += size; size_t remaining_size = size; for (const auto & file_segment : file_segments_holder.file_segments) diff --git a/tests/config/install.sh b/tests/config/install.sh index 2125f515734..f1b4fe1a588 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -49,7 +49,6 @@ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ -ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/s3_cache/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), @@ -86,6 +85,7 @@ fi if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/ + ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/s3_cache/ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index e83b5551821..5bc18d48655 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -9,7 +9,7 @@ SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, re 0 SELECT count() FROM system.filesystem_cache; 0 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── @@ -60,3 +60,16 @@ SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; SELECT count() FROM system.filesystem_cache; 28 +INSERT INTO test SELECT number, toString(number) FROM numbers(5000000); +SYSTEM FLUSH LOGS; +SELECT query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read +FROM system.query_log +WHERE query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; +SELECT count() FROM test; +5010500 +SELECT count() FROM test WHERE value LIKE '%010%'; +18816 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 745af904c5f..946e72ba2fd 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -13,7 +13,7 @@ SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELEC SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; SELECT count() FROM system.filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; @@ -50,3 +50,15 @@ SELECT count() FROM system.filesystem_cache; SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; SELECT count() FROM system.filesystem_cache; + +INSERT INTO test SELECT number, toString(number) FROM numbers(5000000); +SYSTEM FLUSH LOGS; +SELECT query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read +FROM system.query_log +WHERE query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; +SELECT count() FROM test; +SELECT count() FROM test WHERE value LIKE '%010%'; From 44a8622851868cc2a1edd3c1be3293f9d942b852 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 31 Mar 2022 18:31:58 +0200 Subject: [PATCH 159/372] fix test --- tests/queries/0_stateless/01168_mutations_isolation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 47ab89e5646..888858edf32 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -58,7 +58,7 @@ tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" tx 8 "begin transaction" tx_async 8 "alter table mt update n = 0 where 1" >/dev/null -$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" +$CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" 2>&1| grep -Fv "probably it finished" tx_sync 8 "rollback" tx 7 "optimize table mt final" tx 7 "select 8, n, _part from mt order by n" From ececee3817edefb84794e00a2cea93f7ef07d29b Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 1 Apr 2022 20:13:34 +0800 Subject: [PATCH 160/372] to #34966_fix_dateTime_deserialize --- src/IO/ReadHelpers.h | 18 +++++++++++++++--- .../02249_parse_date_time_basic.reference | 3 +++ .../02249_parse_date_time_basic.sql | 6 ++++++ 3 files changed, 24 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02249_parse_date_time_basic.reference create mode 100644 tests/queries/0_stateless/02249_parse_date_time_basic.sql diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9396e1d32f7..8296b8db4d7 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -851,6 +851,8 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons /// YYYY-MM-DD hh:mm:ss static constexpr auto DateTimeStringInputSize = 19; + ///YYYY-MM-DD + static constexpr auto DateStringInputSize = 10; bool optimistic_path_for_date_time_input = s + DateTimeStringInputSize <= buf.buffer().end(); if (optimistic_path_for_date_time_input) @@ -861,16 +863,26 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); - UInt8 hour = (s[11] - '0') * 10 + (s[12] - '0'); - UInt8 minute = (s[14] - '0') * 10 + (s[15] - '0'); - UInt8 second = (s[17] - '0') * 10 + (s[18] - '0'); + UInt8 hour = 0; + UInt8 minute = 0; + UInt8 second = 0; + ///simply determine whether it is YYYY-MM-DD hh:mm:ss or YYYY-MM-DD by the content of the tenth character in an optimistic scenario + if (s[10] == ' ') + { + hour = (s[11] - '0') * 10 + (s[12] - '0'); + minute = (s[14] - '0') * 10 + (s[15] - '0'); + second = (s[17] - '0') * 10 + (s[18] - '0'); + } if (unlikely(year == 0)) datetime = 0; else datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); + if (s[10] == ' ') buf.position() += DateTimeStringInputSize; + else + buf.position() += DateStringInputSize; return ReturnType(true); } else diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.reference b/tests/queries/0_stateless/02249_parse_date_time_basic.reference new file mode 100644 index 00000000000..d67e0ae15e0 --- /dev/null +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.reference @@ -0,0 +1,3 @@ +2022-03-31 00:00:00 1 +2022-04-01 17:10:24 2 +2022-03-31 10:18:56 3 diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.sql b/tests/queries/0_stateless/02249_parse_date_time_basic.sql new file mode 100644 index 00000000000..dd2306d99aa --- /dev/null +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.sql @@ -0,0 +1,6 @@ +drop table if exists t; +CREATE TABLE t (a DateTime, b String, c String, d String, e Int32) ENGINE = Memory; +INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31','','','',1); +INSERT INTO t(a, b, c, d ,e) VALUES (1648804224,'','','',2); +INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31 10:18:56','','','',3); +select a, e from t; \ No newline at end of file From 3cae0c74d9fd18e76a2d02b1178aa7f533600900 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 1 Apr 2022 20:16:07 +0800 Subject: [PATCH 161/372] to #34966_fix_dateTime_deserialize --- src/IO/ReadHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 8296b8db4d7..48c291d8fcc 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -880,7 +880,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); if (s[10] == ' ') - buf.position() += DateTimeStringInputSize; + buf.position() += DateTimeStringInputSize; else buf.position() += DateStringInputSize; return ReturnType(true); From 79c75d8a712a3a49807164a0c618d04cfadbbb9c Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 1 Apr 2022 20:18:28 +0800 Subject: [PATCH 162/372] to #34966_fix_dateTime_deserialize --- tests/queries/0_stateless/02249_parse_date_time_basic.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.sql b/tests/queries/0_stateless/02249_parse_date_time_basic.sql index dd2306d99aa..2cea41874d5 100644 --- a/tests/queries/0_stateless/02249_parse_date_time_basic.sql +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.sql @@ -3,4 +3,4 @@ CREATE TABLE t (a DateTime, b String, c String, d String, e Int32) ENGINE = Memo INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31','','','',1); INSERT INTO t(a, b, c, d ,e) VALUES (1648804224,'','','',2); INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31 10:18:56','','','',3); -select a, e from t; \ No newline at end of file +select a, e from t; From 0116233d36cf78ad505401495a3433399042afbc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 1 Apr 2022 16:18:19 -0400 Subject: [PATCH 163/372] allow INTERPOLATE to reference optimized out columns --- src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++++++++-- src/Interpreters/InterpreterSelectQuery.h | 2 ++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6bcedc5b4f7..cb15b4fd404 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,6 +286,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); + for(const auto & column : query_ptr->as().select()->children) + original_select_set.insert(column->getColumnName()); + query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; @@ -834,7 +837,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP return order_descr; } -static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery & query, Block block, const Aliases & aliases, ContextPtr context) +static InterpolateDescriptionPtr getInterpolateDescription( + const ASTSelectQuery & query, Block block, const Aliases & aliases, const NameSet & original_select_set, ContextPtr context) { InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) @@ -847,8 +851,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(const ASTSelectQuery const auto & interpolate = elem->as(); ColumnWithTypeAndName *block_column = block.findByName(interpolate.column); if (!block_column) + { + if (original_select_set.count(interpolate.column)) /// column was removed + continue; throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); + } if (!col_set.insert(block_column->name).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Duplicate INTERPOLATE column '{}'", interpolate.column); @@ -2533,7 +2541,8 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) if (fill_descr.empty()) return; - InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, context); + InterpolateDescriptionPtr interpolate_descr = + getInterpolateDescription(query, result_header, syntax_analyzer_result->aliases, original_select_set, context); auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 6bb12caff7d..ec3775ab848 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -185,6 +185,8 @@ private: Names required_columns; /// Structure of query source (table, subquery, etc). Block source_header; + /// Original set of selected columns + NameSet original_select_set; /// Actions to calculate ALIAS if required. ActionsDAGPtr alias_actions; From ec0ad8804a45efa91505daf56912f5cf5a21e9fd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 1 Apr 2022 21:45:58 -0400 Subject: [PATCH 164/372] style fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb15b4fd404..cddaa51b8b7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,7 +286,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for(const auto & column : query_ptr->as().select()->children) + for (const auto & column : query_ptr->as().select()->children) original_select_set.insert(column->getColumnName()); query_info.ignore_projections = options.ignore_projections; From b815c38d385adca00d8860a4173fd8fdd3a25271 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 2 Apr 2022 11:45:48 +0000 Subject: [PATCH 165/372] Update test reference file --- .../02149_schema_inference.reference | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index f46e3bee101..3e1d272bbf7 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -38,32 +38,32 @@ JSONCompactEachRow c1 Nullable(Float64) c2 Array(Tuple(Nullable(Float64), Nullable(String))) c3 Map(String, Nullable(Float64)) -c4 Nullable(UInt8) -42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +c4 Nullable(Bool) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true c1 Nullable(Float64) c2 Array(Tuple(Nullable(Float64), Nullable(String))) c3 Map(String, Nullable(Float64)) -c4 Nullable(UInt8) +c4 Nullable(Bool) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N -32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1 +32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} true JSONCompactEachRowWithNames a Nullable(Float64) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) -d Nullable(UInt8) -42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +d Nullable(Bool) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} true JSONEachRow -d Nullable(UInt8) +d Nullable(Bool) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) a Nullable(Float64) -1 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 42.42 -d Nullable(UInt8) +true [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 42.42 +d Nullable(Bool) b Array(Tuple(Nullable(Float64), Nullable(String))) c Map(String, Nullable(Float64)) a Nullable(Float64) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N -1 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 32 +true [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 32 b Nullable(String) c Array(Nullable(Float64)) a Nullable(Float64) From 860b1a1b1bd8a1dec6d1176396400937d0346977 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 2 Apr 2022 14:04:04 +0200 Subject: [PATCH 166/372] Update 02248_nullable_custom_types_to_string.sql --- .../0_stateless/02248_nullable_custom_types_to_string.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql index 313f703fd03..b6032f7741b 100644 --- a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql +++ b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql @@ -1,3 +1,4 @@ +-- Tags: no-backward-compatibility-check:22.3.2.2 select toString(toNullable(true)); select toString(CAST(NULL, 'Nullable(Bool)')); select toString(toNullable(toIPv4('0.0.0.0'))); From 3ce4844d4b30f63ed5e7e7c816b1b5fd69f78145 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 Apr 2022 18:12:51 +0300 Subject: [PATCH 167/372] Properly cancel the query after client format error In case of format error (i.e. select 2 format Template settings format_template_row='/dev/null') the client will reset the connection, since it will not expect Data blocks. To fix this, catch this client error and properly cancel the query. This should also fix query hang checks (the one that executed after each test). v2: use getCurrentExceptionMessage()/getCurrentExceptionCode() Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9e61356566c..4faf3a8028e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -240,6 +240,14 @@ void interruptSignalHandler(int signum) } +/// To cancel the query on local format error. +class LocalFormatError : public DB::Exception +{ +public: + using Exception::Exception; +}; + + ClientBase::~ClientBase() = default; ClientBase::ClientBase() = default; @@ -442,6 +450,7 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) +try { if (!output_format) { @@ -530,6 +539,10 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) output_format->setAutoFlush(); } } +catch (...) +{ + throw LocalFormatError(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); +} void ClientBase::initLogsOutputStream() @@ -721,6 +734,9 @@ void ClientBase::receiveResult(ASTPtr parsed_query) = std::max(min_poll_interval, std::min(receive_timeout.totalMicroseconds(), default_poll_interval)); bool break_on_timeout = connection->getConnectionType() != IServerConnection::Type::LOCAL; + + std::exception_ptr local_format_error; + while (true) { Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE); @@ -769,10 +785,21 @@ void ClientBase::receiveResult(ASTPtr parsed_query) break; } - if (!receiveAndProcessPacket(parsed_query, cancelled)) - break; + try + { + if (!receiveAndProcessPacket(parsed_query, cancelled)) + break; + } + catch (const LocalFormatError &) + { + local_format_error = std::current_exception(); + connection->sendCancel(); + } } + if (local_format_error) + std::rethrow_exception(local_format_error); + if (cancelled && is_interactive) std::cout << "Query was cancelled." << std::endl; } From 93fc604223d205d1fc6e9ad5d6f65e9310bf260a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 3 Apr 2022 18:39:14 +0200 Subject: [PATCH 168/372] Fix --- .../test_storage_postgresql_replica/test.py | 76 +++++++++---------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index e51a9335a65..fe8b92ea606 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -24,25 +24,25 @@ postgres_table_template = """ """ queries = [ - "INSERT INTO postgresql_replica select i, i from generate_series(0, 10000) as t(i);", - "DELETE FROM postgresql_replica WHERE (value*value) % 3 = 0;", - "UPDATE postgresql_replica SET value = value + 125 WHERE key % 2 = 0;", - "UPDATE postgresql_replica SET key=key+20000 WHERE key%2=0", - "INSERT INTO postgresql_replica select i, i from generate_series(40000, 50000) as t(i);", - "DELETE FROM postgresql_replica WHERE key % 10 = 0;", - "UPDATE postgresql_replica SET value = value + 101 WHERE key % 2 = 1;", - "UPDATE postgresql_replica SET key=key+80000 WHERE key%2=1", - "DELETE FROM postgresql_replica WHERE value % 2 = 0;", - "UPDATE postgresql_replica SET value = value + 2000 WHERE key % 5 = 0;", - "INSERT INTO postgresql_replica select i, i from generate_series(200000, 250000) as t(i);", - "DELETE FROM postgresql_replica WHERE value % 3 = 0;", - "UPDATE postgresql_replica SET value = value * 2 WHERE key % 3 = 0;", - "UPDATE postgresql_replica SET key=key+500000 WHERE key%2=1", - "INSERT INTO postgresql_replica select i, i from generate_series(1000000, 1050000) as t(i);", - "DELETE FROM postgresql_replica WHERE value % 9 = 2;", - "UPDATE postgresql_replica SET key=key+10000000", - "UPDATE postgresql_replica SET value = value + 2 WHERE key % 3 = 1;", - "DELETE FROM postgresql_replica WHERE value%5 = 0;", + "INSERT INTO {} select i, i from generate_series(0, 10000) as t(i);", + "DELETE FROM {} WHERE (value*value) % 3 = 0;", + "UPDATE {} SET value = value + 125 WHERE key % 2 = 0;", + "UPDATE {} SET key=key+20000 WHERE key%2=0", + "INSERT INTO {} select i, i from generate_series(40000, 50000) as t(i);", + "DELETE FROM {} WHERE key % 10 = 0;", + "UPDATE {} SET value = value + 101 WHERE key % 2 = 1;", + "UPDATE {} SET key=key+80000 WHERE key%2=1", + "DELETE FROM {} WHERE value % 2 = 0;", + "UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;", + "INSERT INTO {} select i, i from generate_series(200000, 250000) as t(i);", + "DELETE FROM {} WHERE value % 3 = 0;", + "UPDATE {} SET value = value * 2 WHERE key % 3 = 0;", + "UPDATE {} SET key=key+500000 WHERE key%2=1", + "INSERT INTO {} select i, i from generate_series(1000000, 1050000) as t(i);", + "DELETE FROM {} WHERE value % 9 = 2;", + "UPDATE {} SET key=key+10000000", + "UPDATE {} SET value = value + 2 WHERE key % 3 = 1;", + "DELETE FROM {} WHERE value%5 = 0;", ] @@ -103,15 +103,13 @@ def create_clickhouse_postgres_db(ip, port, name="postgres_database"): ) -def create_materialized_table(ip, port): +def create_materialized_table(ip, port, table_name='postgresql_replica'): instance.query( - """ - CREATE TABLE test.postgresql_replica (key UInt64, value UInt64) + f""" + CREATE TABLE test.{table_name} (key UInt64, value UInt64) ENGINE = MaterializedPostgreSQL( - '{}:{}', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword') - PRIMARY KEY key; """.format( - ip, port - ) + '{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword') + PRIMARY KEY key; """ ) @@ -669,17 +667,18 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): database=True, ) cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") + table_name = "postgresql_replica" + create_postgres_table(cursor, table_name) - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") + instance.query(f"DROP TABLE IF EXISTS test.{table_name}") create_materialized_table( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) for i in range(len(queries)): - query = queries[i] + query = queries[i].format(table_name) cursor.execute(query) - print("query {}".format(query)) + print("query {}".format(query.format(table_name))) started_cluster.pause_container("postgres1") @@ -701,25 +700,26 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): database=True, ) cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") + table_name = "postgresql_replica_697" + create_postgres_table(cursor, table_name) - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") + instance.query(f"DROP TABLE IF EXISTS test.{table_name}") create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port + ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, table_name=table_name ) for query in queries: - cursor.execute(query) - print("query {}".format(query)) + cursor.execute(query.format(table_name)) + print("query {}".format(query.format(table_name))) instance.restart_clickhouse() - result = instance.query("SELECT count() FROM test.postgresql_replica") + result = instance.query(f"SELECT count() FROM test.{table_name}") print(result) # Just debug - check_tables_are_synchronized("postgresql_replica") + check_tables_are_synchronized(table_name) - result = instance.query("SELECT count() FROM test.postgresql_replica") + result = instance.query(f"SELECT count() FROM test.{table_name}") print(result) # Just debug From 95ad1bf6e1a3cbb904a0b3ff9cc9447893a9be2a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 3 Apr 2022 22:10:36 -0400 Subject: [PATCH 169/372] use aliases if exist for original_select_set --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cddaa51b8b7..8d2a6d21994 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,8 +286,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for (const auto & column : query_ptr->as().select()->children) - original_select_set.insert(column->getColumnName()); + for (const auto & column : query_ptr->as().select()->children) { + std::string alias = column->tryGetAlias(); + original_select_set.insert(alias.empty() ? column->getColumnName() : alias); + } query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; From ff4d29526541901dc80a7a08602c9b89b032feab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 3 Apr 2022 22:19:35 -0400 Subject: [PATCH 170/372] style fix --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d2a6d21994..d050b378916 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,7 +286,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( { checkStackSize(); - for (const auto & column : query_ptr->as().select()->children) { + for (const auto & column : query_ptr->as().select()->children) + { std::string alias = column->tryGetAlias(); original_select_set.insert(alias.empty() ? column->getColumnName() : alias); } From cbfc0f6bacd33fe5d99ed13ffb21671c79898fe3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 4 Apr 2022 10:42:22 +0800 Subject: [PATCH 171/372] fix typo --- src/Storages/Hive/HiveCommon.cpp | 1 + src/Storages/Hive/StorageHive.cpp | 7 ++++--- src/Storages/Hive/StorageHive.h | 1 - 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index a9d0c22d6a5..60f471c1568 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -172,6 +172,7 @@ std::vector HiveMetastoreClient::HiveTableMetad std::vector result; std::lock_guard lock{mutex}; + result.reserve(partition_infos.size()); for (const auto & partition_info : partition_infos) result.emplace_back(partition_info.second.partition); return result; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 61bbc3ce4f6..c5e50e6b480 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -60,7 +60,7 @@ public: struct SourcesInfo { HiveMetastoreClientPtr hive_metastore_client; - std::string database; + std::string database_name; std::string table_name; HiveFiles hive_files; NamesAndTypesList partition_name_types; @@ -159,7 +159,7 @@ public: { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE) { - source_info->hive_metastore_client->clearTableMetadata(source_info->database, source_info->table_name); + source_info->hive_metastore_client->clearTableMetadata(source_info->database_name, source_info->table_name); throw; } } @@ -562,6 +562,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( } return hive_file; } + bool StorageHive::isColumnOriented() const { return format_name == "Parquet" || format_name == "ORC"; @@ -651,7 +652,7 @@ Pipe StorageHive::read( auto sources_info = std::make_shared(); sources_info->hive_files = std::move(hive_files); - sources_info->database = hive_database; + sources_info->database_name = hive_database; sources_info->table_name = hive_table; sources_info->hive_metastore_client = hive_metastore_client; sources_info->partition_name_types = partition_name_types; diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 376aab311d0..bdf37cc9f04 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -39,7 +39,6 @@ public: return true; } - Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, From 38f149b533607d13d406c4efac4d8398ad4a1eb3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 4 Apr 2022 15:28:26 +0800 Subject: [PATCH 172/372] optimize trivial count hive query --- src/Storages/Hive/HiveFile.cpp | 44 ++++- src/Storages/Hive/HiveFile.h | 23 ++- src/Storages/Hive/StorageHive.cpp | 260 +++++++++++++++++++----------- src/Storages/Hive/StorageHive.h | 51 ++++-- 4 files changed, 266 insertions(+), 112 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 407d9602b61..12d60c4d1b3 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -77,7 +77,14 @@ Range createRangeFromParquetStatistics(std::shared_ptr IHiveFile::getRows() +{ + if (!rows) + rows = getRowsImpl(); + return rows; +} + +Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { if (!col_stats || col_stats->hasNull()) return {}; @@ -122,7 +129,7 @@ Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) return {}; } -void HiveOrcFile::prepareReader() +void HiveORCFile::prepareReader() { in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); @@ -132,7 +139,7 @@ void HiveOrcFile::prepareReader() reader = std::move(result).ValueOrDie(); } -void HiveOrcFile::prepareColumnMapping() +void HiveORCFile::prepareColumnMapping() { const orc::Type & type = reader->GetRawORCReader()->getType(); size_t count = type.getSubtypeCount(); @@ -145,13 +152,13 @@ void HiveOrcFile::prepareColumnMapping() } } -bool HiveOrcFile::hasMinMaxIndex() const +bool HiveORCFile::hasMinMaxIndex() const { return storage_settings->enable_orc_file_minmax_index; } -std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * statistics) +std::unique_ptr HiveORCFile::buildMinMaxIndex(const orc::Statistics * statistics) { if (!statistics) return nullptr; @@ -184,7 +191,7 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c } -void HiveOrcFile::loadMinMaxIndex() +void HiveORCFile::loadMinMaxIndex() { if (!reader) { @@ -196,13 +203,13 @@ void HiveOrcFile::loadMinMaxIndex() minmax_idx = buildMinMaxIndex(statistics.get()); } -bool HiveOrcFile::hasSubMinMaxIndex() const +bool HiveORCFile::hasSubMinMaxIndex() const { return storage_settings->enable_orc_stripe_minmax_index; } -void HiveOrcFile::loadSubMinMaxIndex() +void HiveORCFile::loadSubMinMaxIndex() { if (!reader) { @@ -226,6 +233,18 @@ void HiveOrcFile::loadSubMinMaxIndex() } } +std::optional HiveORCFile::getRowsImpl() +{ + if (!reader) + { + prepareReader(); + prepareColumnMapping(); + } + + auto * raw_reader = reader->GetRawORCReader(); + return raw_reader->getNumberOfRows(); +} + bool HiveParquetFile::hasSubMinMaxIndex() const { return storage_settings->enable_parquet_rowgroup_minmax_index; @@ -312,5 +331,14 @@ void HiveParquetFile::loadSubMinMaxIndex() } } +std::optional HiveParquetFile::getRowsImpl() +{ + if (!reader) + prepareReader(); + + auto meta = reader->parquet_reader()->metadata(); + return meta->num_rows(); +} + } #endif diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index aef9d72755a..21dddc1b68f 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -102,6 +102,12 @@ public: virtual String getPath() const { return path; } + virtual UInt64 getLastModTs() const { return last_modify_time; } + + virtual size_t getSize() const { return size; } + + std::optional getRows(); + virtual FieldVector getPartitionValues() const { return partition_values; } virtual String getNamenodeUrl() { return namenode_url; } @@ -144,15 +150,16 @@ public: return boost::algorithm::join(strs, "|"); } - inline UInt64 getLastModTs() const { return last_modify_time; } - inline size_t getSize() const { return size; } - protected: + virtual std::optional getRowsImpl() = 0; + FieldVector partition_values; String namenode_url; String path; UInt64 last_modify_time; size_t size; + std::optional rows; + NamesAndTypesList index_names_and_types; MinMaxIndexPtr minmax_idx; std::vector sub_minmax_idxes; @@ -182,12 +189,15 @@ public: virtual FileFormat getFormat() const override { return FileFormat::TEXT; } virtual String getName() const override { return "TEXT"; } + +protected: + std::optional getRowsImpl() override { return {}; } }; -class HiveOrcFile : public IHiveFile +class HiveORCFile : public IHiveFile { public: - HiveOrcFile( + HiveORCFile( const FieldVector & values_, const String & namenode_url_, const String & path_, @@ -214,6 +224,8 @@ protected: virtual void prepareReader(); virtual void prepareColumnMapping(); + std::optional getRowsImpl() override; + std::unique_ptr in; std::unique_ptr reader; std::map orc_column_positions; @@ -243,6 +255,7 @@ public: protected: virtual void prepareReader(); + std::optional getRowsImpl() override; std::unique_ptr in; std::unique_ptr reader; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 97d735d4fc1..4d2a01f2c94 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int INVALID_PARTITION_VALUE; extern const int BAD_ARGUMENTS; extern const int CANNOT_OPEN_FILE; + extern const int LOGICAL_ERROR; } @@ -419,7 +420,7 @@ HiveFilePtr createHiveFile( } else if (format_name == "ORC") { - hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); } else if (format_name == "Parquet") { @@ -432,24 +433,26 @@ HiveFilePtr createHiveFile( return hive_file; } -std::vector StorageHive::collectHiveFilesFromPartition( +HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, - SelectQueryInfo & query_info, + const SelectQueryInfo & query_info, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_) + ContextPtr context_, + PruneLevel prune_level) const { - LOG_DEBUG(log, "Collect hive files from partition {}", boost::join(partition.values, ",")); + LOG_DEBUG( + log, "Collect hive files from partition {}, prune_level:{}", boost::join(partition.values, ","), pruneLevelToString(prune_level)); - /// Skip partition "__HIVE_DEFAULT_PARTITION__" - bool has_default_partition = false; - for (const auto & value : partition.values) - { - if (value == "__HIVE_DEFAULT_PARTITION__") - { - has_default_partition = true; - break; - } + /// Skip partition "__HIVE_DEFAULT_PARTITION__" + bool has_default_partition = false; + for (const auto & value : partition.values) + { + if (value == "__HIVE_DEFAULT_PARTITION__") + { + has_default_partition = true; + break; + } } if (has_default_partition) return {}; @@ -479,25 +482,29 @@ std::vector StorageHive::collectHiveFilesFromPartition( if (!reader->pull(block) || !block.rows()) throw Exception("Could not parse partition value: " + wb.str(), ErrorCodes::INVALID_PARTITION_VALUE); - std::vector ranges; - ranges.reserve(partition_names.size()); + /// Get partition values FieldVector fields(partition_names.size()); for (size_t i = 0; i < partition_names.size(); ++i) - { block.getByPosition(i).column->get(0, fields[i]); - ranges.emplace_back(fields[i]); + + if (prune_level >= PruneLevel::Partition) + { + std::vector ranges; + ranges.reserve(partition_names.size()); + for (size_t i = 0; i < partition_names.size(); ++i) + ranges.emplace_back(fields[i]); + + const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) + return {}; } - const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); - if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) - return {}; - + HiveFiles hive_files; auto file_infos = listDirectory(partition.sd.location, hive_table_metadata, fs); - std::vector hive_files; hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_); + auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); if (hive_file) hive_files.push_back(hive_file); } @@ -511,12 +518,17 @@ StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_ } HiveFilePtr StorageHive::createHiveFileIfNeeded( - const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_) + const FileInfo & file_info, + const FieldVector & fields, + const SelectQueryInfo & query_info, + ContextPtr context_, + PruneLevel prune_level) const { - LOG_TRACE(log, "Append hive file {}", file_info.path); + LOG_TRACE(log, "create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' - if (filename.find('.') == 0) + if (startsWith(filename, ".") == 0) return {}; auto hive_file = createHiveFile( @@ -531,34 +543,44 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( context_); /// Load file level minmax index and apply - const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); - if (hive_file->hasMinMaxIndex()) + if (prune_level >= PruneLevel::File) { - hive_file->loadMinMaxIndex(); - if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) - .can_be_true) + const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + if (hive_file->hasMinMaxIndex()) { - LOG_TRACE(log, "Skip hive file {} by index {}", hive_file->getPath(), hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); - return {}; - } - } - - /// Load sub-file level minmax index and apply - if (hive_file->hasSubMinMaxIndex()) - { - std::set skip_splits; - hive_file->loadSubMinMaxIndex(); - const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); - for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) - { - if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + hive_file->loadMinMaxIndex(); + if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); - skip_splits.insert(i); + LOG_TRACE( + log, + "Skip hive file {} by index {}", + hive_file->getPath(), + hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); + return {}; + } + } + + if (prune_level >= PruneLevel::Split) + { + /// Load sub-file level minmax index and apply + if (hive_file->hasSubMinMaxIndex()) + { + std::set skip_splits; + hive_file->loadSubMinMaxIndex(); + const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); + for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) + { + if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); + skip_splits.insert(i); + } + } + hive_file->setSkipSplits(skip_splits); } } - hive_file->setSkipSplits(skip_splits); } return hive_file; } @@ -591,6 +613,7 @@ void StorageHive::getActualColumnsToRead(Block & sample_block, const Block & hea } } } + Pipe StorageHive::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -607,47 +630,9 @@ Pipe StorageHive::read( auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - std::vector partitions = hive_table_metadata->getPartitions(); - /// Hive files to read - HiveFiles hive_files; - /// Mutext to protect hive_files, which maybe appended in multiple threads - std::mutex hive_files_mutex; - - ThreadPool pool{num_streams}; - if (!partitions.empty()) - { - for (const auto & partition : partitions) - { - pool.scheduleOrThrowOnError([&]() - { - auto hive_files_in_partition = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_); - if (!hive_files_in_partition.empty()) - { - std::lock_guard lock(hive_files_mutex); - hive_files.insert(std::end(hive_files), std::begin(hive_files_in_partition), std::end(hive_files_in_partition)); - } - }); - } - pool.wait(); - } - else if (partition_name_types.empty()) /// Partition keys is empty - { - auto file_infos = listDirectory(hive_table_metadata->getTable()->sd.location, hive_table_metadata, fs); - for (const auto & file_info : file_infos) - { - pool.scheduleOrThrowOnError([&] - { - auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_); - if (hive_file) - { - std::lock_guard lock(hive_files_mutex); - hive_files.push_back(hive_file); - } - }); - } - pool.wait(); - } - else /// Partition keys is not empty but partitions is empty + /// Collect Hive files to read + HiveFiles hive_files = collectHiveFiles(num_streams, query_info, hive_table_metadata, fs, context_); + if (hive_files.empty()) return {}; auto sources_info = std::make_shared(); @@ -689,6 +674,63 @@ Pipe StorageHive::read( return Pipe::unitePipes(std::move(pipes)); } +HiveFiles StorageHive::collectHiveFiles( + unsigned max_threads, + const SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_, + PruneLevel prune_level) const +{ + + std::vector partitions = hive_table_metadata->getPartitions(); + /// Hive table have no partition + if (!partition_name_types.empty() && partitions.empty()) + return {}; + + /// Hive files to collect + HiveFiles hive_files; + /// Mutext to protect hive_files, which maybe appended in multiple threads + std::mutex hive_files_mutex; + ThreadPool pool{max_threads}; + if (!partitions.empty()) + { + for (const auto & partition : partitions) + { + pool.scheduleOrThrowOnError( + [&]() + { + auto hive_files_in_partition + = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level); + if (!hive_files_in_partition.empty()) + { + std::lock_guard lock(hive_files_mutex); + hive_files.insert(std::end(hive_files), std::begin(hive_files_in_partition), std::end(hive_files_in_partition)); + } + }); + } + } + else /// Partition keys is empty but still have files + { + auto file_infos = listDirectory(hive_table_metadata->getTable()->sd.location, hive_table_metadata, fs); + for (const auto & file_info : file_infos) + { + pool.scheduleOrThrowOnError( + [&]() + { + auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_, prune_level); + if (hive_file) + { + std::lock_guard lock(hive_files_mutex); + hive_files.push_back(hive_file); + } + }); + } + } + pool.wait(); + return hive_files; +} + SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/) { throw Exception("Method write is not implemented for StorageHive", ErrorCodes::NOT_IMPLEMENTED); @@ -701,6 +743,44 @@ NamesAndTypesList StorageHive::getVirtuals() const {"_file", std::make_shared(std::make_shared())}}; } +std::optional StorageHive::totalRows(const Settings & settings) const +{ + /// query_info is not used when prune_level == PruneLevel::None + SelectQueryInfo query_info; + return totalRowsImpl(settings, query_info, getContext(), PruneLevel::None); +} + +std::optional StorageHive::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const +{ + return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition); +} + +std::optional +StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const +{ + /// Row-based format like Text doesn't support totalRowsByPartitionPredicate + if (!isColumnOriented()) + return {}; + + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); + HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, getContext()->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + HiveFiles hive_files = collectHiveFiles(settings.max_threads, query_info, hive_table_metadata, fs, context_, prune_level); + + UInt64 total_rows = 0; + for (const auto & hive_file : hive_files) + { + auto file_rows = hive_file->getRows(); + if (!file_rows) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Rows of hive file:{} with format:{} not initialized", hive_file->getPath(), format_name); + total_rows += *file_rows; + } + return total_rows; +} + + void registerStorageHive(StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index bdf37cc9f04..1470b883585 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -26,7 +26,6 @@ class HiveSettings; class StorageHive final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; - public: String getName() const override { return "Hive"; } @@ -54,6 +53,9 @@ public: bool isColumnOriented() const override; + std::optional totalRows(const Settings & settings) const override; + std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; + protected: friend class StorageHiveSource; StorageHive( @@ -73,24 +75,56 @@ private: using FileInfo = HiveMetastoreClient::FileInfo; using HiveTableMetadataPtr = HiveMetastoreClient::HiveTableMetadataPtr; + enum class PruneLevel + { + None, /// Do not prune + Partition, + File, + Split, + Max = Split, + }; + + static String pruneLevelToString(PruneLevel level) + { + return String(magic_enum::enum_name(level)); + } + static ASTPtr extractKeyExpressionList(const ASTPtr & node); static std::vector listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs); void initMinMaxIndexExpression(); - std::vector collectHiveFilesFromPartition( - const Apache::Hadoop::Hive::Partition & partition, - SelectQueryInfo & query_info, + HiveFiles collectHiveFiles( + unsigned max_threads, + const SelectQueryInfo & query_info, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_); + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; - HiveFilePtr - createHiveFileIfNeeded(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); + HiveFiles collectHiveFilesFromPartition( + const Apache::Hadoop::Hive::Partition & partition, + const SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; + + HiveFilePtr createHiveFileIfNeeded( + const FileInfo & file_info, + const FieldVector & fields, + const SelectQueryInfo & query_info, + ContextPtr context_, + PruneLevel prune_level = PruneLevel::Max) const; void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const; + void lazyInitialize(); + + std::optional + totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const; + String hive_metastore_url; /// Hive database and table @@ -122,9 +156,8 @@ private: std::shared_ptr storage_settings; Poco::Logger * log = &Poco::Logger::get("StorageHive"); - - void lazyInitialize(); }; + } #endif From 36c583d0dee412d398a7c253e12dff6c6b670447 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Apr 2022 16:45:15 +0200 Subject: [PATCH 173/372] Better version of cache on insert --- src/Common/CurrentThread.h | 1 + src/Common/FileSegment.cpp | 72 +++++++++++++++++++ src/Common/FileSegment.h | 4 ++ src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/WriteBufferFromS3.cpp | 66 +++++++++++++++-- src/IO/WriteBufferFromS3.h | 7 +- src/Interpreters/ThreadStatusExt.cpp | 10 +++ src/Interpreters/threadPoolCallbackRunner.cpp | 9 ++- src/Interpreters/threadPoolCallbackRunner.h | 2 +- tests/config/install.sh | 2 +- 10 files changed, 163 insertions(+), 12 deletions(-) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 9dbe8d355d6..4888adb511a 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -91,6 +91,7 @@ public: struct QueryScope { explicit QueryScope(ContextMutablePtr query_context); + explicit QueryScope(ContextPtr query_context); ~QueryScope(); void logPeakMemoryUsage(); diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 76749d24f43..d8e7a994df4 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -262,6 +262,78 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) assert(getDownloadOffset() == offset_ + size); } +void FileSegment::writeInMemory(const char * from, size_t size) +{ + if (!size) + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed"); + + if (availableSize() < size) + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Not enough space is reserved. Available: {}, expected: {}", availableSize(), size); + + std::lock_guard segment_lock(mutex); + + if (cache_writer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache writer already initialized"); + + auto download_path = cache->getPathInLocalCache(key(), offset()); + cache_writer = std::make_unique(download_path, size + 1); + + try + { + cache_writer->write(from, size); + } + catch (...) + { + LOG_ERROR(log, "Failed to write to cache. File segment info: {}", getInfoForLogImpl(segment_lock)); + + download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + + cache_writer->finalize(); + cache_writer.reset(); + + throw; + } +} + +size_t FileSegment::finalizeWrite() +{ + std::lock_guard segment_lock(mutex); + + if (!cache_writer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache writer not initialized"); + + size_t size = cache_writer->offset(); + + if (size == 0) + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing size is not allowed"); + + try + { + cache_writer->next(); + } + catch (...) + { + download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + + cache_writer->finalize(); + cache_writer.reset(); + + throw; + } + + downloaded_size += size; + cache_writer.reset(); + downloader_id.clear(); + download_state = State::DOWNLOADED; + + if (downloaded_size != range().size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} == {}", downloaded_size, range().size()); + + return size; +} + FileSegment::State FileSegment::wait() { std::unique_lock segment_lock(mutex); diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index ed9d33d37d1..c9e4146c726 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -97,6 +97,10 @@ public: void write(const char * from, size_t size, size_t offset_); + void writeInMemory(const char * from, size_t size); + + size_t finalizeWrite(); + RemoteFileReaderPtr getRemoteFileReader(); void setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_); diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index f036d6a08c8..64550e9430b 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -33,7 +33,7 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock & /*buffer auto worker = read_workers.emplace_back(std::make_shared(std::move(reader))); - schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }); + schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }, nullptr); return true; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index b5e61724ede..20d9a054230 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -43,6 +44,7 @@ struct WriteBufferFromS3::UploadPartTask bool is_finised = false; std::string tag; std::exception_ptr exception; + std::optional cache_files; }; struct WriteBufferFromS3::PutObjectTask @@ -93,25 +95,50 @@ void WriteBufferFromS3::nextImpl() size_t size = offset(); temporary_buffer->write(working_buffer.begin(), size); + ThreadGroupStatusPtr running_group = CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup() + ? CurrentThread::get().getThreadGroup() + : MainThreadStatus::getInstance().getThreadGroup(); + + if (CurrentThread::isInitialized()) + query_context = CurrentThread::get().getQueryContext(); + + if (!query_context) + { + if (!shared_query_context) + { + ContextPtr global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; + if (global_context) + { + shared_query_context = Context::createCopy(global_context); + shared_query_context->makeQueryContext(); + } + } + + if (shared_query_context) + { + shared_query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); + query_context = shared_query_context; + } + } + if (cacheEnabled()) { if (blob_name.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty blob name"); auto cache_key = cache->hash(blob_name); - auto file_segments_holder = cache->setDownloading(cache_key, current_download_offset, size); + file_segments_holder.emplace(cache->setDownloading(cache_key, current_download_offset, size)); current_download_offset += size; size_t remaining_size = size; - for (const auto & file_segment : file_segments_holder.file_segments) + for (const auto & file_segment : file_segments_holder->file_segments) { size_t current_size = std::min(file_segment->range().size(), remaining_size); remaining_size -= current_size; if (file_segment->reserve(current_size)) { - file_segment->write(working_buffer.begin(), current_size); - ProfileEvents::increment(ProfileEvents::RemoteFSCacheDownloadBytes, current_size); + file_segment->writeInMemory(working_buffer.begin(), current_size); } else { @@ -273,7 +300,9 @@ void WriteBufferFromS3::writePart() /// Releasing lock and condvar notification. bg_tasks_condvar.notify_one(); } - }); + + finalizeCacheIfNeeded(); + }, query_context); } else { @@ -281,6 +310,7 @@ void WriteBufferFromS3::writePart() fillUploadRequest(task.req, part_tags.size() + 1); processUploadRequest(task); part_tags.push_back(task.tag); + finalizeCacheIfNeeded(); } } @@ -389,13 +419,15 @@ void WriteBufferFromS3::makeSinglepartUpload() bg_tasks_condvar.notify_one(); } - }); + finalizeCacheIfNeeded(); + }, query_context); } else { PutObjectTask task; fillPutRequest(task.req); processPutRequest(task); + finalizeCacheIfNeeded(); } } @@ -423,6 +455,28 @@ void WriteBufferFromS3::processPutRequest(PutObjectTask & task) throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); } +void WriteBufferFromS3::finalizeCacheIfNeeded() +{ + if (!file_segments_holder) + return; + + auto & file_segments = file_segments_holder->file_segments; + for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();) + { + try + { + size_t size = (*file_segment_it)->finalizeWrite(); + file_segment_it = file_segments.erase(file_segment_it); + + ProfileEvents::increment(ProfileEvents::RemoteFSCacheDownloadBytes, size); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + void WriteBufferFromS3::waitForReadyBackGroundTasks() { if (schedule) diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index ecddd72b9e8..1987bbe76a5 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -12,6 +12,7 @@ #include #include +#include #include #include @@ -32,7 +33,7 @@ namespace Aws::S3::Model namespace DB { -using ScheduleFunc = std::function)>; +using ScheduleFunc = std::function, ContextPtr)>; class WriteBufferFromFile; /** @@ -125,6 +126,10 @@ private: const String blob_name; FileCachePtr cache; size_t current_download_offset = 0; + std::optional file_segments_holder; + void finalizeCacheIfNeeded(); + ContextMutablePtr shared_query_context; + ContextPtr query_context; }; } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 2ea371d3d03..8fbbdb44c99 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -597,6 +597,16 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context) query_context->makeQueryContext(); } +CurrentThread::QueryScope::QueryScope(ContextPtr query_context) +{ + if (!query_context->hasQueryContext()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Cannot initialize query scope without query context"); + + CurrentThread::initializeQuery(); + CurrentThread::attachQueryContext(query_context); +} + void CurrentThread::QueryScope::logPeakMemoryUsage() { auto group = CurrentThread::getGroup(); diff --git a/src/Interpreters/threadPoolCallbackRunner.cpp b/src/Interpreters/threadPoolCallbackRunner.cpp index 288079e49d2..9eeea986d09 100644 --- a/src/Interpreters/threadPoolCallbackRunner.cpp +++ b/src/Interpreters/threadPoolCallbackRunner.cpp @@ -9,14 +9,19 @@ namespace DB CallbackRunner threadPoolCallbackRunner(ThreadPool & pool) { - return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback) mutable + return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback, ContextPtr query_context) mutable { pool->scheduleOrThrow( - [&, callback = std::move(callback), thread_group]() + [&, callback = std::move(callback), thread_group, query_context]() { if (thread_group) CurrentThread::attachTo(thread_group); + std::optional query_scope; + + if (query_context && !CurrentThread::get().getQueryContext()) + query_scope.emplace(query_context); + SCOPE_EXIT_SAFE({ if (thread_group) CurrentThread::detachQueryIfNotDetached(); diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 59d06f2f1bc..8d9d5d4d45b 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -7,7 +7,7 @@ namespace DB { /// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously -using CallbackRunner = std::function)>; +using CallbackRunner = std::function, ContextPtr)>; /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()' CallbackRunner threadPoolCallbackRunner(ThreadPool & pool); diff --git a/tests/config/install.sh b/tests/config/install.sh index f1b4fe1a588..323ded10370 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -85,7 +85,7 @@ fi if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/ - ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/s3_cache/ + ln -sf $SRC_PATH/users.d/s3_cache.xml $DEST_SERVER_PATH/config.d/ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then From 14538f645606468e4d6026c3efb8b0a3dfb0125a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Feb 2022 19:33:42 +0300 Subject: [PATCH 174/372] Add system.processors_profile_log This is the system table that will contain Processors level profiling. v2: one entry per Processor, not 3 (PortFull/NeedData/work()) v3: us over ms v4: Enable processors_profile_log table by default Signed-off-by: Azat Khuzhin --- .../system-tables/processors_profile_log.md | 73 +++++++++++++++++++ programs/server/config.xml | 9 +++ src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 1 + src/Interpreters/Context.cpp | 11 +++ src/Interpreters/Context.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 4 +- src/Interpreters/ProcessorsProfileLog.cpp | 57 +++++++++++++++ src/Interpreters/ProcessorsProfileLog.h | 42 +++++++++++ src/Interpreters/SystemLog.cpp | 4 + src/Interpreters/SystemLog.h | 3 + src/Interpreters/executeQuery.cpp | 5 ++ 12 files changed, 211 insertions(+), 1 deletion(-) create mode 100644 docs/en/operations/system-tables/processors_profile_log.md create mode 100644 src/Interpreters/ProcessorsProfileLog.cpp create mode 100644 src/Interpreters/ProcessorsProfileLog.h diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md new file mode 100644 index 00000000000..a2f851d6791 --- /dev/null +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -0,0 +1,73 @@ +# system.processors_profile_log {#system-processors_profile_log} + +This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)). + +Columns: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened. +- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened. +- `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query +- `name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the processor. +- `elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was executed. +- `need_data_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting for data (from other processor). +- `port_full_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting because output port was full. + +**Example** + +Query: + +``` sql +EXPLAIN PIPELINE +SELECT sleep(1) + +┌─explain─────────────────────────┐ +│ (Expression) │ +│ ExpressionTransform │ +│ (SettingQuotaAndLimits) │ +│ (ReadFromStorage) │ +│ SourceFromSingleChunk 0 → 1 │ +└─────────────────────────────────┘ + +SELECT sleep(1) +SETTINGS log_processors_profiles = 1 + +Query id: feb5ed16-1c24-4227-aa54-78c02b3b27d4 + +┌─sleep(1)─┐ +│ 0 │ +└──────────┘ + +1 rows in set. Elapsed: 1.018 sec. + +SELECT + name, + elapsed_us, + need_data_elapsed_us, + port_full_elapsed_us +FROM system.processors_profile_log +WHERE query_id = 'feb5ed16-1c24-4227-aa54-78c02b3b27d4' +ORDER BY name ASC +``` + +Result: + +``` text +┌─name────────────────────┬─elapsed_us─┬─need_data_elapsed_us─┬─port_full_elapsed_us─┐ +│ ExpressionTransform │ 1000497 │ 2823 │ 197 │ +│ LazyOutputFormat │ 36 │ 1002188 │ 0 │ +│ LimitsCheckingTransform │ 10 │ 1002994 │ 106 │ +│ NullSource │ 5 │ 1002074 │ 0 │ +│ NullSource │ 1 │ 1002084 │ 0 │ +│ SourceFromSingleChunk │ 45 │ 4736 │ 1000819 │ +└─────────────────────────┴────────────┴──────────────────────┴──────────────────────┘ +``` + +Here you can see: + +- `ExpressionTransform` was executing `sleep(1)` function, so it `work` will takes 1e6, and so `elapsed_us` > 1e6. +- `SourceFromSingleChunk` need to wait, because `ExpressionTransform` does not accept any data during execution of `sleep(1)`, so it will be in `PortFull` state for 1e6 us, and so `port_full_elapsed_us` > 1e6. +- `LimitsCheckingTransform`/`NullSource`/`LazyOutputFormat` need to wait until `ExpressionTransform` will execute `sleep(1)` to process the result, so `need_data_elapsed_us` > 1e6. + +**See Also** + +- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) diff --git a/programs/server/config.xml b/programs/server/config.xml index 3b035fb39ac..3bb26a3a368 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1042,6 +1042,15 @@ 7500 --> + + + system + processors_profile_log
+ + toYYYYMM(event_date) + 7500 +
+ From a34d9f9cc3e9e19be65b01c6e3f12dd01b4a246b Mon Sep 17 00:00:00 2001 From: Roman Nikonov Date: Wed, 6 Apr 2022 02:40:56 +0300 Subject: [PATCH 235/372] Remove arg duplication and fix a typo --- tests/integration/runner | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 737eaeef683..522fdb7b745 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -76,7 +76,7 @@ def check_args_and_update_paths(args): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.dockerd_volume)) if (not os.path.exists(os.path.join(args.base_configs_dir, "config.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "config.yaml"))): - raise Exception("No configs.xml or configs.yaml in {}".format(args.base_configs_dir)) + raise Exception("No config.xml or config.yaml in {}".format(args.base_configs_dir)) if (not os.path.exists(os.path.join(args.base_configs_dir, "users.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "users.yaml"))): raise Exception("No users.xml or users.yaml in {}".format(args.base_configs_dir)) @@ -279,7 +279,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged \ --volume={odbc_bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ - --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ + --volume={library_bridge_bin}:/clickhouse-library-bridge \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ --volume=/run:/run/host:ro \ From ea9ce3ea18cbe80ce63c2b02867dc11e39ac0bd8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Apr 2022 19:50:16 -0400 Subject: [PATCH 236/372] 'T' is added as delimiter, tests added --- src/IO/ReadHelpers.h | 5 +++-- .../0_stateless/02249_parse_date_time_basic.reference | 8 +++++--- tests/queries/0_stateless/02249_parse_date_time_basic.sql | 6 +++++- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 48c291d8fcc..e68da3a1c7d 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -867,7 +867,8 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons UInt8 minute = 0; UInt8 second = 0; ///simply determine whether it is YYYY-MM-DD hh:mm:ss or YYYY-MM-DD by the content of the tenth character in an optimistic scenario - if (s[10] == ' ') + bool dt_long = (s[10] == ' ' || s[10] == 'T'); + if (dt_long) { hour = (s[11] - '0') * 10 + (s[12] - '0'); minute = (s[14] - '0') * 10 + (s[15] - '0'); @@ -879,7 +880,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons else datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); - if (s[10] == ' ') + if (dt_long) buf.position() += DateTimeStringInputSize; else buf.position() += DateStringInputSize; diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.reference b/tests/queries/0_stateless/02249_parse_date_time_basic.reference index d67e0ae15e0..027c72d802f 100644 --- a/tests/queries/0_stateless/02249_parse_date_time_basic.reference +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.reference @@ -1,3 +1,5 @@ -2022-03-31 00:00:00 1 -2022-04-01 17:10:24 2 -2022-03-31 10:18:56 3 +2022-03-31T04:00:00Z 1 +2022-04-01T09:10:24Z 2 +2022-03-31T14:18:56Z 3 +2022-03-31T14:18:56Z 4 +2022-04-01T09:10:24Z 5 diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.sql b/tests/queries/0_stateless/02249_parse_date_time_basic.sql index 2cea41874d5..cb443bbdd8e 100644 --- a/tests/queries/0_stateless/02249_parse_date_time_basic.sql +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.sql @@ -1,6 +1,10 @@ +SET date_time_output_format='iso'; drop table if exists t; CREATE TABLE t (a DateTime, b String, c String, d String, e Int32) ENGINE = Memory; INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31','','','',1); INSERT INTO t(a, b, c, d ,e) VALUES (1648804224,'','','',2); INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31 10:18:56','','','',3); -select a, e from t; +INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31T10:18:56','','','',4); +INSERT INTO t(a, b, c, d ,e) VALUES ('1648804224','','','',5); +select a, e from t order by e; +drop table if exists t; From d59d4eda4f3ffc4c45ea543fd5acaab64243bd72 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 5 Apr 2022 15:35:59 +0200 Subject: [PATCH 237/372] Fix tests --- docker/test/fuzzer/run-fuzzer.sh | 1 - src/Client/ClientBase.cpp | 7 +++++++ tests/clickhouse-test | 2 +- .../0_stateless/00921_datetime64_compatibility_long.sh | 2 +- tests/queries/0_stateless/02221_parallel_replicas_bug.sh | 2 +- 5 files changed, 10 insertions(+), 4 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 74711f476f8..32799a669eb 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -226,7 +226,6 @@ quit --receive_data_timeout_ms=10000 \ --stacktrace \ --query-fuzzer-runs=1000 \ - --testmode \ --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ $NEW_TESTS_OPT \ > >(tail -n 100000 > fuzzer.log) \ diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 24dba19a72c..e1d2b673571 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1499,6 +1499,13 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) /// Test tags are started with "--" so they are interpreted as comments anyway. /// But if the echo is enabled we have to remove the test tags from `all_queries_text` /// because we don't want test tags to be echoed. + { + /// disable logs if expects errors + TestHint test_hint(all_queries_text); + if (test_hint.clientError() || test_hint.serverError()) + processTextAsSingleQuery("SET send_logs_level = 'fatal'"); + } + size_t test_tags_length = getTestTagsLength(all_queries_text); /// Several queries separated by ';'. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f925fddcd1a..de36fc3da27 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1140,7 +1140,7 @@ def run_tests_array(all_tests_with_params): sys.stdout.flush() -server_logs_level = "fatal" +server_logs_level = "warning" def check_server_started(args): diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.sh b/tests/queries/0_stateless/00921_datetime64_compatibility_long.sh index 6d2cd0a176b..d310a2c3612 100755 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.sh +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.sh @@ -13,5 +13,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # ${CURDIR}/00921_datetime64_compatibility.python python3 "${CURDIR}"/00921_datetime64_compatibility_long.python \ - | ${CLICKHOUSE_CLIENT} --ignore-error -T -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \ + | ${CLICKHOUSE_CLIENT} --ignore-error -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \ | grep -v -e 'Received exception .*$' -e '^(query: ' | sed 's/^\(Code: [0-9]\+\).*$/\1/g' diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index b4ac6817a54..cce32bf8272 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 -nmT < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null From f4bfa0fea3ab02401df49b676f2f1f92c0a60f82 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 5 Apr 2022 19:01:41 +0200 Subject: [PATCH 238/372] Update test.py --- .../test_storage_postgresql_replica/test.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index fe8b92ea606..8cc4b2049a2 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -106,7 +106,7 @@ def create_clickhouse_postgres_db(ip, port, name="postgres_database"): def create_materialized_table(ip, port, table_name='postgresql_replica'): instance.query( f""" - CREATE TABLE test.{table_name} (key UInt64, value UInt64) + CREATE TABLE test.{table_name} (key Int64, value Int64) ENGINE = MaterializedPostgreSQL( '{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword') PRIMARY KEY key; """ @@ -703,11 +703,21 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): table_name = "postgresql_replica_697" create_postgres_table(cursor, table_name) + instance.query( + f"INSERT INTO postgres_database.{table_name} SELECT -1, 1" + ) instance.query(f"DROP TABLE IF EXISTS test.{table_name}") create_materialized_table( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, table_name=table_name ) + n = 1 + while int(instance.query(f"select count() from test.{table_name}")) != 1: + sleep(1) + n += 1 + if n > 10: + break; + for query in queries: cursor.execute(query.format(table_name)) print("query {}".format(query.format(table_name))) From c7d72b92dac1d3650c61bdfd3d6a4286dbc97819 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Apr 2022 20:32:52 -0400 Subject: [PATCH 239/372] explicit timezone added to test --- .../0_stateless/02249_parse_date_time_basic.reference | 6 +++--- tests/queries/0_stateless/02249_parse_date_time_basic.sql | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.reference b/tests/queries/0_stateless/02249_parse_date_time_basic.reference index 027c72d802f..eb030a8fd3d 100644 --- a/tests/queries/0_stateless/02249_parse_date_time_basic.reference +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.reference @@ -1,5 +1,5 @@ -2022-03-31T04:00:00Z 1 +2022-03-31T00:00:00Z 1 2022-04-01T09:10:24Z 2 -2022-03-31T14:18:56Z 3 -2022-03-31T14:18:56Z 4 +2022-03-31T10:18:56Z 3 +2022-03-31T10:18:56Z 4 2022-04-01T09:10:24Z 5 diff --git a/tests/queries/0_stateless/02249_parse_date_time_basic.sql b/tests/queries/0_stateless/02249_parse_date_time_basic.sql index cb443bbdd8e..7146462fb74 100644 --- a/tests/queries/0_stateless/02249_parse_date_time_basic.sql +++ b/tests/queries/0_stateless/02249_parse_date_time_basic.sql @@ -1,6 +1,6 @@ SET date_time_output_format='iso'; drop table if exists t; -CREATE TABLE t (a DateTime, b String, c String, d String, e Int32) ENGINE = Memory; +CREATE TABLE t (a DateTime('UTC'), b String, c String, d String, e Int32) ENGINE = Memory; INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31','','','',1); INSERT INTO t(a, b, c, d ,e) VALUES (1648804224,'','','',2); INSERT INTO t(a, b, c, d ,e) VALUES ('2022-03-31 10:18:56','','','',3); From 43e8af697aa57ce3b2bb593f4cb147ae54e338c4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 11:41:16 +0800 Subject: [PATCH 240/372] fix code style --- src/Storages/Hive/HiveFile.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 407d9602b61..867b5650ea2 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -139,7 +139,7 @@ void HiveOrcFile::prepareColumnMapping() for (size_t pos = 0; pos < count; pos++) { /// Column names in hive is case-insensitive. - String column{type.getFieldName(pos)}; + String columnn{type.getFieldName(pos)}; boost::to_lower(column); orc_column_positions[column] = pos; } @@ -267,9 +267,9 @@ void HiveParquetFile::loadSubMinMaxIndex() auto it = index_names_and_types.begin(); for (; it != index_names_and_types.end(); ++j, ++it) { - String name{it->name}; - boost::to_lower(name); - auto mit = parquet_column_positions.find(name); + String column{it->name}; + boost::to_lower(column); + auto mit = parquet_column_positions.find(column); if (mit == parquet_column_positions.end()) continue; From ac441b9dcf96120139be4efaa79def31369795e4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 6 Apr 2022 00:04:51 -0400 Subject: [PATCH 241/372] compiler suggestions --- src/Interpreters/ExpressionAnalyzer.cpp | 8 ++++---- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 8 ++++---- src/Parsers/ASTSelectQuery.cpp | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 40539fdaf76..4494ce82157 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1338,19 +1338,19 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { auto find_columns = [&step](IAST * function) { - auto fImpl = [&step](IAST * fn, auto fi) + auto f_impl = [&step](IAST * fn, auto fi) { - if (auto ident = fn->as()) + if (auto * ident = fn->as()) { step.addRequiredOutput(ident->getColumnName()); return; } if (fn->as() || fn->as()) - for (auto ch : fn->children) + for (const auto & ch : fn->children) fi(ch.get(), fi); return; }; - fImpl(function, fImpl); + f_impl(function, f_impl); }; for (const auto & interpolate : interpolate_list->children) diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index f6907aaafd7..8ff47ee7972 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -128,19 +128,19 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS { auto find_columns = [&data](IAST * function) { - auto fImpl = [&data](IAST * fn, auto fi) + auto f_impl = [&data](IAST * fn, auto fi) { - if (auto ident = fn->as()) + if (auto * ident = fn->as()) { data.addColumnIdentifier(*ident); return; } if (fn->as() || fn->as()) - for (auto ch : fn->children) + for (const auto & ch : fn->children) fi(ch.get(), fi); return; }; - fImpl(function, fImpl); + f_impl(function, f_impl); }; for (const auto & interpolate : interpolate_list->children) diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 1a12877c1c8..048b9dfc38f 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -133,7 +133,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (interpolate()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTERPOLATE" << (s.hilite ? hilite_none : ""); - if (interpolate()->children.size()) + if (!interpolate()->children.empty()) { s.ostr << " ("; interpolate()->formatImpl(s, state, frame); From a2ce366c3420a2806eb6974e3cb45020115deb39 Mon Sep 17 00:00:00 2001 From: fenglv Date: Wed, 6 Apr 2022 04:49:43 +0000 Subject: [PATCH 242/372] parallel reading files for FileLog Engine --- src/Storages/FileLog/StorageFileLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 32ca936f039..4fb19b12bab 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -723,6 +723,7 @@ bool StorageFileLog::streamToViews() size_t rows = 0; { block_io.pipeline.complete(std::move(input)); + block_io.pipeline.setNumThreads(max_streams_number); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); From 84eef61d17ee430f2703f24e17c0ab8bb05715b9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Apr 2022 06:39:56 +0000 Subject: [PATCH 243/372] Pull under reader mutex --- src/Storages/StorageURL.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index a435ab1a654..7bdb070bfdf 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -224,14 +224,12 @@ namespace } Chunk chunk; + std::lock_guard lock(reader_mutex); if (reader->pull(chunk)) return chunk; - { - std::lock_guard lock(reader_mutex); - pipeline->reset(); - reader.reset(); - } + pipeline->reset(); + reader.reset(); } } From bf1f34ddb372ad45ccfd0297e6cd328abfc06f6b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Apr 2022 07:43:48 +0000 Subject: [PATCH 244/372] Fix unit tests --- src/Coordination/tests/gtest_coordination.cpp | 56 +++++++++---------- utils/keeper-data-dumper/main.cpp | 4 +- 2 files changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 07544dfbb89..7dfd451e111 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -976,31 +976,31 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) using Node = DB::KeeperStorage::Node; DB::SnapshotableHashTable world; Node n1; - n1.data = "1234"; + n1.setData("1234"); Node n2; - n2.data = "123456"; - n2.children.insert(""); + n2.setData("123456"); + n2.addChild(""); world.disableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 98); + EXPECT_EQ(world.getApproximateDataSize(), 177); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 98); + EXPECT_EQ(world.getApproximateDataSize(), 195); world.erase("world"); EXPECT_EQ(world.getApproximateDataSize(), 0); world.enableSnapshotMode(100000); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 98); + EXPECT_EQ(world.getApproximateDataSize(), 177); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 196); + EXPECT_EQ(world.getApproximateDataSize(), 372); world.clearOutdatedNodes(); - EXPECT_EQ(world.getApproximateDataSize(), 98); + EXPECT_EQ(world.getApproximateDataSize(), 195); world.erase("world"); - EXPECT_EQ(world.getApproximateDataSize(), 98); + EXPECT_EQ(world.getApproximateDataSize(), 195); world.clear(); EXPECT_EQ(world.getApproximateDataSize(), 0); @@ -1010,7 +1010,7 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s { using Node = DB::KeeperStorage::Node; Node node{}; - node.data = data; + node.setData(data); node.stat.ephemeralOwner = ephemeral_owner; storage.container.insertOrReplace(path, node); } @@ -1048,13 +1048,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 3); - EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello").children.size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").children.size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); - EXPECT_EQ(restored_storage->container.getValue("/").data, ""); - EXPECT_EQ(restored_storage->container.getValue("/hello").data, "world"); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").data, "somedata"); + EXPECT_EQ(restored_storage->container.getValue("/").getData(), ""); + EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world"); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->zxid, 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); @@ -1099,7 +1099,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) EXPECT_EQ(restored_storage->container.size(), 51); for (size_t i = 0; i < 50; ++i) { - EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i)); + EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); } } @@ -1139,7 +1139,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) for (size_t i = 0; i < 250; ++i) { - EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i)); + EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); } } @@ -1162,7 +1162,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) } for (size_t i = 0; i < 50; ++i) { - EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).data, "wlrd_" + std::to_string(i)); + EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).getData(), "wlrd_" + std::to_string(i)); } for (size_t i = 0; i < 50; ++i) { @@ -1182,7 +1182,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) for (size_t i = 0; i < 50; ++i) { if (i % 2 != 0) - EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).data, "wlrd_" + std::to_string(i)); + EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).getData(), "wlrd_" + std::to_string(i)); else EXPECT_FALSE(storage.container.contains("/hello_" + std::to_string(i))); } @@ -1191,7 +1191,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) for (size_t i = 0; i < 50; ++i) { - EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i)); + EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); } } @@ -1314,7 +1314,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint for (size_t i = 1; i < total_logs + 1; ++i) { auto path = "/hello_" + std::to_string(i); - EXPECT_EQ(source_storage.container.getValue(path).data, restored_storage.container.getValue(path).data); + EXPECT_EQ(source_storage.container.getValue(path).getData(), restored_storage.container.getValue(path).getData()); } } @@ -1589,13 +1589,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 3); - EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello").children.size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").children.size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); - EXPECT_EQ(restored_storage->container.getValue("/").data, ""); - EXPECT_EQ(restored_storage->container.getValue("/hello").data, "world"); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").data, "somedata"); + EXPECT_EQ(restored_storage->container.getValue("/").getData(), ""); + EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world"); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->zxid, 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 0f86d34d334..df6083e4bd7 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -32,9 +32,9 @@ void dumpMachine(std::shared_ptr machine) ", numChildren: " << value.stat.numChildren << ", dataLength: " << value.stat.dataLength << "}" << std::endl; - std::cout << "\tData: " << storage.container.getValue(key).data << std::endl; + std::cout << "\tData: " << storage.container.getValue(key).getData() << std::endl; - for (const auto & child : value.children) + for (const auto & child : value.getChildren()) { if (key == "/") keys.push(key + child.toString()); From e4408d42dc12dc0fe855579981376db0c85eaa4b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 10:13:09 +0200 Subject: [PATCH 245/372] Fix context expanding, quote secrets to avoid word split --- .github/workflows/nightly.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index b9541589a64..1e70213adf5 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -105,13 +105,13 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" "$${{ secrets.COV_TOKEN }}" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" "${{ secrets.COV_TOKEN }}" - name: Upload Coverity Analysis if: ${{ success() || failure() }} run: | - curl --form token=$${{ secrets.COV_TOKEN }} \ - --form email=${{ secrets.ROBOT_CLICKHOUSE_EMAIL }} \ - --form file=@$TEMP_PATH/$BUILD_NAME/clickhouse-scan.tgz \ + curl --form token='${{ secrets.COV_TOKEN }}' \ + --form email='${{ secrets.ROBOT_CLICKHOUSE_EMAIL }}' \ + --form file="@$TEMP_PATH/$BUILD_NAME/clickhouse-scan.tgz" \ --form version="${GITHUB_REF#refs/heads/}-${GITHUB_SHA::6}" \ --form description="Nighly Scan: $(date +'%Y-%m-%dT%H:%M:%S')" \ https://scan.coverity.com/builds?project=ClickHouse%2FClickHouse From 62fa528e7d7baa4e48c84d7e02dff179faacffd0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 10:15:36 +0200 Subject: [PATCH 246/372] Fix syntax error in build_check.py --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 1dcfb9d01d3..74ebebf1e6f 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -302,7 +302,7 @@ def main(): version.string, image_version, ccache_path, - official=official_flag, + official_flag, cov_token, ) From 06ff47f72f4e634b35a48122efdfea4122682531 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Apr 2022 10:36:54 +0200 Subject: [PATCH 247/372] Add drops --- .../test_storage_postgresql_replica/test.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 8cc4b2049a2..d2ac5d1a109 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -174,6 +174,7 @@ def test_initial_load_from_snapshot(started_cluster): cursor.execute("DROP TABLE postgresql_replica;") postgresql_replica_check_result(result, True) + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") @pytest.mark.timeout(320) @@ -210,6 +211,7 @@ def test_no_connection_at_startup(started_cluster): result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") cursor.execute("DROP TABLE postgresql_replica;") postgresql_replica_check_result(result, True) + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") @pytest.mark.timeout(320) @@ -248,6 +250,7 @@ def test_detach_attach_is_ok(started_cluster): cursor.execute("DROP TABLE postgresql_replica;") postgresql_replica_check_result(result, True) + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") @pytest.mark.timeout(320) @@ -301,6 +304,7 @@ def test_replicating_insert_queries(started_cluster): result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") cursor.execute("DROP TABLE postgresql_replica;") postgresql_replica_check_result(result, True) + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") @pytest.mark.timeout(320) @@ -657,6 +661,7 @@ def test_virtual_columns(started_cluster): ) print(result) cursor.execute("DROP TABLE postgresql_replica;") + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") def test_abrupt_connection_loss_while_heavy_replication(started_cluster): @@ -691,6 +696,7 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): result = instance.query("SELECT count() FROM test.postgresql_replica") print(result) # Just debug + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") def test_abrupt_server_restart_while_heavy_replication(started_cluster): @@ -706,7 +712,7 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): instance.query( f"INSERT INTO postgres_database.{table_name} SELECT -1, 1" ) - instance.query(f"DROP TABLE IF EXISTS test.{table_name}") + instance.query(f"DROP TABLE IF EXISTS test.{table_name} NO DELAY") create_materialized_table( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, table_name=table_name ) @@ -731,6 +737,7 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): result = instance.query(f"SELECT count() FROM test.{table_name}") print(result) # Just debug + instance.query(f"DROP TABLE test.{table_name} NO DELAY") def test_drop_table_immediately(started_cluster): @@ -754,7 +761,7 @@ def test_drop_table_immediately(started_cluster): ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) check_tables_are_synchronized("postgresql_replica") - instance.query("DROP TABLE test.postgresql_replica") + instance.query(f"DROP TABLE test.postgresql_replica NO DELAY") if __name__ == "__main__": From acb9f1632eeda8489cc2eecdb2c9fcb5541ebc16 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 16:40:22 +0800 Subject: [PATCH 248/372] suppoort skip splits in orc and parquet --- src/Formats/FormatSettings.h | 2 + .../Formats/Impl/ORCBlockInputFormat.cpp | 42 ++++++++++++------- .../Formats/Impl/ORCBlockInputFormat.h | 5 ++- .../Formats/Impl/ParquetBlockInputFormat.cpp | 5 ++- .../Formats/Impl/ParquetBlockInputFormat.h | 3 +- src/Storages/Hive/HiveFile.cpp | 2 +- src/Storages/Hive/HiveFile.h | 6 +-- src/Storages/Hive/StorageHive.cpp | 22 +++++++--- 8 files changed, 58 insertions(+), 29 deletions(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bd0a84d9ded..695c63d6379 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -138,6 +138,7 @@ struct FormatSettings bool import_nested = false; bool allow_missing_columns = false; bool case_insensitive_column_matching = false; + std::unordered_set skip_row_groups = {}; } parquet; struct Pretty @@ -219,6 +220,7 @@ struct FormatSettings bool allow_missing_columns = false; int64_t row_batch_size = 100'000; bool case_insensitive_column_matching = false; + std::unordered_set skip_stripes = {}; } orc; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index c68b59833db..8b8426132de 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -20,13 +20,12 @@ namespace ErrorCodes } ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) - : IInputFormat(std::move(header_), in_), format_settings(format_settings_) + : IInputFormat(std::move(header_), in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) { } Chunk ORCBlockInputFormat::generate() { - Chunk res; block_missing_values.clear(); if (!file_reader) @@ -35,24 +34,32 @@ Chunk ORCBlockInputFormat::generate() if (is_stopped) return {}; - std::shared_ptr batch_reader; - auto result = file_reader->NextStripeReader(format_settings.orc.row_batch_size, include_indices); - if (!result.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", result.status().ToString()); - batch_reader = std::move(result).ValueOrDie(); - if (!batch_reader) - { - return res; - } + for (; stripe_current < stripe_total && skip_stripes.contains(stripe_current); ++stripe_current) + ; - std::shared_ptr table; - arrow::Status table_status = batch_reader->ReadAll(&table); - if (!table_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_status.ToString()); + if (stripe_current >= stripe_total) + return {}; + auto batch_result = file_reader->ReadStripe(stripe_current, include_indices); + if (!batch_result.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString()); + + auto batch = batch_result.ValueOrDie(); + if (!batch) + return {}; + + auto table_result = arrow::Table::FromRecordBatches({batch}); + if (!table_result.ok()) + throw ParsingException( + ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString()); + + auto table = table_result.ValueOrDie(); if (!table || !table->num_rows()) - return res; + return {}; + ++stripe_current; + + Chunk res; arrow_column_to_ch_column->arrowTableToCHChunk(res, table); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. @@ -130,6 +137,9 @@ void ORCBlockInputFormat::prepareReader() if (is_stopped) return; + stripe_total = file_reader->NumberOfStripes(); + stripe_current = 0; + arrow_column_to_ch_column = std::make_unique( getPort().getHeader(), "ORC", diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index b7a771730ea..3c363699a0b 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -38,6 +38,7 @@ protected: } private: + void prepareReader(); // TODO: check that this class implements every part of its parent @@ -52,8 +53,10 @@ private: BlockMissingValues block_missing_values; const FormatSettings format_settings; + const std::unordered_set & skip_stripes; - void prepareReader(); + int stripe_total = 0; + int stripe_current = 0; std::atomic is_stopped{0}; }; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 13582ce5019..69e51e0dad2 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes } while (false) ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) - : IInputFormat(std::move(header_), in_), format_settings(format_settings_) + : IInputFormat(std::move(header_), in_), format_settings(format_settings_), skip_row_groups(format_settings.parquet.skip_row_groups) { } @@ -47,6 +47,9 @@ Chunk ParquetBlockInputFormat::generate() if (is_stopped) return {}; + for (; row_group_current < row_group_total && skip_row_groups.contains(row_group_current); ++row_group_current) + ; + if (row_group_current >= row_group_total) return res; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 1faadaa3d21..76803bb5b89 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -38,13 +38,14 @@ private: std::unique_ptr file_reader; int row_group_total = 0; + int row_group_current = 0; // indices of columns to read from Parquet file std::vector column_indices; std::unique_ptr arrow_column_to_ch_column; - int row_group_current = 0; std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; + const std::unordered_set & skip_row_groups; std::atomic is_stopped{0}; }; diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 867b5650ea2..3f4260d9f9e 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -139,7 +139,7 @@ void HiveOrcFile::prepareColumnMapping() for (size_t pos = 0; pos < count; pos++) { /// Column names in hive is case-insensitive. - String columnn{type.getFieldName(pos)}; + String column{type.getFieldName(pos)}; boost::to_lower(column); orc_column_positions[column] = pos; } diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index aef9d72755a..74f893a073f 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -125,9 +125,9 @@ public: virtual const std::vector & getSubMinMaxIndexes() const { return sub_minmax_idxes; } - virtual void setSkipSplits(const std::set & splits) { skip_splits = splits; } + virtual void setSkipSplits(const std::unordered_set & skip_splits_) { skip_splits = skip_splits_; } - virtual const std::set & getSkipSplits() const { return skip_splits; } + virtual const std::unordered_set & getSkipSplits() const { return skip_splits; } inline std::string describeMinMaxIndex(const MinMaxIndexPtr & idx) const { @@ -157,7 +157,7 @@ protected: MinMaxIndexPtr minmax_idx; std::vector sub_minmax_idxes; /// Skip splits for this file after applying minmax index (if any) - std::set skip_splits; + std::unordered_set skip_splits; std::shared_ptr storage_settings; }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 4296df3d7b1..a7537f5a1f6 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -111,9 +111,9 @@ public: : SourceWithProgress(getHeader(sample_block_, source_info_)) , WithContext(context_) , source_info(std::move(source_info_)) - , hdfs_namenode_url(hdfs_namenode_url_) + , hdfs_namenode_url(std::move(hdfs_namenode_url_)) , format(std::move(format_)) - , compression_method(compression_method_) + , compression_method(std::move(compression_method_)) , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) , columns_description(getColumnsDescription(sample_block, source_info)) @@ -121,15 +121,25 @@ public: , format_settings(getFormatSettings(getContext())) { to_read_block = sample_block; + /// Initialize to_read_block, which is used to read data from HDFS. for (const auto & name_type : source_info->partition_name_types) { if (to_read_block.has(name_type.name)) to_read_block.erase(name_type.name); } + } - /// Initialize format settings - format_settings.hive_text.input_field_names = text_input_field_names; + FormatSettings updateFormatSettings(const HiveFilePtr & hive_file) + { + auto updated = format_settings; + if (format == "HiveText") + updated.hive_text.input_field_names = text_input_field_names; + else if (format == "ORC") + updated.orc.skip_stripes = hive_file->getSkipSplits(); + else if (format == "Parquet") + updated.parquet.skip_row_groups = hive_file->getSkipSplits(); + return updated; } String getName() const override { return "Hive"; } @@ -188,7 +198,7 @@ public: read_buf = std::move(remote_read_buf); auto input_format = FormatFactory::instance().getInputFormat( - format, *read_buf, to_read_block, getContext(), max_block_size, format_settings); + format, *read_buf, to_read_block, getContext(), max_block_size, updateFormatSettings(curr_file)); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -545,7 +555,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( /// Load sub-file level minmax index and apply if (hive_file->hasSubMinMaxIndex()) { - std::set skip_splits; + std::unordered_set skip_splits; hive_file->loadSubMinMaxIndex(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) From 060f5118f00fef5ee3ce20fb8d8f8f7b50c5c9eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Apr 2022 10:48:12 +0200 Subject: [PATCH 249/372] Improvements based on PR review --- src/Common/DNSResolver.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 66f67caf456..1a3eaf91f68 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -123,7 +123,10 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) } if (addresses.empty()) + { + ProfileEvents::increment(ProfileEvents::DNSError); throw Exception("Not found address of host: " + host, ErrorCodes::DNS_ERROR); + } return addresses; } @@ -142,8 +145,8 @@ static String reverseResolveImpl(const Poco::Net::IPAddress & address) struct DNSResolver::Impl { - using HostWithConsecutiveFailures = std::unordered_map; - using AddressWithConsecutiveFailures = std::unordered_map; + using HostWithConsecutiveFailures = std::unordered_map; + using AddressWithConsecutiveFailures = std::unordered_map; CachedFn<&resolveIPAddressImpl> cache_host; CachedFn<&reverseResolveImpl> cache_address; @@ -275,7 +278,6 @@ bool DNSResolver::updateCacheImpl( tryLogCurrentException(log, __PRETTY_FUNCTION__); continue; } - ProfileEvents::increment(ProfileEvents::DNSError); if (!lost_elems.empty()) lost_elems += ", "; lost_elems += cacheElemToString(it->first); From 9e7ffcce59fb7b013149f1036eedcfa0a47c46d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Apr 2022 11:09:08 +0200 Subject: [PATCH 250/372] Add test to check DNS profile events --- .../02265_test_dns_profile_events.reference | 2 ++ .../02265_test_dns_profile_events.sh | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02265_test_dns_profile_events.reference create mode 100755 tests/queries/0_stateless/02265_test_dns_profile_events.sh diff --git a/tests/queries/0_stateless/02265_test_dns_profile_events.reference b/tests/queries/0_stateless/02265_test_dns_profile_events.reference new file mode 100644 index 00000000000..97ca33b311f --- /dev/null +++ b/tests/queries/0_stateless/02265_test_dns_profile_events.reference @@ -0,0 +1,2 @@ +first_check 1 +second_check 1 diff --git a/tests/queries/0_stateless/02265_test_dns_profile_events.sh b/tests/queries/0_stateless/02265_test_dns_profile_events.sh new file mode 100755 index 00000000000..756a761a0ae --- /dev/null +++ b/tests/queries/0_stateless/02265_test_dns_profile_events.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +current_dns_errors=$($CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM system.events where event = 'DNSError';") +${CLICKHOUSE_CLIENT} --query "SELECT * FROM remote('ThisHostNameDoesNotExistSoItShouldFail', system, one)" 2>/dev/null +${CLICKHOUSE_CLIENT} --query "SELECT 'first_check', sum(value) > ${current_dns_errors} FROM system.events where event = 'DNSError';" + +current_dns_errors=$($CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM system.events where event = 'DNSError';") +${CLICKHOUSE_CLIENT} --query "SELECT * FROM remote('ThisHostNameDoesNotExistSoItShouldFail2', system, one)" 2>/dev/null +${CLICKHOUSE_CLIENT} --query "SELECT 'second_check', sum(value) > ${current_dns_errors} FROM system.events where event = 'DNSError';" + +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP DNS CACHE" From df06f9f974102cc6c5b45e622fc3a5a5f4fa1ec5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Apr 2022 14:53:10 +0800 Subject: [PATCH 251/372] Fix performance regression of scalar query --- src/Functions/getScalar.cpp | 2 +- src/Interpreters/Context.cpp | 10 +++++----- src/Interpreters/Context.h | 7 ++++--- .../ExecuteScalarSubqueriesVisitor.cpp | 8 ++++++-- .../IInterpreterUnionOrSelectQuery.h | 4 ++-- tests/performance/scalar2.xml | 17 +++++++++++++++++ 6 files changed, 35 insertions(+), 13 deletions(-) create mode 100644 tests/performance/scalar2.xml diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index b06fb360366..c165ef26ffa 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -78,7 +78,7 @@ public: static ColumnWithTypeAndName createScalar(ContextPtr context_) { - if (const auto * block = context_->tryGetLocalScalar(Scalar::scalar_name)) + if (const auto * block = context_->tryGetSpecialScalar(Scalar::scalar_name)) return block->getByPosition(0); else if (context_->hasQueryContext()) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e837ce5dae1..493e250ea85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -930,10 +930,10 @@ const Block & Context::getScalar(const String & name) const return it->second; } -const Block * Context::tryGetLocalScalar(const String & name) const +const Block * Context::tryGetSpecialScalar(const String & name) const { - auto it = local_scalars.find(name); - if (local_scalars.end() == it) + auto it = special_scalars.find(name); + if (special_scalars.end() == it) return nullptr; return &it->second; } @@ -1004,12 +1004,12 @@ void Context::addScalar(const String & name, const Block & block) } -void Context::addLocalScalar(const String & name, const Block & block) +void Context::addSpecialScalar(const String & name, const Block & block) { if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have local scalars"); - local_scalars[name] = block; + special_scalars[name] = block; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ad68f2a2245..31d853318d4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -227,7 +227,8 @@ private: /// Thus, used in HTTP interface. If not specified - then some globally default format is used. TemporaryTablesMapping external_tables_mapping; Scalars scalars; - Scalars local_scalars; + /// Used to store constant values which are different on each instance during distributed plan, such as _shard_num. + Scalars special_scalars; /// Used in s3Cluster table function. With this callback, a worker node could ask an initiator /// about next file to read from s3. @@ -487,8 +488,8 @@ public: void addScalar(const String & name, const Block & block); bool hasScalar(const String & name) const; - const Block * tryGetLocalScalar(const String & name) const; - void addLocalScalar(const String & name, const Block & block); + const Block * tryGetSpecialScalar(const String & name) const; + void addSpecialScalar(const String & name, const Block & block); const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; } void addQueryAccessInfo( diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index ac8a27484d9..55fdaffdfe9 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -113,14 +113,18 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr bool is_local = false; Block scalar; - if (data.local_scalars.count(scalar_query_hash_str)) + if (data.only_analyze) + { + /// Don't use scalar cache during query analysis + } + else if (data.local_scalars.contains(scalar_query_hash_str)) { hit = true; scalar = data.local_scalars[scalar_query_hash_str]; is_local = true; ProfileEvents::increment(ProfileEvents::ScalarSubqueriesLocalCacheHit); } - else if (data.scalars.count(scalar_query_hash_str)) + else if (data.scalars.contains(scalar_query_hash_str)) { hit = true; scalar = data.scalars[scalar_query_hash_str]; diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 7906ab189fc..a9262fbfa1e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -18,11 +18,11 @@ public: , max_streams(context->getSettingsRef().max_threads) { if (options.shard_num) - context->addLocalScalar( + context->addSpecialScalar( "_shard_num", Block{{DataTypeUInt32().createColumnConst(1, *options.shard_num), std::make_shared(), "_shard_num"}}); if (options.shard_count) - context->addLocalScalar( + context->addSpecialScalar( "_shard_count", Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared(), "_shard_count"}}); } diff --git a/tests/performance/scalar2.xml b/tests/performance/scalar2.xml new file mode 100644 index 00000000000..eb427536646 --- /dev/null +++ b/tests/performance/scalar2.xml @@ -0,0 +1,17 @@ + + CREATE TABLE tbl0 (`ds` Date, `x1` String, `x2` UInt32, `x3` UInt32, `x4` UInt32, `bm` AggregateFunction(groupBitmap, UInt32)) ENGINE = MergeTree PARTITION BY (ds, x1) ORDER BY (x2, x3, x4) SETTINGS index_granularity = 1 + + CREATE TABLE tbl (`ds` Date, `y1` UInt32, `x4` UInt32, `y2` UInt32, `y3` UInt32, `bm` AggregateFunction(groupBitmap, UInt32), `y4` UInt32 DEFAULT 0) ENGINE = MergeTree PARTITION BY (ds) ORDER BY (x4, y2, y3) SETTINGS index_granularity = 8192, max_parts_in_total = 10000000 + + insert into tbl0 with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, concat('xx',toString(rand()%10+1)) as x1, 1 as x2, 2 as x3, bitShiftRight(uid, 22) as x4, groupBitmapState(uid) as bm from numbers(100000000) where x4%40=0 group by ds, x1, x2, x3, x4 + + insert into tbl with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, rand()%1000+5000 as y1, bitShiftRight(uid, 22) as x4, rand()%100 as y2, rand()%2000 as y3, groupBitmapState(uid) as bm, rand()%1 as y4 from numbers(100000000) where x4%40=0 group by ds, y1, x4, y2, y3, y4 + + CREATE TABLE tmp_acc_hit engine Memory AS SELECT x1, x2, x3, arrayReduceInRanges('groupBitmapMergeState', [(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6), (1, 7)], bs) AS bs FROM (SELECT x1, x2, x3, groupArrayInsertAt(b, multiIf(ds = '2022-03-01', 0, ds = '2022-03-02', 1, ds = '2022-03-03', 2, ds = '2022-03-04', 3, ds = '2022-03-05', 4, ds = '2022-03-06', 5, ds = '2022-03-07', 6, 7)) AS bs FROM (SELECT x1, x2, x3, ds, groupBitmapOrState(bm) AS b FROM tbl0 WHERE ((ds >= '2022-03-01') AND (ds <= '2022-03-07')) AND (((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx2') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx3') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx4') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx5') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx6') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx7') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx8') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx9') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx10') AND (x2 = 1) AND (x3 = 2))) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) GROUP BY x1, x2, x3, ds) AS t_hit GROUP BY x1, x2, x3) + + WITH (SELECT groupArrayInsertAt(b, multiIf((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2), 0, (x1 = 'xx2') AND (x2 = 1) AND (x3 = 2), 1, (x1 = 'xx3') AND (x2 = 1) AND (x3 = 2), 2, (x1 = 'xx4') AND (x2 = 1) AND (x3 = 2), 3, (x1 = 'xx5') AND (x2 = 1) AND (x3 = 2), 4, (x1 = 'xx6') AND (x2 = 1) AND (x3 = 2), 5, (x1 = 'xx7') AND (x2 = 1) AND (x3 = 2), 6, (x1 = 'xx8') AND (x2 = 1) AND (x3 = 2), 7, (x1 = 'xx9') AND (x2 = 1) AND (x3 = 2), 8, (x1 = 'xx10') AND (x2 = 1) AND (x3 = 2), 9, 10)) FROM (SELECT x1, x2, x3, bs AS b FROM tmp_acc_hit)) AS bs SELECT y1, x4, toString(flat_arr) AS flat_arr, toString([bitmapAndCardinality(bmor1, (bs[1])[1]), bitmapAndCardinality(bmor2, (bs[1])[1]), bitmapAndCardinality(bmor3, (bs[1])[1]), bitmapAndCardinality(bmor1, (bs[2])[1]), bitmapAndCardinality(bmor2, (bs[2])[1]), bitmapAndCardinality(bmor3, (bs[2])[1]), bitmapAndCardinality(bmor1, (bs[3])[1]), bitmapAndCardinality(bmor2, (bs[3])[1]), bitmapAndCardinality(bmor3, (bs[3])[1]), bitmapAndCardinality(bmor1, (bs[4])[1]), bitmapAndCardinality(bmor2, (bs[4])[1]), bitmapAndCardinality(bmor3, (bs[4])[1]), bitmapAndCardinality(bmor1, (bs[5])[1]), bitmapAndCardinality(bmor2, (bs[5])[1]), bitmapAndCardinality(bmor3, (bs[5])[1]), bitmapAndCardinality(bmor1, (bs[6])[1]), bitmapAndCardinality(bmor2, (bs[6])[1]), bitmapAndCardinality(bmor3, (bs[6])[1]), bitmapAndCardinality(bmor1, (bs[7])[1]), bitmapAndCardinality(bmor2, (bs[7])[1]), bitmapAndCardinality(bmor3, (bs[7])[1]), bitmapAndCardinality(bmor1, (bs[8])[1]), bitmapAndCardinality(bmor2, (bs[8])[1]), bitmapAndCardinality(bmor3, (bs[8])[1]), bitmapAndCardinality(bmor1, (bs[9])[1]), bitmapAndCardinality(bmor2, (bs[9])[1]), bitmapAndCardinality(bmor3, (bs[9])[1]), bitmapAndCardinality(bmor1, (bs[10])[1]), bitmapAndCardinality(bmor2, (bs[10])[1]), bitmapAndCardinality(bmor3, (bs[10])[1])]) AS flat_arr_2 from (SELECT toString(y1) AS y1, toString(x4) AS x4, arrayFlatten(groupArrayInsertAt(flat_arr, multiIf(date_ = '2022-03-01', 0, 1))) AS flat_arr, groupBitmapOrState(bmor1) AS bmor1, groupBitmapOrState(bmor2) AS bmor2, groupBitmapOrState(bmor3) AS bmor3 FROM (WITH '2022-03-01' AS start_ds SELECT y1, x4, groupBitmapOrState(bm) AS bmor1, groupBitmapOrStateIf(bm, y2 > 0) AS bmor2, groupBitmapOrStateIf(bm, y4 = 1) AS bmor3, [sum(y2 * bitmapAndCardinality(bm, (bs[1])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[2])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[3])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[4])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[5])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[6])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[7])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[8])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[9])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[10])[1]))] AS flat_arr, start_ds AS date_ FROM tbl WHERE (ds = start_ds) AND (y1 IN (7063, 5010, 5006, 6788, 6176, 6203, 6769, 6555, 7062, 5119, 5007, 5212, 6814, 6177, 6789, 5095, 4942, 6243, 7061, 6744, 6201, 7196, 6181, 7195, 6178, 5004, 6790, 5008, 6877, 7281, 6791, 6179, 5214, 5005, 7146, 6980, 6322, 5222, 5217, 5137, 6561, 5133, 6937, 5142, 5130, 6885, 7250, 5103, 6867, 7066, 5096, 6868, 6199, 7269, 5131, 6414, 6884, 6560, 5136, 6883, 5158, 6869, 5097, 5132, 5102, 7251, 5219, 4695, 5220, 5202, 4203, 4204, 5098, 6870, 7064, 5101, 5105, 5140, 5135, 5139, 6880, 6194, 5218, 4202, 6655, 5104, 5183, 7245, 5100, 7065, 5099, 6938, 5138, 6881, 5134, 6886, 5141, 5129)) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) AND (y4 IN (0, 1)) GROUP BY y1, x4) GROUP BY y1, x4) LIMIT 1 + + DROP TABLE IF EXISTS tbl + DROP TABLE IF EXISTS tbl0 + DROP TABLE IF EXISTS tmp_acc_hit + From 670fad56fcdec83a1510003b306cb7ea6ae51d45 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Apr 2022 11:57:11 +0200 Subject: [PATCH 252/372] Print labels to log in run_check.py --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 6f00232be77..bda3cf40bba 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -255,6 +255,7 @@ if __name__ == "__main__": elif SUBMODULE_CHANGED_LABEL in pr_info.labels: pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL) + print("change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove)) if pr_labels_to_add: post_labels(gh, pr_info, pr_labels_to_add) From d3763c4a62590c19c4fa9525504edac591b49b2c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 5 Apr 2022 15:38:44 +0200 Subject: [PATCH 253/372] Update ClientBase.h --- programs/client/Client.cpp | 1 + programs/local/LocalServer.cpp | 8 +++++++- programs/local/LocalServer.h | 2 +- src/Client/ClientBase.cpp | 2 +- src/Client/ClientBase.h | 4 +++- 5 files changed, 13 insertions(+), 4 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c2094b3b00d..b1baf978d87 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1005,6 +1005,7 @@ void Client::processConfig() global_context->setCurrentQueryId(query_id); } print_stack_trace = config().getBool("stacktrace", false); + logging_initialized = true; if (config().has("multiquery")) is_multiquery = true; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1dfb5c1d636..18b62e65765 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -434,8 +434,11 @@ catch (...) return getCurrentExceptionCode(); } -void LocalServer::setLogger(const String & logs_level) +void LocalServer::updateLoggerLevel(const String & logs_level) { + if (!logging_initialized) + return; + config().setString("logger.level", logs_level); updateLevels(config(), logger()); } @@ -475,6 +478,7 @@ void LocalServer::processConfig() auto poco_logs_level = Poco::Logger::parseLevel(level); Poco::Logger::root().setLevel(poco_logs_level); Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::SimpleFileChannel(server_logs_file))); + logging_initialized = true; } else if (logging || is_interactive) { @@ -482,11 +486,13 @@ void LocalServer::processConfig() auto log_level_default = is_interactive && !logging ? "none" : level; config().setString("logger.level", config().getString("log-level", config().getString("send_logs_level", log_level_default))); buildLoggers(config(), logger(), "clickhouse-local"); + logging_initialized = true; } else { Poco::Logger::root().setLevel("none"); Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); + logging_initialized = false; } shared_context = Context::createShared(); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 3ee6d80136e..e96fb211554 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -46,7 +46,7 @@ protected: void processConfig() override; - void setLogger(const String & logs_level) override; + void updateLoggerLevel(const String & logs_level) override; private: /** Composes CREATE subquery based on passed arguments (--structure --file --table and --input-format) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7f5b5b82082..568205c6dfe 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1302,7 +1302,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin { const auto * logs_level_field = set_query->changes.tryGet(std::string_view{"send_logs_level"}); if (logs_level_field) - setLogger(logs_level_field->safeGet()); + updateLoggerLevel(logs_level_field->safeGet()); } processed_rows = 0; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 1ce8be94d39..6846fa247e8 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -95,7 +95,7 @@ protected: std::optional hosts_and_ports_description; }; - virtual void setLogger(const String &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `initializeLogger()` is not implemented for `{}`", getName()); } + virtual void updateLoggerLevel(const String &) {} virtual void printHelpMessage(const OptionsDescription & options_description) = 0; virtual void addOptions(OptionsDescription & options_description) = 0; virtual void processOptions(const OptionsDescription & options_description, @@ -266,6 +266,8 @@ protected: bool allow_repeated_settings = false; bool cancelled = false; + + bool logging_initialized = false; }; } From 0b0c8ef09e1409bceae44c134c2fbc8eee6044c8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 18:47:34 +0800 Subject: [PATCH 254/372] add integration tests --- src/Storages/Hive/StorageHive.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a7537f5a1f6..6c31a959269 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -176,7 +176,8 @@ public: /// Use local cache for remote storage if enabled. std::unique_ptr remote_read_buf; - if (ExternalDataSourceCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) + if (ExternalDataSourceCache::instance().isInitialized() + && getContext()->getSettingsRef().use_local_cache_for_remote_storage) { size_t buff_size = raw_read_buf->internalBuffer().size(); if (buff_size == 0) @@ -547,7 +548,8 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_TRACE(log, "Skip hive file {} by index {}", hive_file->getPath(), hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); + LOG_TRACE( + log, "Skip hive file {} by index {}", hive_file->getPath(), hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); return {}; } } @@ -563,7 +565,12 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); + LOG_TRACE( + log, + "Skip split {} of hive file {} by index {}", + i, + hive_file->getPath(), + hive_file->describeMinMaxIndex(sub_minmax_idxes[i])); skip_splits.insert(i); } } From 5d8a1a4465e766d814630af3f2a2a87d32a6fdc4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 18:50:39 +0800 Subject: [PATCH 255/372] add integration tests --- tests/integration/test_hive_query/test.py | 87 +++++++++++++++++++++++ 1 file changed, 87 insertions(+) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 9e9a20fa6d1..6a8e152a4bd 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -149,6 +149,93 @@ def test_orc_groupby(started_cluster): assert result == expected_result +@pytest.mark.parametrize( + "table,use_local_cache_for_remote_storage,enable_orc_file_minmax_index,enable_orc_stripe_minmax_index", + [ + pytest.param( + "demo_orc_no_cache_no_index", + "false", + "false", + "false", + id="demo_orc_no_cache_no_index", + ), + pytest.param( + "demo_orc_with_cache_no_index", + "true", + "false", + "false", + id="demo_orc_with_cache_no_index", + ), + pytest.param( + "demo_orc_no_cache_file_index", + "false", + "true", + "false", + id="demo_orc_no_cache_file_index", + ), + pytest.param( + "demo_orc_with_cache_file_index", + "true", + "true", + "false", + id="demo_orc_with_cache_file_index", + ), + pytest.param( + "demo_orc_no_cache_stripe_index", + "false", + "true", + "true", + id="demo_orc_no_cache_stripe_index", + ), + pytest.param( + "demo_orc_with_cache_stripe_index", + "true", + "true", + "true", + id="demo_orc_with_cache_stripe_index", + ), + ], +) +def test_orc_minmax_index( + started_cluster, + table, + use_local_cache_for_remote_storage, + enable_orc_file_minmax_index, + enable_orc_stripe_minmax_index, +): + node = started_cluster.instances["h0_0_0"] + result = node.query( + """ + DROP TABLE IF EXISTS default.{table}; + CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) + SETTINGS enable_orc_file_minmax_index = {enable_orc_file_minmax_index}, enable_orc_stripe_minmax_index = {enable_orc_stripe_minmax_index}; + """.format( + table=table, + enable_orc_file_minmax_index=enable_orc_file_minmax_index, + enable_orc_stripe_minmax_index=enable_orc_stripe_minmax_index, + ) + ) + assert result.strip() == "" + + for i in range(2): + result = node.query( + """ + SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id + SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} + """.format( + table=table, + use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, + ) + ) + + assert ( + result.strip() + == """2021-11-05 abd 15 +2021-11-16 aaa 22 +""" + ) + + def test_hive_columns_prunning(started_cluster): logging.info("Start testing groupby ...") node = started_cluster.instances["h0_0_0"] From acbd11ccbcae8900a5cec43bc0a0d4147e5a34e4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 18:59:17 +0800 Subject: [PATCH 256/372] add integration tests --- tests/integration/test_hive_query/test.py | 67 +++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 6a8e152a4bd..e1aa811c1a2 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -236,6 +236,73 @@ def test_orc_minmax_index( ) +@pytest.mark.parametrize( + "table,use_local_cache_for_remote_storage,enable_parquet_rowgroup_minmax_index", + [ + pytest.param( + "demo_parquet_no_cache_no_index", + "false", + "false", + id="demo_parquet_no_cache_no_index", + ), + pytest.param( + "demo_parquet_with_cache_no_index", + "true", + "false", + id="demo_parquet_with_cache_no_index", + ), + pytest.param( + "demo_parquet_no_cache_rowgroup_index", + "false", + "true", + id="demo_parquet_no_cache_rowgroup_index", + ), + pytest.param( + "demo_parquet_with_cache_rowgroup_index", + "true", + "true", + id="demo_parquet_with_cache_rowgroup_index", + ), + ], +) +def test_parquet_minmax_index( + started_cluster, + table, + use_local_cache_for_remote_storage, + enable_parquet_rowgroup_minmax_index, +): + node = started_cluster.instances["h0_0_0"] + result = node.query( + """ + DROP TABLE IF EXISTS default.{table}; + CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + SETTINGS enable_parquet_rowgroup_minmax_index = {enable_parquet_rowgroup_minmax_index} + """.format( + table=table, + enable_parquet_rowgroup_minmax_index=enable_parquet_rowgroup_minmax_index, + ) + ) + assert result.strip() == "" + + for i in range(2): + result = node.query( + """ + SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id + SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} + """.format( + table=table, + use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, + ) + ) + + assert ( + result.strip() + == """2021-11-05 abd 15 +2021-11-16 aaa 22 +""" + ) + + def test_hive_columns_prunning(started_cluster): logging.info("Start testing groupby ...") node = started_cluster.instances["h0_0_0"] From 058cde820fd06f76a0f0fcca7782540fa17d745f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Apr 2022 09:48:18 +0300 Subject: [PATCH 257/372] test/stress: improve error checks for backward compatiblity check Before the log was analyzed only after the server had been restarted after stressing, and hence it cannot find anything in case of error, like here [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/34355/acd48ebe2d51d20d0c2a5bc545d46d27f9bf7904/stress_test__thread__actions_.html Signed-off-by: Azat Khuzhin --- docker/test/stress/run.sh | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index e56afcbfd7a..1ca49b5efaf 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -289,7 +289,7 @@ then install_packages package_folder mkdir tmp_stress_output - + ./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \ && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv @@ -297,8 +297,9 @@ then clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" - stop - + stop + mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log + # Start new server configure start 500 @@ -310,8 +311,9 @@ then # Let the server run for a while before checking log. sleep 60 - + stop + mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.clean.log # Error messages (we should ignore some errors) echo "Check for Error messages in server log:" @@ -332,7 +334,7 @@ then -e "Code: 1000, e.code() = 111, Connection refused" \ -e "UNFINISHED" \ -e "Renaming unexpected part" \ - /var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ + /var/log/clickhouse-server/clickhouse-server.backward.*.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv @@ -348,13 +350,13 @@ then rm -f /test_output/tmp # OOM - zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \ + zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Logical errors echo "Check for Logical errors in server log:" - zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_logical_errors.txt \ + zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \ && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv @@ -362,19 +364,18 @@ then [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt # Crash - zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \ + zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" - zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_fatal_messages.txt \ + zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_fatal_messages.txt if it's empty [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt - else echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv fi From 2fb6d6acb4ff12488ce18fbb9c6687802edfa3e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Apr 2022 09:52:45 +0300 Subject: [PATCH 258/372] stress: use more meaningful names for clickhouse-server.log But note, that stderr.log/stdout.log/clickhouse-server.err.log still not separated. Signed-off-by: Azat Khuzhin --- docker/test/stress/run.sh | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 1ca49b5efaf..ba85999caa5 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -106,17 +106,6 @@ function stop() function start() { - # Rename existing log file - it will be more convenient to read separate files for separate server runs. - if [ -f '/var/log/clickhouse-server/clickhouse-server.log' ] - then - log_file_counter=1 - while [ -f "/var/log/clickhouse-server/clickhouse-server.log.${log_file_counter}" ] - do - log_file_counter=$((log_file_counter + 1)) - done - mv '/var/log/clickhouse-server/clickhouse-server.log' "/var/log/clickhouse-server/clickhouse-server.log.${log_file_counter}" - fi - counter=0 until clickhouse-client --query "SELECT 1" do @@ -190,6 +179,8 @@ clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordin clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" stop +mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log + start clickhouse-client --query "SHOW TABLES FROM datasets" @@ -205,6 +196,8 @@ clickhouse-client --query "SHOW TABLES FROM test" || echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv stop +mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.stress.log + start clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_output/test_results.tsv \ @@ -263,10 +256,12 @@ mkdir previous_release_package_folder clickhouse-client --query="SELECT version()" | ./download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \ || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv +stop +mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log + if [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] then echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv - stop # Uninstall current packages dpkg --remove clickhouse-client From 4cb7b7e49baadbaabb23f46fd0f45bce2d12b0a9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Apr 2022 11:15:37 +0000 Subject: [PATCH 259/372] Create parent directories in DiskLocal::replaceFile --- src/Disks/DiskLocal.cpp | 1 + ...265_rename_join_ordinary_to_atomic.reference | 1 + .../02265_rename_join_ordinary_to_atomic.sql | 17 +++++++++++++++++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.reference create mode 100644 tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.sql diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 44fdbb77323..a91db508295 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -333,6 +333,7 @@ void DiskLocal::replaceFile(const String & from_path, const String & to_path) { fs::path from_file = fs::path(disk_path) / from_path; fs::path to_file = fs::path(disk_path) / to_path; + fs::create_directories(to_file.parent_path()); fs::rename(from_file, to_file); } diff --git a/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.reference b/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.reference new file mode 100644 index 00000000000..58c9bdf9d01 --- /dev/null +++ b/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.reference @@ -0,0 +1 @@ +111 diff --git a/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.sql b/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.sql new file mode 100644 index 00000000000..3ec995a6a24 --- /dev/null +++ b/tests/queries/0_stateless/02265_rename_join_ordinary_to_atomic.sql @@ -0,0 +1,17 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS 02265_atomic_db; +DROP DATABASE IF EXISTS 02265_ordinary_db; + +CREATE DATABASE 02265_atomic_db ENGINE = Atomic; +CREATE DATABASE 02265_ordinary_db ENGINE = Ordinary; + +CREATE TABLE 02265_ordinary_db.join_table ( `a` Int64 ) ENGINE = Join(`ALL`, LEFT, a); +INSERT INTO 02265_ordinary_db.join_table VALUES (111); + +RENAME TABLE 02265_ordinary_db.join_table TO 02265_atomic_db.join_table; + +SELECT * FROM 02265_atomic_db.join_table; + +DROP DATABASE IF EXISTS 02265_atomic_db; +DROP DATABASE IF EXISTS 02265_ordinary_db; From 1309e781b624fdfa5c5886eb25f5f2ee52c5fe89 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 6 Apr 2022 13:56:26 +0200 Subject: [PATCH 260/372] apply suggestion --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1127337adff..39840f91325 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -969,7 +969,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper static constexpr std::array flat_nodes = {"block_numbers", "blocks", "leader_election", "log", "mutations", "pinned_part_uuids"}; /// First try to remove paths that are known to be flat - for (const auto & node : flat_nodes) + for (const auto * node : flat_nodes) { bool removed_quickly = zookeeper->tryRemoveChildrenRecursive(fs::path(zookeeper_path) / node, /* probably flat */ true); if (!removed_quickly) @@ -979,7 +979,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper /// Then try to remove nodes that are known to have no children (and should always exist) Coordination::Requests ops; - for (const auto & node : flat_nodes) + for (const auto * node : flat_nodes) ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/" + node, -1)); ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/alter_partition_version", -1)); From 53628092e2ae20d6f4054344438cda041aa1dfde Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Apr 2022 20:04:13 +0800 Subject: [PATCH 261/372] Fix test --- .../queries/0_stateless/02174_cte_scalar_cache_mv.reference | 2 +- tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference index 246706164df..055c88160ad 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference +++ b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference @@ -18,7 +18,7 @@ 89 89 89 89 5 94 94 94 94 5 99 99 99 99 5 -02177_MV 7 80 22 +02177_MV 3 80 26 10 40 70 diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql index 4d4447c7f31..742d72fe2b2 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql @@ -39,13 +39,13 @@ SYSTEM FLUSH LOGS; -- The main query should have a cache miss and 3 global hits -- The MV is executed 20 times (100 / 5) and each run does 1 miss and 4 hits to the LOCAL cache -- In addition to this, to prepare the MV, there is an extra preparation to get the list of columns via --- InterpreterSelectQuery, which adds 1 miss and 4 global hits (since it uses the global cache) +-- InterpreterSelectQuery, which adds 5 miss (since we don't use cache for preparation) -- So in total we have: -- Main query: 1 miss, 3 global --- Preparation: 1 miss, 4 global +-- Preparation: 5 miss -- Blocks (20): 20 miss, 0 global, 80 local hits --- TOTAL: 22 miss, 7 global, 80 local +-- TOTAL: 26 miss, 3 global, 80 local SELECT '02177_MV', ProfileEvents['ScalarSubqueriesGlobalCacheHit'] as scalar_cache_global_hit, From e2044b65049aa343268dee72f5cfbfe386e05edb Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Wed, 6 Apr 2022 16:17:22 +0400 Subject: [PATCH 262/372] Set ENABLE_BUILD_PATH_MAPPING to OFF by default, if CMAKE_BUILD_TYPE is set to Debug --- CMakeLists.txt | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index a9ce64b87ba..d893ba773cc 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -294,14 +294,19 @@ include(cmake/cpu_features.cmake) # Enable it explicitly. set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables") -# Reproducible builds -# If turned `ON`, remap file source paths in debug info, predefined preprocessor macros and __builtin_FILE(). -option(ENABLE_BUILD_PATH_MAPPING "Enable remap file source paths in debug info, predefined preprocessor macros and __builtin_FILE(). It's to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ON) +# Reproducible builds. +if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + set (ENABLE_BUILD_PATH_MAPPING_DEFAULT OFF) +else () + set (ENABLE_BUILD_PATH_MAPPING_DEFAULT ON) +endif () + +option (ENABLE_BUILD_PATH_MAPPING "Enable remapping of file source paths in debug info, predefined preprocessor macros, and __builtin_FILE(). It's used to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ${ENABLE_BUILD_PATH_MAPPING_DEFAULT}) if (ENABLE_BUILD_PATH_MAPPING) set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") -endif() +endif () if (${CMAKE_VERSION} VERSION_LESS "3.12.4") # CMake < 3.12 doesn't support setting 20 as a C++ standard version. From 846faa51d808ab4bbd92176a6f857916d6ec044b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Apr 2022 14:35:19 +0200 Subject: [PATCH 263/372] Fix bug --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d12e91f62e4..a126478857b 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -246,8 +246,9 @@ void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPt return; replication_handler->shutdownFinal(); + replication_handler.reset(); - auto nested_table = getNested(); + auto nested_table = tryGetNested() != nullptr; if (nested_table) InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), no_delay); } From 59e47472f182f9541af5048b7da29f52d61c7e0e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 6 Apr 2022 20:51:31 +0800 Subject: [PATCH 264/372] fix integration tests --- tests/integration/test_hive_query/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index e1aa811c1a2..374a86d51e8 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -229,7 +229,7 @@ def test_orc_minmax_index( ) assert ( - result.strip() + result == """2021-11-05 abd 15 2021-11-16 aaa 22 """ @@ -296,7 +296,7 @@ def test_parquet_minmax_index( ) assert ( - result.strip() + result == """2021-11-05 abd 15 2021-11-16 aaa 22 """ From 19e0e6e7652101dbf3a14137c25254adc0147c69 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 14:27:29 +0200 Subject: [PATCH 265/372] Fix failing BuilderReport --- .github/workflows/backport_branches.yml | 11 ++++++++--- .github/workflows/master.yml | 12 +++++++++--- .github/workflows/pull_request.yml | 12 +++++++++--- .github/workflows/release_branches.yml | 12 +++++++++--- tests/ci/build_report_check.py | 4 +++- 5 files changed, 38 insertions(+), 13 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 75f8a63368d..417284f14d5 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -341,10 +341,15 @@ jobs: steps: - name: Set envs run: | + DEPENDENCIES=$(cat << 'EOF' | jq '. | length' + ${{ toJSON(needs) }} + EOF + ) + echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV" cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -360,7 +365,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" + python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES" - name: Cleanup if: always() run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 081fa165c68..eab7ce36eb7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -992,10 +992,16 @@ jobs: steps: - name: Set envs run: | + DEPENDENCIES=$(cat << 'EOF' | jq '. | length' + ${{ toJSON(needs) }} + EOF + ) + echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV" cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1011,7 +1017,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" + python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES" - name: Cleanup if: always() run: | diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c01d1821d0f..8942cca391e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1044,10 +1044,16 @@ jobs: steps: - name: Set envs run: | + DEPENDENCIES=$(cat << 'EOF' | jq '. | length' + ${{ toJSON(needs) }} + EOF + ) + echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV" cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1063,7 +1069,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" + python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES" - name: Cleanup if: always() run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index d916699acc2..b2af465142b 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -436,10 +436,16 @@ jobs: steps: - name: Set envs run: | + DEPENDENCIES=$(cat << 'EOF' | jq '. | length' + ${{ toJSON(needs) }} + EOF + ) + echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV" cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -455,7 +461,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" + python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES" - name: Cleanup if: always() run: | diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 4e0859ef865..dc3126d32c3 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -120,6 +120,7 @@ if __name__ == "__main__": os.makedirs(temp_path) build_check_name = sys.argv[1] + reports_length = int(sys.argv[2]) if len(sys.argv) > 2 else 0 gh = Github(get_best_robot_token()) pr_info = PRInfo() @@ -148,7 +149,8 @@ if __name__ == "__main__": build_name, ) - some_builds_are_missing = len(build_reports_map) < len(reports_order) + reports_length = reports_length or len(reports_order) + some_builds_are_missing = len(build_reports_map) < reports_length if some_builds_are_missing: logging.info( From 1323c3617f70b1d79e4f92415b181effb3eb812a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 14:34:20 +0200 Subject: [PATCH 266/372] Fix linter points --- tests/ci/build_report_check.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index dc3126d32c3..f29c6ced122 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -139,7 +139,7 @@ if __name__ == "__main__": logging.info("Found build report json %s", f) build_name = get_build_name_from_file_name(f) if build_name in reports_order: - with open(os.path.join(root, f), "r") as file_handler: + with open(os.path.join(root, f), "rb") as file_handler: build_report = json.load(file_handler) build_reports_map[build_name] = build_report else: @@ -190,7 +190,7 @@ if __name__ == "__main__": branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" if pr_info.number != 0: - branch_name = "PR #{}".format(pr_info.number) + branch_name = f"PR #{pr_info.number}" branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/pull/{pr_info.number}" commit_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commit/{pr_info.sha}" task_url = GITHUB_RUN_URL @@ -206,8 +206,8 @@ if __name__ == "__main__": ) report_path = os.path.join(temp_path, "report.html") - with open(report_path, "w") as f: - f.write(report) + with open(report_path, "w", encoding="utf-8") as fd: + fd.write(report) logging.info("Going to upload prepared report") context_name_for_path = build_check_name.lower().replace(" ", "_") @@ -241,7 +241,7 @@ if __name__ == "__main__": description = f"{ok_builds}/{total_builds} builds are OK {addition}" - print("::notice ::Report url: {}".format(url)) + print(f"::notice ::Report url: {url}") commit = get_commit(gh, pr_info.sha) commit.create_status( From 12442fd0a5bd915c9bbe9fd602c0fe699c76cf43 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 14:39:23 +0200 Subject: [PATCH 267/372] Add description for missing builds --- tests/ci/build_report_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index f29c6ced122..7ca958bd745 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -237,7 +237,7 @@ if __name__ == "__main__": addition = "" if some_builds_are_missing: - addition = "(some builds are missing)" + addition = f"({len(build_reports_map)} < {reports_length})" description = f"{ok_builds}/{total_builds} builds are OK {addition}" From 0d0b1ef7dc961877054f709633428a9c70a0b5c4 Mon Sep 17 00:00:00 2001 From: Meena Renganathan Date: Wed, 6 Apr 2022 06:28:35 -0700 Subject: [PATCH 268/372] Fix to restrict the curl, azure and sentry build in MacOS --- CMakeLists.txt | 6 ++++++ contrib/CMakeLists.txt | 10 +++++++--- contrib/curl-cmake/CMakeLists.txt | 19 +------------------ contrib/curl-cmake/curl_config.h | 3 --- 4 files changed, 14 insertions(+), 24 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9649fc32d74..fa94e1c6782 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -222,6 +222,12 @@ else () set(NO_WHOLE_ARCHIVE --no-whole-archive) endif () +option(ENABLE_CURL_BUILD "Enable curl, azure, sentry build on by default except MacOS." ON) +if (OS_DARWIN) + # Disable the curl, azure, senry build on MacOS + set (ENABLE_CURL_BUILD OFF) +endif () + # Ignored if `lld` is used option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold linker.") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 9cf307c473e..1f03c0fd341 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -119,9 +119,13 @@ add_contrib (fastops-cmake fastops) add_contrib (libuv-cmake libuv) add_contrib (amqpcpp-cmake AMQP-CPP) # requires: libuv add_contrib (cassandra-cmake cassandra) # requires: libuv -add_contrib (curl-cmake curl) -add_contrib (azure-cmake azure) -add_contrib (sentry-native-cmake sentry-native) # requires: curl + +if (ENABLE_CURL_BUILD) + add_contrib (curl-cmake curl) + add_contrib (azure-cmake azure) + add_contrib (sentry-native-cmake sentry-native) # requires: curl +endif() + add_contrib (fmtlib-cmake fmtlib) add_contrib (krb5-cmake krb5) add_contrib (cyrus-sasl-cmake cyrus-sasl) # for krb5 diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index fdd73612549..b1e1a0ded8a 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -153,22 +153,6 @@ set (SRCS add_library (_curl ${SRCS}) -set(CURL_LIBS "") -if(APPLE) - - find_library(SYSTEMCONFIGURATION_FRAMEWORK "SystemConfiguration") - if(NOT SYSTEMCONFIGURATION_FRAMEWORK) - message(FATAL_ERROR "SystemConfiguration framework not found") - endif() - set(CURL_LIBS ${CURL_LIBS} "-framework SystemConfiguration") - - find_library(COREFOUNDATION_FRAMEWORK "CoreFoundation") - if(NOT COREFOUNDATION_FRAMEWORK) - message(FATAL_ERROR "CoreFoundation framework not found") - endif() - set(CURL_LIBS ${CURL_LIBS} "-framework CoreFoundation") -endif() - target_compile_definitions (_curl PRIVATE HAVE_CONFIG_H BUILDING_LIBCURL @@ -182,8 +166,7 @@ target_include_directories (_curl SYSTEM PUBLIC . # curl_config.h ) -set(CURL_LIBS ${CURL_LIBS} OpenSSL::SSL) -target_link_libraries (_curl PRIVATE ${CURL_LIBS}) +target_link_libraries (_curl PRIVATE OpenSSL::SSL) # The library is large - avoid bloat (XXX: is it?) target_compile_options (_curl PRIVATE -g0) diff --git a/contrib/curl-cmake/curl_config.h b/contrib/curl-cmake/curl_config.h index 268b2a98287..1efdd88600f 100644 --- a/contrib/curl-cmake/curl_config.h +++ b/contrib/curl-cmake/curl_config.h @@ -48,6 +48,3 @@ #define ENABLE_IPV6 #define USE_OPENSSL #define USE_THREADS_POSIX - -#define HAVE_BOOL_T 1 -#define HAVE_STDBOOL_H 1 From 57ddb18f861c683949b174ac5cb37f4396ac7bc7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Apr 2022 16:04:55 +0200 Subject: [PATCH 269/372] Make test 00159_parallel_formatting_tsv_and_friends.sh more stable --- .../1_stateful/00159_parallel_formatting_tsv_and_friends.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00159_parallel_formatting_tsv_and_friends.sh b/tests/queries/1_stateful/00159_parallel_formatting_tsv_and_friends.sh index 9d48774dd2d..02441190b91 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_tsv_and_friends.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_tsv_and_friends.sh @@ -10,10 +10,10 @@ FORMATS=('TSV' 'TSVWithNames' 'TSKV') for format in "${FORMATS[@]}" do echo "$format, false"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=false -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum echo "$format, true"; - $CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \ + $CLICKHOUSE_CLIENT --max_threads=0 --output_format_parallel_formatting=true -q \ "SELECT ClientEventTime::DateTime('Asia/Dubai') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum done From 1cbacbc3e61872fce5c14a64f6b80c3404bda2d6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Apr 2022 14:28:30 +0000 Subject: [PATCH 270/372] Update version_date.tsv after v22.3.3.44-lts --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e87c4ea2b46..6366aef19ce 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.3.3.44-lts 2022-04-06 v22.3.2.2-lts 2022-03-17 v22.2.3.5-stable 2022-02-25 v22.2.2.1-stable 2022-02-17 From 00137eaa342cb31296701a70c0e1e8ebd7dd181c Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 6 Apr 2022 19:13:56 +0300 Subject: [PATCH 271/372] Update test.py --- tests/integration/test_system_merges/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 9239cb11065..775706f4df6 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -124,7 +124,7 @@ def test_merge_simple(started_cluster, replicated): assert ( node_check.query( - "SELECT * FROM system.merges WHERE table = '{name}'".format( + "SELECT * FROM system.merges WHERE table = '{name}' and progress < 1".format( name=table_name ) ) From 5af0537f2cc5da4ee27d659b6c9896810757e22c Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 6 Apr 2022 19:23:33 +0300 Subject: [PATCH 272/372] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index de36fc3da27..9ca820652d9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -371,7 +371,7 @@ class SettingsRandomizer: "group_by_two_level_threshold_bytes": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 50000000, "distributed_aggregation_memory_efficient": lambda: random.randint(0, 1), "fsync_metadata": lambda: random.randint(0, 1), - "priority": lambda: int(abs(random.gauss(0, 2))), + "priority": lambda: min(3, int(abs(random.gauss(0, 2)))), "output_format_parallel_formatting": lambda: random.randint(0, 1), "input_format_parallel_parsing": lambda: random.randint(0, 1), "min_chunk_bytes_for_parallel_parsing": lambda: max(1024, int(random.gauss(10 * 1024 * 1024, 5 * 1000 * 1000))), From 14b860f9bbf9408e821a70cd37e44d295e08b3d3 Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 12:39:52 -0400 Subject: [PATCH 273/372] disallow the use of known weak implementations in requirements --- .../requirements/requirements.md | 33 ++-------- .../requirements/requirements.py | 65 +++---------------- 2 files changed, 12 insertions(+), 86 deletions(-) diff --git a/tests/testflows/aes_encryption/requirements/requirements.md b/tests/testflows/aes_encryption/requirements/requirements.md index 80cb614268c..23906f797d0 100644 --- a/tests/testflows/aes_encryption/requirements/requirements.md +++ b/tests/testflows/aes_encryption/requirements/requirements.md @@ -311,7 +311,7 @@ version: 1.0 of the `encrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid @@ -327,9 +327,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `encrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -403,9 +400,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -476,7 +470,7 @@ version: 1.0 of the `decrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid @@ -492,9 +486,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `decrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -570,9 +561,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -644,7 +632,7 @@ version: 1.0 of the `aes_encrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -659,9 +647,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_encrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -750,9 +735,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -810,7 +792,7 @@ version: 1.0 of the `aes_decrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -825,9 +807,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_decrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -916,9 +895,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -954,7 +930,6 @@ version: 1.0 [GCM]: https://en.wikipedia.org/wiki/Galois/Counter_Mode [CTR]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_(CTR) [CBC]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_block_chaining_(CBC) -[ECB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Electronic_codebook_(ECB) [CFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [CFB128]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [OFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Output_feedback_(OFB) diff --git a/tests/testflows/aes_encryption/requirements/requirements.py b/tests/testflows/aes_encryption/requirements/requirements.py index 0fbbea7e85a..4523f2d820f 100644 --- a/tests/testflows/aes_encryption/requirements/requirements.py +++ b/tests/testflows/aes_encryption/requirements/requirements.py @@ -429,7 +429,7 @@ RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `encrypt` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB] as well as\n" "[CTR] and [GCM] as the values. For example, `aes-256-ofb`.\n" "\n" ), @@ -467,9 +467,6 @@ RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `encrypt` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -642,9 +639,6 @@ RQ_SRS008_AES_Encrypt_Function_NonGCMMode_KeyAndInitializationVector_Length = Re "[ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values\n" "when using non-GCM modes\n" "\n" - "* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified\n" - "* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified\n" - "* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified\n" "* `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" @@ -790,7 +784,7 @@ RQ_SRS008_AES_Decrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `decrypt` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB] as well as\n" "[CTR] and [GCM] as the values. For example, `aes-256-ofb`.\n" "\n" ), @@ -828,9 +822,6 @@ RQ_SRS008_AES_Decrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `decrypt` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1005,9 +996,6 @@ RQ_SRS008_AES_Decrypt_Function_NonGCMMode_KeyAndInitializationVector_Length = Re "[ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values\n" "when using non-GCM modes\n" "\n" - "* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified\n" - "* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified\n" - "* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified\n" "* `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" @@ -1154,7 +1142,7 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `aes_encrypt_mysql` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" "\n" ), link=None, @@ -1191,9 +1179,6 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `aes_encrypt_mysql` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1392,9 +1377,6 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Mode_KeyAndInitializationVector_Length = Re description=( "[ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values\n" "\n" - "* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified\n" - "* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified\n" - "* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified\n" "* `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes\n" @@ -1516,7 +1498,7 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `aes_decrypt_mysql` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" "\n" ), link=None, @@ -1553,9 +1535,6 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `aes_decrypt_mysql` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1754,9 +1733,6 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Mode_KeyAndInitializationVector_Length = Re description=( "[ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values\n" "\n" - "* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified\n" - "* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified\n" - "* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified\n" "* `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes\n" @@ -2606,7 +2582,7 @@ version: 1.0 of the `encrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid @@ -2622,9 +2598,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `encrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -2698,9 +2671,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -2771,7 +2741,7 @@ version: 1.0 of the `decrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid @@ -2787,9 +2757,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `decrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -2865,9 +2832,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -2939,7 +2903,7 @@ version: 1.0 of the `aes_encrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -2954,9 +2918,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_encrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -3045,9 +3006,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -3105,7 +3063,7 @@ version: 1.0 of the `aes_decrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -3120,9 +3078,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_decrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -3211,9 +3166,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -3249,7 +3201,6 @@ version: 1.0 [GCM]: https://en.wikipedia.org/wiki/Galois/Counter_Mode [CTR]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_(CTR) [CBC]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_block_chaining_(CBC) -[ECB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Electronic_codebook_(ECB) [CFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [CFB128]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [OFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Output_feedback_(OFB) From 33ea119013271b9cb8b81f86d3c5c198d5d2dc2e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Apr 2022 00:56:10 +0800 Subject: [PATCH 274/372] Bump replxx to fix wrong assertion --- contrib/replxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/replxx b/contrib/replxx index 6f0b6f151ae..3fd0e3c9364 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 6f0b6f151ae2a044625ae93acd19ca365fcea64d +Subproject commit 3fd0e3c9364a589447453d9906d854ebd8d385c5 From 45b284d09820a21885e67cc7e242da40518913ed Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 6 Apr 2022 20:22:24 +0300 Subject: [PATCH 275/372] Update 02248_nullable_custom_types_to_string.sql --- .../0_stateless/02248_nullable_custom_types_to_string.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql index b6032f7741b..605500ee840 100644 --- a/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql +++ b/tests/queries/0_stateless/02248_nullable_custom_types_to_string.sql @@ -1,4 +1,4 @@ --- Tags: no-backward-compatibility-check:22.3.2.2 +-- Tags: no-backward-compatibility-check:22.3.4.44 select toString(toNullable(true)); select toString(CAST(NULL, 'Nullable(Bool)')); select toString(toNullable(toIPv4('0.0.0.0'))); From b3f59537ca3e201b5dc47cc4625f74aff46aa83f Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 13:34:20 -0400 Subject: [PATCH 276/372] added additional check --- .../external_user_directory/tests/common.py | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/common.py b/tests/testflows/ldap/external_user_directory/tests/common.py index 871be815a35..7400459d266 100644 --- a/tests/testflows/ldap/external_user_directory/tests/common.py +++ b/tests/testflows/ldap/external_user_directory/tests/common.py @@ -84,14 +84,25 @@ def rbac_roles(*roles, node=None): node.query(f"DROP ROLE IF EXISTS {role}") -def verify_ldap_user_exists(server, username, password): +# def verify_ldap_user_exists(server, username, password): +# """Check that LDAP user is defined on the LDAP server.""" +# with By("searching LDAP database"): +# ldap_node = current().context.cluster.node(server) +# r = ldap_node.command( +# f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" +# ) +# assert r.exitcode == 0, error() + + +def verify_ldap_user_exists(server, username, password, check=False): """Check that LDAP user is defined on the LDAP server.""" - with By("searching LDAP database"): - ldap_node = current().context.cluster.node(server) - r = ldap_node.command( - f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" - ) - assert r.exitcode == 0, error() + if check: + with By("searching LDAP database"): + ldap_node = current().context.cluster.node(server) + r = ldap_node.command( + f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" + ) + assert r.exitcode == 0, error() def create_ldap_external_user_directory_config_content( From 87e0656c3b320e7f8200fb56490c161acbd9b134 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Apr 2022 19:44:37 +0200 Subject: [PATCH 277/372] Fix flaky test 00155_long_merges --- tests/queries/0_stateless/00155_long_merges.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00155_long_merges.sh b/tests/queries/0_stateless/00155_long_merges.sh index f2d9cd1dade..15ad0892a42 100755 --- a/tests/queries/0_stateless/00155_long_merges.sh +++ b/tests/queries/0_stateless/00155_long_merges.sh @@ -32,7 +32,7 @@ function test { SUM=$(( $1 + $2 )) MAX=$(( $1 > $2 ? $1 : $2 )) - SETTINGS="--min_insert_block_size_rows=0 --min_insert_block_size_bytes=0" + SETTINGS="--min_insert_block_size_rows=0 --min_insert_block_size_bytes=0 --max_block_size=65505" $CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO summing_00155 (x) SELECT number AS x FROM system.numbers LIMIT $1" $CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO summing_00155 (x) SELECT number AS x FROM system.numbers LIMIT $2" From 68bdcdca80cefe8af0479c2e9d6516ac289f6c00 Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 14:15:30 -0400 Subject: [PATCH 278/372] code fix for Ldap Injection --- .../external_user_directory/tests/common.py | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/common.py b/tests/testflows/ldap/external_user_directory/tests/common.py index 7400459d266..c0b6e72cd8e 100644 --- a/tests/testflows/ldap/external_user_directory/tests/common.py +++ b/tests/testflows/ldap/external_user_directory/tests/common.py @@ -84,27 +84,6 @@ def rbac_roles(*roles, node=None): node.query(f"DROP ROLE IF EXISTS {role}") -# def verify_ldap_user_exists(server, username, password): -# """Check that LDAP user is defined on the LDAP server.""" -# with By("searching LDAP database"): -# ldap_node = current().context.cluster.node(server) -# r = ldap_node.command( -# f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" -# ) -# assert r.exitcode == 0, error() - - -def verify_ldap_user_exists(server, username, password, check=False): - """Check that LDAP user is defined on the LDAP server.""" - if check: - with By("searching LDAP database"): - ldap_node = current().context.cluster.node(server) - r = ldap_node.command( - f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" - ) - assert r.exitcode == 0, error() - - def create_ldap_external_user_directory_config_content( server=None, roles=None, **kwargs ): From 81d150ed432138481debc29aa78be74fdfee0971 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 4 Apr 2022 20:12:10 +0200 Subject: [PATCH 279/372] Implementation of makeDateTime() and makeDateTime64() --- src/Functions/makeDate.cpp | 239 ++++++++++++++++++ .../0_stateless/02244_make_datetime.reference | 33 +++ .../0_stateless/02244_make_datetime.sql | 39 +++ .../02245_make_datetime64.reference | 47 ++++ .../0_stateless/02245_make_datetime64.sql | 64 +++++ 5 files changed, 422 insertions(+) create mode 100644 tests/queries/0_stateless/02244_make_datetime.reference create mode 100644 tests/queries/0_stateless/02244_make_datetime.sql create mode 100644 tests/queries/0_stateless/02245_make_datetime64.reference create mode 100644 tests/queries/0_stateless/02245_make_datetime64.sql diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 4ec765f0191..675d22d7f00 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -167,12 +169,249 @@ struct MakeDate32Traits static constexpr auto MAX_DATE = YearMonthDayToSingleInt(MAX_YEAR, 11, 11); }; +/// Common implementation for makeDateTime, makeDateTime64 +class FunctionMakeDateTimeBase : public FunctionWithNumericParamsBase +{ +protected: + static constexpr std::array argument_names = {"year", "month", "day", "hour", "minute", "second"}; + +public: + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + +protected: + void checkRequiredArguments(const ColumnsWithTypeAndName & arguments, const size_t optional_argument_count) const + { + FunctionWithNumericParamsBase::checkRequiredArguments(arguments, argument_names, optional_argument_count); + } + + void convertRequiredArguments(const ColumnsWithTypeAndName & arguments, Columns & converted_arguments) const + { + FunctionWithNumericParamsBase::convertRequiredArguments(arguments, argument_names, converted_arguments); + } + + template + static Int64 dateTime(T year, T month, T day_of_month, T hour, T minute, T second, const DateLUTImpl & lut) + { + /// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String + /// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39" + if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || + hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99)) + return lut.makeDateTime(DATE_LUT_MIN_YEAR-1, 1, 1, 0, 0, 0); + + if (unlikely(year > DATE_LUT_MAX_YEAR)) + return lut.makeDateTime(DATE_LUT_MAX_YEAR+1, 1, 1, 23, 59, 59); + + return lut.makeDateTime(year, month, day_of_month, hour, minute, second); + } + + std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const + { + std::string timezone; + if (!isStringOrFixedString(timezone_argument.type) || !timezone_argument.column || (timezone_argument.column->size() != 1 && !typeid_cast(timezone_argument.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'timezone' for function {} must be const string", getName()); + timezone = timezone_argument.column->getDataAt(0).toString(); + + return timezone; + } +}; + +/// makeDateTime(year, month, day, hour, minute, second, [timezone]) +class FunctionMakeDateTime : public FunctionMakeDateTimeBase +{ +private: + static constexpr std::array optional_argument_names = {"timezone"}; + +public: + static constexpr auto name = "makeDateTime"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + checkRequiredArguments(arguments, optional_argument_names.size()); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments; + convertRequiredArguments(arguments, converted_arguments); + + auto res_column = ColumnUInt32::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + const auto & hour_data = typeid_cast(*converted_arguments[3]).getData(); + const auto & minute_data = typeid_cast(*converted_arguments[4]).getData(); + const auto & second_data = typeid_cast(*converted_arguments[5]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto year = year_data[i]; + const auto month = month_data[i]; + const auto day = day_data[i]; + const auto hour = hour_data[i]; + const auto minute = minute_data[i]; + const auto second = second_data[i]; + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + if (unlikely(date_time < 0)) + date_time = 0; + else if (unlikely(date_time > 0x0ffffffffll)) + date_time = 0x0ffffffffll; + + result_data[i] = date_time; + } + + return res_column; + } +}; + +/// makeDateTime64(year, month, day, hour, minute, second, [fraction], [precision], [timezone]) +class FunctionMakeDateTime64 : public FunctionMakeDateTimeBase +{ +private: + static constexpr std::array optional_argument_names = {"fraction", "precision", "timezone"}; + static constexpr UInt8 DEFAULT_PRECISION = 3; + +public: + static constexpr auto name = "makeDateTime64"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + checkRequiredArguments(arguments, optional_argument_names.size()); + + if (arguments.size() >= argument_names.size() + 1) + { + const auto& fraction_argument = arguments[argument_names.size()]; + if (!isNumber(fraction_argument.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'fraction' for function {} must be number", getName()); + } + + /// Optional precision argument + Int64 precision = DEFAULT_PRECISION; + if (arguments.size() >= argument_names.size() + 2) + precision = extractPrecision(arguments[argument_names.size() + 1]); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 3) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(precision, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + /// Optional precision argument + Int64 precision = DEFAULT_PRECISION; + if (arguments.size() >= argument_names.size() + 2) + precision = extractPrecision(arguments[argument_names.size() + 1]); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == argument_names.size() + 3) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments; + convertRequiredArguments(arguments, converted_arguments); + + /// Optional fraction argument + const ColumnVector::Container * fraction_data = nullptr; + if (arguments.size() >= argument_names.size() + 1) + { + ColumnPtr fraction_column = castColumn(arguments[argument_names.size()], std::make_shared()); + fraction_column = fraction_column->convertToFullColumnIfConst(); + converted_arguments.push_back(fraction_column); + fraction_data = &typeid_cast(*converted_arguments[6]).getData(); + } + + auto res_column = ColumnDecimal::create(input_rows_count, precision); + auto & result_data = res_column->getData(); + + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + const auto & hour_data = typeid_cast(*converted_arguments[3]).getData(); + const auto & minute_data = typeid_cast(*converted_arguments[4]).getData(); + const auto & second_data = typeid_cast(*converted_arguments[5]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + const auto max_fraction = pow(10, precision) - 1; + + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto year = year_data[i]; + const auto month = month_data[i]; + const auto day = day_data[i]; + const auto hour = hour_data[i]; + const auto minute = minute_data[i]; + const auto second = second_data[i]; + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + + auto fraction = fraction_data ? (*fraction_data)[i] : 0; + if (unlikely(fraction < 0)) + fraction = 0; + else if (unlikely(fraction > max_fraction)) + fraction = max_fraction; + + result_data[i] = DecimalUtils::decimalFromComponents(date_time, fraction, precision); + } + + return res_column; + } + +private: + UInt8 extractPrecision(const ColumnWithTypeAndName & precision_argument) const + { + Int64 precision = DEFAULT_PRECISION; + if (!isNumber(precision_argument.type) || !precision_argument.column || (precision_argument.column->size() != 1 && !typeid_cast(precision_argument.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'precision' for function {} must be constant number", getName()); + precision = precision_argument.column->getInt(0); + if (precision < 0 || precision > 9) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Argument 'precision' for function {} must be in range [0, 9]", getName()); + + return precision; + } +}; + } void registerFunctionsMakeDate(FunctionFactory & factory) { factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02244_make_datetime.reference b/tests/queries/0_stateless/02244_make_datetime.reference new file mode 100644 index 00000000000..57524c26254 --- /dev/null +++ b/tests/queries/0_stateless/02244_make_datetime.reference @@ -0,0 +1,33 @@ +1991-08-24 21:04:00 +1991-08-24 21:04:00 +1991-08-24 19:04:00 +DateTime +DateTime(\'CET\') +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1984-01-02 01:00:00 +1984-01-01 01:10:00 +1984-01-01 00:01:10 +1984-01-01 00:00:00 +1983-03-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02244_make_datetime.sql b/tests/queries/0_stateless/02244_make_datetime.sql new file mode 100644 index 00000000000..9b8f561994b --- /dev/null +++ b/tests/queries/0_stateless/02244_make_datetime.sql @@ -0,0 +1,39 @@ +select makeDateTime(1991, 8, 24, 21, 4, 0); +select makeDateTime(1991, 8, 24, 21, 4, 0, 'CET'); +select cast(makeDateTime(1991, 8, 24, 21, 4, 0, 'CET') as DateTime('UTC')); + +select toTypeName(makeDateTime(1991, 8, 24, 21, 4, 0)); +select toTypeName(makeDateTime(1991, 8, 24, 21, 4, 0, 'CET')); + +select makeDateTime(1925, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1924, 12, 31, 23, 59, 59, 'UTC'); +select makeDateTime(2283, 11, 11, 23, 59, 59, 'UTC'); +select makeDateTime(2283, 11, 12, 0, 0, 0, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 16, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 17, 'UTC'); +select makeDateTime(2262, 4, 11, 23, 47, 16, 'UTC'); + +select makeDateTime(1984, 0, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 0, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 13, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 41, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 25, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 70, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, 70, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError 1000 } + +select makeDateTime(1984, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1983, 2, 29, 0, 0, 0, 'UTC'); +select makeDateTime(-1984, 1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, -1, 1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, -1, 0, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, -1, 0, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, -1, 0, 'UTC'); +select makeDateTime(1984, 1, 1, 0, 0, -1, 'UTC'); + +select makeDateTime(65537, 8, 24, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 65537, 24, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 65537, 21, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 65537, 4, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 21, 65537, 0, 'UTC'); +select makeDateTime(1991, 8, 24, 21, 4, 65537, 'UTC'); \ No newline at end of file diff --git a/tests/queries/0_stateless/02245_make_datetime64.reference b/tests/queries/0_stateless/02245_make_datetime64.reference new file mode 100644 index 00000000000..bcdf7dacccc --- /dev/null +++ b/tests/queries/0_stateless/02245_make_datetime64.reference @@ -0,0 +1,47 @@ +1991-08-24 21:04:00.000 +1991-08-24 21:04:00.123 +1991-08-24 21:04:00.001234 +1991-08-24 21:04:00.0001234 +1991-08-24 19:04:00.0001234 +DateTime64(3) +DateTime64(3) +DateTime64(6) +DateTime64(7, \'CET\') +DateTime64(7, \'UTC\') +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.999999999 +2283-11-11 23:59:59.99999999 +2262-04-11 23:47:16.854775807 +2262-04-11 23:47:16.85477581 +1991-08-24 21:04:00 +1991-08-24 21:04:00.9 +1991-08-24 21:04:00.99 +1991-08-24 21:04:00.999 +1991-08-24 21:04:00.1234 +1991-08-24 21:04:00.01234 +1991-08-24 21:04:00.001234 +1991-08-24 21:04:00.0001234 +1991-08-24 21:04:00.00001234 +1991-08-24 21:04:00.000001234 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-02 01:00:00.000000000 +1984-01-01 01:10:00.000000000 +1984-01-01 00:01:10.000000000 +1984-01-01 00:00:00.000000000 +1983-03-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +2283-11-11 23:59:59.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 +1925-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql new file mode 100644 index 00000000000..7bd3c1df2da --- /dev/null +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -0,0 +1,64 @@ +select makeDateTime64(1991, 8, 24, 21, 4, 0); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 123); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET'); +select cast(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET') as DateTime64(7, 'UTC')); + +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 123)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6)); +select toTypeName(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET')); +select toTypeName(cast(makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET') as DateTime64(7, 'UTC'))); + +select makeDateTime64(1925, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1924, 12, 31, 23, 59, 59, 999999999, 9, 'UTC'); +select makeDateTime64(2283, 11, 11, 23, 59, 59, 99999999, 8, 'UTC'); +select makeDateTime64(2283, 11, 11, 23, 59, 59, 999999999, 9, 'UTC'); -- { serverError 407 } +select makeDateTime64(2262, 4, 11, 23, 47, 16, 854775807, 9, 'UTC'); +select makeDateTime64(2262, 4, 11, 23, 47, 16, 854775808, 9, 'UTC'); -- { serverError 407 } +select makeDateTime64(2262, 4, 11, 23, 47, 16, 85477581, 8, 'UTC'); + +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 0, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 1, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 2, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 3, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 4, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 5, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 6, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 7, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 8, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 9, 'CET'); +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, 10, 'CET'); -- { serverError 69 } +select makeDateTime64(1991, 8, 24, 21, 4, 0, 1234, -1, 'CET'); -- { serverError 69 } + +select makeDateTime64(1984, 0, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 0, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 13, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 41, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 25, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } + +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1983, 2, 29, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(-1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, -1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, -1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, -1, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, -1, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, -1, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, -1, 9, 'UTC'); + +select makeDateTime64(65537, 8, 24, 21, 4, 0); +select makeDateTime64(1991, 65537, 24, 21, 4, 0); +select makeDateTime64(1991, 8, 65537, 21, 4, 0); +select makeDateTime64(1991, 8, 24, 65537, 4, 0); +select makeDateTime64(1991, 8, 24, 21, 65537, 0); +select makeDateTime64(1991, 8, 24, 21, 4, 65537); + +select makeDateTime64(year, 1, 1, 1, 0, 0, 0, precision, timezone) from ( + select 1984 as year, 5 as precision, 'UTC' as timezone + union all + select 1985 as year, 5 as precision, 'UTC' as timezone +); -- { serverError 43 } From 517c2ae8dfc67d9a280ecfadf917c28758e26344 Mon Sep 17 00:00:00 2001 From: fenglv Date: Thu, 7 Apr 2022 02:20:20 +0000 Subject: [PATCH 280/372] Fix data race in StorgeFileLog --- src/Storages/FileLog/StorageFileLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 4fb19b12bab..d7c732aee02 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -720,7 +720,7 @@ bool StorageFileLog::streamToViews() assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews"); - size_t rows = 0; + std::atomic rows = 0; { block_io.pipeline.complete(std::move(input)); block_io.pipeline.setNumThreads(max_streams_number); From acc7046d54738054df39cd43842534265a723d6e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 11:46:57 +0800 Subject: [PATCH 281/372] remove some useless virtual and rename some functions in HiveFile --- src/Storages/Hive/HiveFile.cpp | 6 +- src/Storages/Hive/HiveFile.h | 114 +++++++++++++----------------- src/Storages/Hive/StorageHive.cpp | 4 +- 3 files changed, 56 insertions(+), 68 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 3f4260d9f9e..cc2687415ff 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -145,7 +145,7 @@ void HiveOrcFile::prepareColumnMapping() } } -bool HiveOrcFile::hasMinMaxIndex() const +bool HiveOrcFile::useFileMinMaxIndex() const { return storage_settings->enable_orc_file_minmax_index; } @@ -196,7 +196,7 @@ void HiveOrcFile::loadMinMaxIndex() minmax_idx = buildMinMaxIndex(statistics.get()); } -bool HiveOrcFile::hasSubMinMaxIndex() const +bool HiveOrcFile::useSplitMinMaxIndex() const { return storage_settings->enable_orc_stripe_minmax_index; } @@ -226,7 +226,7 @@ void HiveOrcFile::loadSubMinMaxIndex() } } -bool HiveParquetFile::hasSubMinMaxIndex() const +bool HiveParquetFile::useSplitMinMaxIndex() const { return storage_settings->enable_parquet_rowgroup_minmax_index; } diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 74f893a073f..6a25fa3698b 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -76,7 +76,7 @@ public: } IHiveFile( - const FieldVector & values_, + const FieldVector & partition_values_, const String & namenode_url_, const String & path_, UInt64 last_modify_time_, @@ -85,7 +85,7 @@ public: const std::shared_ptr & storage_settings_, ContextPtr context_) : WithContext(context_) - , partition_values(values_) + , partition_values(partition_values_) , namenode_url(namenode_url_) , path(path_) , last_modify_time(last_modify_time_) @@ -96,56 +96,47 @@ public: } virtual ~IHiveFile() = default; - virtual FileFormat getFormat() const = 0; + String getFormatName() const { return String(magic_enum::enum_name(getFormat())); } + const String & getPath() const { return path; } + UInt64 getLastModTs() const { return last_modify_time; } + size_t getSize() const { return size; } + const FieldVector & getPartitionValues() const { return partition_values; } + const String & getNamenodeUrl() { return namenode_url; } + MinMaxIndexPtr getMinMaxIndex() const { return minmax_idx; } + const std::vector & getSubMinMaxIndexes() const { return sub_minmax_idxes; } - virtual String getName() const = 0; + const std::unordered_set & getSkipSplits() const { return skip_splits; } + void setSkipSplits(const std::unordered_set & skip_splits_) { skip_splits = skip_splits_; } - virtual String getPath() const { return path; } - - virtual FieldVector getPartitionValues() const { return partition_values; } - - virtual String getNamenodeUrl() { return namenode_url; } - - virtual bool hasMinMaxIndex() const { return false; } - - virtual void loadMinMaxIndex() - { - throw Exception("Method loadMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - virtual MinMaxIndexPtr getMinMaxIndex() const { return minmax_idx; } - - // Do hive file contains sub-file level minmax index? - virtual bool hasSubMinMaxIndex() const { return false; } - - virtual void loadSubMinMaxIndex() - { - throw Exception("Method loadSubMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - virtual const std::vector & getSubMinMaxIndexes() const { return sub_minmax_idxes; } - - virtual void setSkipSplits(const std::unordered_set & skip_splits_) { skip_splits = skip_splits_; } - - virtual const std::unordered_set & getSkipSplits() const { return skip_splits; } - - inline std::string describeMinMaxIndex(const MinMaxIndexPtr & idx) const + String describeMinMaxIndex(const MinMaxIndexPtr & idx) const { if (!idx) return ""; - std::vector strs; + std::vector strs; strs.reserve(index_names_and_types.size()); size_t i = 0; for (const auto & name_type : index_names_and_types) - { strs.push_back(name_type.name + ":" + name_type.type->getName() + idx->hyperrectangle[i++].toString()); - } return boost::algorithm::join(strs, "|"); } - inline UInt64 getLastModTs() const { return last_modify_time; } - inline size_t getSize() const { return size; } + virtual FileFormat getFormat() const = 0; + + virtual bool useFileMinMaxIndex() const { return false; } + + virtual void loadMinMaxIndex() + { + throw Exception("Method loadMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } + + /// If hive query could use contains sub-file level minmax index? + virtual bool useSplitMinMaxIndex() const { return false; } + + virtual void loadSubMinMaxIndex() + { + throw Exception("Method loadSubMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } protected: FieldVector partition_values; @@ -168,7 +159,7 @@ class HiveTextFile : public IHiveFile { public: HiveTextFile( - const FieldVector & values_, + const FieldVector & partition_values_, const String & namenode_url_, const String & path_, UInt64 last_modify_time_, @@ -176,19 +167,18 @@ public: const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } virtual FileFormat getFormat() const override { return FileFormat::TEXT; } - virtual String getName() const override { return "TEXT"; } }; class HiveOrcFile : public IHiveFile { public: HiveOrcFile( - const FieldVector & values_, + const FieldVector & partition_values_, const String & namenode_url_, const String & path_, UInt64 last_modify_time_, @@ -196,23 +186,22 @@ public: const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } - virtual FileFormat getFormat() const override { return FileFormat::ORC; } - virtual String getName() const override { return "ORC"; } - virtual bool hasMinMaxIndex() const override; - virtual void loadMinMaxIndex() override; + FileFormat getFormat() const override { return FileFormat::ORC; } + bool useFileMinMaxIndex() const override; + void loadMinMaxIndex() override; - virtual bool hasSubMinMaxIndex() const override; - virtual void loadSubMinMaxIndex() override; + bool useSplitMinMaxIndex() const override; + void loadSubMinMaxIndex() override; -protected: - virtual std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); - virtual Range buildRange(const orc::ColumnStatistics * col_stats); - virtual void prepareReader(); - virtual void prepareColumnMapping(); +private: + std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); + Range buildRange(const orc::ColumnStatistics * col_stats); + void prepareReader(); + void prepareColumnMapping(); std::unique_ptr in; std::unique_ptr reader; @@ -223,7 +212,7 @@ class HiveParquetFile : public IHiveFile { public: HiveParquetFile( - const FieldVector & values_, + const FieldVector & partition_values_, const String & namenode_url_, const String & path_, UInt64 last_modify_time_, @@ -231,18 +220,17 @@ public: const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } - virtual FileFormat getFormat() const override { return FileFormat::PARQUET; } - virtual String getName() const override { return "PARQUET"; } + FileFormat getFormat() const override { return FileFormat::PARQUET; } - virtual bool hasSubMinMaxIndex() const override; - virtual void loadSubMinMaxIndex() override; + bool useSplitMinMaxIndex() const override; + void loadSubMinMaxIndex() override; -protected: - virtual void prepareReader(); +private: + void prepareReader(); std::unique_ptr in; std::unique_ptr reader; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index b4b0862f5d4..c11db119ab7 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -543,7 +543,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( /// Load file level minmax index and apply const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); - if (hive_file->hasMinMaxIndex()) + if (hive_file->useFileMinMaxIndex()) { hive_file->loadMinMaxIndex(); if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) @@ -556,7 +556,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( } /// Load sub-file level minmax index and apply - if (hive_file->hasSubMinMaxIndex()) + if (hive_file->useSplitMinMaxIndex()) { std::unordered_set skip_splits; hive_file->loadSubMinMaxIndex(); From 7dbe8bc2dcff387503945e4ac4ad34e0aa5071cb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 7 Apr 2022 01:21:24 -0400 Subject: [PATCH 282/372] major bugs fixed, tests added, docs updated --- .../sql-reference/statements/select/index.md | 2 +- .../statements/select/order-by.md | 4 +- .../sql-reference/statements/select/index.md | 2 +- .../statements/select/order-by.md | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 12 ++- src/Interpreters/InterpreterSelectQuery.cpp | 39 +++++--- .../RequiredSourceColumnsVisitor.cpp | 10 +- src/Interpreters/TreeRewriter.cpp | 26 ++++- src/Parsers/ASTInterpolateElement.h | 3 +- src/Parsers/ExpressionElementParsers.cpp | 1 + .../0_stateless/02233_interpolate_1.reference | 96 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 23 +++++ 12 files changed, 196 insertions(+), 26 deletions(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 0386083b5df..c5421c83091 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -22,7 +22,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE [(expr_list)]] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 06d3b985854..04630ba1075 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -280,7 +280,7 @@ To fill multiple columns, add `WITH FILL` modifier with optional parameters afte ``` sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] -[INTERPOLATE (col AS expr, ... colN AS exprN)] +[INTERPOLATE [(col [AS expr], ... colN [AS exprN])]] ``` `WITH FILL` can be applied for fields with Numeric (all kinds of float, decimal, int) or Date/DateTime types. When applied for `String` fields, missed values are filled with empty strings. @@ -288,7 +288,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. -`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous field value by applying `expr`. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeate previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 240a0fff7b6..fead3c11060 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -20,7 +20,7 @@ SELECT [DISTINCT [ON (column1, column2, ...)]] expr_list [WHERE expr] [GROUP BY expr_list] [WITH ROLLUP|WITH CUBE] [WITH TOTALS] [HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE (expr_list)] +[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [INTERPOLATE [(expr_list)]] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] [SETTINGS ...] diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 8a4a8d5d111..e293e62e34c 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -280,7 +280,7 @@ SELECT * FROM collate_test ORDER BY s ASC COLLATE 'en'; ```sql ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] -[INTERPOLATE (col AS expr, ... colN AS exprN)] +[INTERPOLATE [(col [AS expr], ... colN [AS exprN])]] ``` `WITH FILL` может быть применен к полям с числовыми (все разновидности float, int, decimal) или временными (все разновидности Date, DateTime) типами. В случае применения к полям типа `String` недостающие значения заполняются пустой строкой. @@ -290,7 +290,7 @@ ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_ Когда `STEP const_numeric_expr` не указан, тогда используется `1.0` для числовых типов, `1 день` для типа Date и `1 секунда` для типа DateTime. -`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. +`INTERPOLATE` может быть применен к колонкам, не участвующим в `ORDER BY WITH FILL`. Такие колонки заполняются значениями, вычисляемыми применением `expr` к предыдущему значению. Если `expr` опущен, то колонка заполняется предыдущим значением. Если список колонок не указан, то включаются все разрешенные колонки. Пример запроса без использования `WITH FILL`: ```sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 4494ce82157..dd3162aa8a4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1336,13 +1336,19 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai if (auto interpolate_list = select_query->interpolate()) { - auto find_columns = [&step](IAST * function) + + NameSet select; + for (const auto & child : select_query->select()->children) + select.insert(child->getAliasOrColumnName()); + + auto find_columns = [&step, &select](IAST * function) { - auto f_impl = [&step](IAST * fn, auto fi) + auto f_impl = [&step, &select](IAST * fn, auto fi) { if (auto * ident = fn->as()) { - step.addRequiredOutput(ident->getColumnName()); + if (select.count(ident->getColumnName()) == 0) + step.addRequiredOutput(ident->getColumnName()); return; } if (fn->as() || fn->as()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index dcd2f2556a9..2c3c2dcf23d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -840,8 +840,8 @@ static InterpolateDescriptionPtr getInterpolateDescription( InterpolateDescriptionPtr interpolate_descr; if (query.interpolate()) { - std::unordered_set col_set; - ColumnsWithTypeAndName columns; + NamesAndTypesList source_columns; + ColumnsWithTypeAndName result_columns; ASTPtr exprs = std::make_shared(); if (query.interpolate()->children.empty()) @@ -854,34 +854,49 @@ static InterpolateDescriptionPtr getInterpolateDescription( column_names.erase(elem->as()->children.front()->getColumnName()); for (const auto & [name, type] : column_names) { - columns.emplace_back(type, name); + source_columns.emplace_back(name, type); + result_columns.emplace_back(type, name); exprs->children.emplace_back(std::make_shared(name)); } } else { + NameSet col_set; for (const auto & elem : query.interpolate()->children) { const auto & interpolate = elem->as(); - const ColumnWithTypeAndName *block_column = source_block.findByName(interpolate.column); - if (!block_column) + + if (const ColumnWithTypeAndName *result_block_column = result_block.findByName(interpolate.column)) + { + if (!col_set.insert(result_block_column->name).second) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Duplicate INTERPOLATE column '{}'", interpolate.column); + + result_columns.emplace_back(result_block_column->type, result_block_column->name); + } + else throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Missing column '{}' as an INTERPOLATE expression target", interpolate.column); - if (!col_set.insert(block_column->name).second) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Duplicate INTERPOLATE column '{}'", interpolate.column); - - columns.emplace_back(block_column->type, block_column->name); exprs->children.emplace_back(interpolate.expr->clone()); } + + col_set.clear(); + for (const auto & column : source_block) + { + source_columns.emplace_back(column.name, column.type); + col_set.insert(column.name); + } + for (const auto & column : result_block) + if( col_set.count(column.name) == 0) + source_columns.emplace_back(column.name, column.type); } - auto syntax_result = TreeRewriter(context).analyze(exprs, source_block.getNamesAndTypesList()); + auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns); ExpressionAnalyzer analyzer(exprs, syntax_result, context); ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), - columns, ActionsDAG::MatchColumnsMode::Position, true); + result_columns, ActionsDAG::MatchColumnsMode::Position, true); ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag)); interpolate_descr = std::make_shared(merge_dag, aliases); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 8ff47ee7972..c46d0ac7bf0 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -115,9 +115,12 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, Data & data) { + NameSet select_columns; /// special case for top-level SELECT items: they are publics for (auto & node : select.select()->children) { + select_columns.insert(node->getAliasOrColumnName()); + if (const auto * identifier = node->as()) data.addColumnIdentifier(*identifier); else @@ -126,13 +129,14 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS if (auto interpolate_list = select.interpolate()) { - auto find_columns = [&data](IAST * function) + auto find_columns = [&data, &select_columns](IAST * function) { - auto f_impl = [&data](IAST * fn, auto fi) + auto f_impl = [&data, &select_columns](IAST * fn, auto fi) { if (auto * ident = fn->as()) { - data.addColumnIdentifier(*ident); + if (select_columns.count(ident->getColumnName()) == 0) + data.addColumnIdentifier(*ident); return; } if (fn->as() || fn->as()) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 929e516f687..d0e54f9f598 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include @@ -420,7 +421,8 @@ void renameDuplicatedColumns(const ASTSelectQuery * select_query) /// Sometimes we have to calculate more columns in SELECT clause than will be returned from query. /// This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. /// Also we have to remove duplicates in case of GLOBAL subqueries. Their results are placed into tables so duplicates are impossible. -void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups) +/// Also remove all INTERPOLATE columns which are not in SELECT anymore. +void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const Names & required_result_columns, bool remove_dups) { ASTs & elements = select_query->select()->children; @@ -449,6 +451,8 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, ASTs new_elements; new_elements.reserve(elements.size()); + NameSet remove_columns; + for (const auto & elem : elements) { String name = elem->getAliasOrColumnName(); @@ -465,6 +469,8 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } else { + remove_columns.insert(name); + ASTFunction * func = elem->as(); /// Never remove untuple. It's result column may be in required columns. @@ -478,6 +484,24 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } } + if (select_query->interpolate()) + { + auto & children = select_query->interpolate()->children; + if (!children.empty()) + { + for (auto it = children.begin(); it != children.end();) + { + if (remove_columns.count((*it)->as()->column)) + it = select_query->interpolate()->children.erase(it); + else + ++it; + } + + if (children.empty()) + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, nullptr); + } + } + elements = std::move(new_elements); } diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index ffb492787c9..70178de053f 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -12,12 +12,13 @@ public: String column; ASTPtr expr; - String getID(char) const override { return "InterpolateElement"; } + String getID(char delim) const override { return String("InterpolateElement") + delim + "(column " + column + ")"; } ASTPtr clone() const override { auto clone = std::make_shared(*this); clone->cloneChildren(); + clone->expr = clone->expr->clone(); return clone; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 1be1e925ca3..cd07e304a39 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2339,6 +2339,7 @@ bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & ex auto elem = std::make_shared(); elem->column = ident->getColumnName(); elem->expr = expr; + elem->children.push_back(expr); node = elem; diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index c7774fe05ff..bb1a93131e5 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -94,3 +94,99 @@ 10.5 14 11 15 11.5 16 +0 1 +0.5 2 +1 original 2 +1.5 3 +2 4 +2.5 5 +3 6 +3.5 7 +4 original 5 +4.5 6 +5 7 +5.5 8 +6 9 +6.5 10 +7 original 8 +7.5 9 +8 10 +8.5 11 +9 12 +9.5 13 +10 14 +10.5 15 +11 16 +11.5 17 + 0 + 0 +original 1 + 3 + 3 + 3 + 3 + 3 +original 4 + 9 + 9 + 9 + 9 + 9 +original 7 + 15 + 15 + 15 + 15 + 15 + 15 + 15 + 15 + 15 +0 0 +0.5 0 +1 original 1 +1.5 3 +2 3 +2.5 3 +3 3 +3.5 3 +4 original 4 +4.5 9 +5 9 +5.5 9 +6 9 +6.5 9 +7 original 7 +7.5 15 +8 15 +8.5 15 +9 15 +9.5 15 +10 15 +10.5 15 +11 15 +11.5 15 +0 1 +0.5 2 +1 original 1 +1.5 2 +2 3 +2.5 4 +3 5 +3.5 6 +4 original 4 +4.5 5 +5 6 +5.5 7 +6 8 +6.5 9 +7 original 7 +7.5 8 +8 9 +8.5 10 +9 11 +9.5 12 +10 13 +10.5 14 +11 15 +11.5 16 diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index a355e903dae..52b597e06e0 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -37,3 +37,26 @@ SELECT n, source, inter FROM ( SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 ) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } + +# Test INTERPOLATE with aliased column +SELECT n, source, inter + 1 AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); + +# Test INTERPOLATE with column not present in select +SELECT source, inter FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter, number + 1 AS inter2 FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter AS inter2 + inter ); + +# Test INTERPOLATE in sub-select +SELECT n, source, inter FROM ( + SELECT n, source, inter, inter2 FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter, number + 1 AS inter2 FROM numbers(10) WHERE (number % 3) = 1 + ) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter AS inter + inter2 ) +); + +# Test INTERPOLATE with aggregates +SELECT n, any(source), sum(inter) AS inter_s FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) GROUP BY n +ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); \ No newline at end of file From f02d76934346453492f90715cde8ecb0c9b660cc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 14:29:35 +0800 Subject: [PATCH 283/372] fix build error --- src/Storages/Hive/HiveFile.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 6a25fa3698b..bd4f12cf6b6 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -198,8 +198,9 @@ public: void loadSubMinMaxIndex() override; private: + static Range buildRange(const orc::ColumnStatistics * col_stats); + std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); - Range buildRange(const orc::ColumnStatistics * col_stats); void prepareReader(); void prepareColumnMapping(); From 046a2ba51c61424aa7a9c00cf1fc651dd6526c51 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 15:35:08 +0800 Subject: [PATCH 284/372] rename some symboles --- src/Storages/Hive/HiveFile.cpp | 32 +++++++++++++++---------------- src/Storages/Hive/HiveFile.h | 22 ++++++++++----------- src/Storages/Hive/StorageHive.cpp | 4 ++-- 3 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index cc2687415ff..166336df78d 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -184,7 +184,7 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c } -void HiveOrcFile::loadMinMaxIndex() +void HiveOrcFile::loadFileMinMaxIndex() { if (!reader) { @@ -193,7 +193,7 @@ void HiveOrcFile::loadMinMaxIndex() } auto statistics = reader->GetRawORCReader()->getStatistics(); - minmax_idx = buildMinMaxIndex(statistics.get()); + file_minmax_idx = buildMinMaxIndex(statistics.get()); } bool HiveOrcFile::useSplitMinMaxIndex() const @@ -202,7 +202,7 @@ bool HiveOrcFile::useSplitMinMaxIndex() const } -void HiveOrcFile::loadSubMinMaxIndex() +void HiveOrcFile::loadSplitMinMaxIndex() { if (!reader) { @@ -218,11 +218,11 @@ void HiveOrcFile::loadSubMinMaxIndex() fmt::format("orc file:{} has different strip num {} and strip statistics num {}", path, stripe_num, stripe_stats_num), ErrorCodes::BAD_ARGUMENTS); - sub_minmax_idxes.resize(stripe_num); + split_minmax_idxes.resize(stripe_num); for (size_t i = 0; i < stripe_num; ++i) { auto stripe_stats = raw_reader->getStripeStatistics(i); - sub_minmax_idxes[i] = buildMinMaxIndex(stripe_stats.get()); + split_minmax_idxes[i] = buildMinMaxIndex(stripe_stats.get()); } } @@ -239,7 +239,7 @@ void HiveParquetFile::prepareReader() THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); } -void HiveParquetFile::loadSubMinMaxIndex() +void HiveParquetFile::loadSplitMinMaxIndex() { if (!reader) prepareReader(); @@ -256,12 +256,12 @@ void HiveParquetFile::loadSubMinMaxIndex() } - sub_minmax_idxes.resize(num_row_groups); + split_minmax_idxes.resize(num_row_groups); for (size_t i = 0; i < num_row_groups; ++i) { auto row_group_meta = meta->RowGroup(i); - sub_minmax_idxes[i] = std::make_shared(); - sub_minmax_idxes[i]->hyperrectangle.resize(num_cols); + split_minmax_idxes[i] = std::make_shared(); + split_minmax_idxes[i]->hyperrectangle.resize(num_cols); size_t j = 0; auto it = index_names_and_types.begin(); @@ -284,31 +284,31 @@ void HiveParquetFile::loadSubMinMaxIndex() if (auto bool_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(bool_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(bool_stats); } else if (auto int32_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int32_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int32_stats); } else if (auto int64_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int64_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(int64_stats); } else if (auto float_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(float_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(float_stats); } else if (auto double_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(double_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(double_stats); } else if (auto string_stats = std::dynamic_pointer_cast(stats)) { - sub_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(string_stats); + split_minmax_idxes[i]->hyperrectangle[j] = createRangeFromParquetStatistics(string_stats); } /// Other types are not supported for minmax index, skip } - sub_minmax_idxes[i]->initialized = true; + split_minmax_idxes[i]->initialized = true; } } diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index bd4f12cf6b6..1404f97eff0 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -102,8 +102,8 @@ public: size_t getSize() const { return size; } const FieldVector & getPartitionValues() const { return partition_values; } const String & getNamenodeUrl() { return namenode_url; } - MinMaxIndexPtr getMinMaxIndex() const { return minmax_idx; } - const std::vector & getSubMinMaxIndexes() const { return sub_minmax_idxes; } + MinMaxIndexPtr getMinMaxIndex() const { return file_minmax_idx; } + const std::vector & getSubMinMaxIndexes() const { return split_minmax_idxes; } const std::unordered_set & getSkipSplits() const { return skip_splits; } void setSkipSplits(const std::unordered_set & skip_splits_) { skip_splits = skip_splits_; } @@ -125,17 +125,17 @@ public: virtual bool useFileMinMaxIndex() const { return false; } - virtual void loadMinMaxIndex() + virtual void loadFileMinMaxIndex() { - throw Exception("Method loadMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Method loadFileMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); } /// If hive query could use contains sub-file level minmax index? virtual bool useSplitMinMaxIndex() const { return false; } - virtual void loadSubMinMaxIndex() + virtual void loadSplitMinMaxIndex() { - throw Exception("Method loadSubMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Method loadSplitMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); } protected: @@ -145,8 +145,8 @@ protected: UInt64 last_modify_time; size_t size; NamesAndTypesList index_names_and_types; - MinMaxIndexPtr minmax_idx; - std::vector sub_minmax_idxes; + MinMaxIndexPtr file_minmax_idx; + std::vector split_minmax_idxes; /// Skip splits for this file after applying minmax index (if any) std::unordered_set skip_splits; std::shared_ptr storage_settings; @@ -192,10 +192,10 @@ public: FileFormat getFormat() const override { return FileFormat::ORC; } bool useFileMinMaxIndex() const override; - void loadMinMaxIndex() override; + void loadFileMinMaxIndex() override; bool useSplitMinMaxIndex() const override; - void loadSubMinMaxIndex() override; + void loadSplitMinMaxIndex() override; private: static Range buildRange(const orc::ColumnStatistics * col_stats); @@ -228,7 +228,7 @@ public: FileFormat getFormat() const override { return FileFormat::PARQUET; } bool useSplitMinMaxIndex() const override; - void loadSubMinMaxIndex() override; + void loadSplitMinMaxIndex() override; private: void prepareReader(); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index c11db119ab7..6745a0f6ad0 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -545,7 +545,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { - hive_file->loadMinMaxIndex(); + hive_file->loadFileMinMaxIndex(); if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { @@ -559,7 +559,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( if (hive_file->useSplitMinMaxIndex()) { std::unordered_set skip_splits; - hive_file->loadSubMinMaxIndex(); + hive_file->loadSplitMinMaxIndex(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) { From 2dc420c66bff45ac68f3160be04b3bca791aaf86 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 15:48:42 +0800 Subject: [PATCH 285/372] rename some symbols in hivefile --- src/Storages/Hive/HiveFile.cpp | 16 ++++++++-------- src/Storages/Hive/HiveFile.h | 4 ++-- src/Storages/Hive/StorageHive.cpp | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 166336df78d..9251f2805cb 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -77,7 +77,7 @@ Range createRangeFromParquetStatistics(std::shared_ptrhasNull()) return {}; @@ -122,7 +122,7 @@ Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) return {}; } -void HiveOrcFile::prepareReader() +void HiveORCFile::prepareReader() { in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); @@ -132,7 +132,7 @@ void HiveOrcFile::prepareReader() reader = std::move(result).ValueOrDie(); } -void HiveOrcFile::prepareColumnMapping() +void HiveORCFile::prepareColumnMapping() { const orc::Type & type = reader->GetRawORCReader()->getType(); size_t count = type.getSubtypeCount(); @@ -145,13 +145,13 @@ void HiveOrcFile::prepareColumnMapping() } } -bool HiveOrcFile::useFileMinMaxIndex() const +bool HiveORCFile::useFileMinMaxIndex() const { return storage_settings->enable_orc_file_minmax_index; } -std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * statistics) +std::unique_ptr HiveORCFile::buildMinMaxIndex(const orc::Statistics * statistics) { if (!statistics) return nullptr; @@ -184,7 +184,7 @@ std::unique_ptr HiveOrcFile::buildMinMaxIndex(c } -void HiveOrcFile::loadFileMinMaxIndex() +void HiveORCFile::loadFileMinMaxIndex() { if (!reader) { @@ -196,13 +196,13 @@ void HiveOrcFile::loadFileMinMaxIndex() file_minmax_idx = buildMinMaxIndex(statistics.get()); } -bool HiveOrcFile::useSplitMinMaxIndex() const +bool HiveORCFile::useSplitMinMaxIndex() const { return storage_settings->enable_orc_stripe_minmax_index; } -void HiveOrcFile::loadSplitMinMaxIndex() +void HiveORCFile::loadSplitMinMaxIndex() { if (!reader) { diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 1404f97eff0..90cbb881c48 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -174,10 +174,10 @@ public: virtual FileFormat getFormat() const override { return FileFormat::TEXT; } }; -class HiveOrcFile : public IHiveFile +class HiveORCFile : public IHiveFile { public: - HiveOrcFile( + HiveORCFile( const FieldVector & partition_values_, const String & namenode_url_, const String & path_, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 6745a0f6ad0..40dce955f01 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -430,7 +430,7 @@ HiveFilePtr createHiveFile( } else if (format_name == "ORC") { - hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); } else if (format_name == "Parquet") { From 77edd41b2e0e2a8e9d89541eb3bf6a8323aaedbb Mon Sep 17 00:00:00 2001 From: jewisliu Date: Wed, 6 Apr 2022 15:23:18 +0800 Subject: [PATCH 286/372] [Improvement] improvement in PARTITION ALL 1. ASTPartition::formatImpl should output ALL while executing ALTER TABLE t DETACH PARTITION ALL 2. prohibit PARTITION ALL excepte DETACH PARTITION ALL --- src/Parsers/ASTPartition.cpp | 13 +++++++---- src/Parsers/ParserPartition.cpp | 4 ---- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++++++++----- src/Storages/StorageMergeTree.cpp | 5 ++-- .../0_stateless/00753_alter_attach.sql | 23 +++++++++++++++++++ 5 files changed, 49 insertions(+), 16 deletions(-) diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index 06bfe4f5217..87d159b5817 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -35,10 +35,15 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta } else { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); - WriteBufferFromOwnString id_buf; - writeQuoted(id, id_buf); - settings.ostr << id_buf.str(); + if (all) + settings.ostr << "ALL"; + else + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); + WriteBufferFromOwnString id_buf; + writeQuoted(id, id_buf); + settings.ostr << id_buf.str(); + } } } diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 5af442826df..9f1d4d4e889 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -32,10 +32,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (s_all.ignore(pos, expected)) { - ASTPtr value = makeASTFunction("tuple"); - partition->value = value; - partition->children.push_back(value); - partition->fields_count = 0; partition->all = true; } else diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index eacec8f50e5..478c1570a23 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3385,7 +3385,14 @@ void MergeTreeData::checkAlterPartitionIsPossible( else { /// We are able to parse it - getPartitionIDFromQuery(command.partition, getContext()); + const auto * partition_ast = command.partition->as(); + if (partition_ast && partition_ast->all) + { + if (command.type != PartitionCommand::DROP_PARTITION) + throw DB::Exception("Only support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED); + } + else + getPartitionIDFromQuery(command.partition, getContext()); } } } @@ -3393,14 +3400,15 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) { - const String partition_id = getPartitionIDFromQuery(partition, getContext()); DataPartsVector parts_to_remove; const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) parts_to_remove = getDataPartsVector(); else + { + const String partition_id = getPartitionIDFromQuery(partition, getContext()); parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); - + } UInt64 partition_size = 0; for (const auto & part : parts_to_remove) @@ -3828,6 +3836,9 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc { const auto & partition_ast = ast->as(); + if (partition_ast.all) + throw Exception("Only Support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED); + if (!partition_ast.value) { MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); @@ -3847,11 +3858,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } /// Re-parse partition key fields using the information about expected field types. - auto metadata_snapshot = getInMemoryMetadataPtr(); const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block; - if (partition_ast.all) - return "ALL"; size_t fields_count = key_sample_block.columns(); if (partition_ast.fields_count != fields_count) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b1392f073ea..e7e4528dc83 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1347,13 +1347,14 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); - String partition_id = getPartitionIDFromQuery(partition, local_context); const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) parts_to_remove = getDataPartsVector(); else + { + String partition_id = getPartitionIDFromQuery(partition, local_context); parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); - + } /// TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(parts_to_remove, true); } diff --git a/tests/queries/0_stateless/00753_alter_attach.sql b/tests/queries/0_stateless/00753_alter_attach.sql index 2910bcc222b..9fa4f92c2c1 100644 --- a/tests/queries/0_stateless/00753_alter_attach.sql +++ b/tests/queries/0_stateless/00753_alter_attach.sql @@ -66,6 +66,29 @@ select * from replicated_table_detach_all1 order by id; SYSTEM SYNC REPLICA replicated_table_detach_all2; select * from replicated_table_detach_all2 order by id; +ALTER TABLE replicated_table_detach_all1 FETCH PARTITION ALL FROM '/clickhouse/tables/test_00753_{database}/replicated_table_detach_all1'; -- { serverError 344 } + DROP TABLE replicated_table_detach_all1; DROP TABLE replicated_table_detach_all2; +DROP TABLE IF EXISTS partition_all; +DROP TABLE IF EXISTS partition_all2; + +CREATE TABLE partition_all (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; +INSERT INTO partition_all VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4); + +CREATE TABLE partition_all2 (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; +INSERT INTO partition_all2 VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4); + +-- test PARTITION ALL +ALTER TABLE partition_all2 REPLACE PARTITION ALL FROM partition_all; -- { serverError 344 } +ALTER TABLE partition_all MOVE PARTITION ALL TO TABLE partition_all2; -- { serverError 344 } +ALTER TABLE partition_all2 CLEAR INDEX p IN PARTITION ALL; -- { serverError 344 } +ALTER TABLE partition_all2 CLEAR COLUMN q IN PARTITION ALL; -- { serverError 344 } +ALTER TABLE partition_all2 UPDATE q = q + 1 IN PARTITION ALL where p = 1; -- { serverError 344 } +ALTER TABLE partition_all2 FREEZE PARTITION ALL; -- { serverError 344 } +CHECK TABLE partition_all2 PARTITION ALL; -- { serverError 344 } +OPTIMIZE TABLE partition_all2 PARTITION ALL; -- { serverError 344 } + +DROP TABLE partition_all; +DROP TABLE partition_all2; From dbdc6322436b85bcbc0dd246a456454c95306ae4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Apr 2022 18:11:49 +0800 Subject: [PATCH 287/372] fixed code style --- tests/integration/test_storage_hdfs/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index b6dcff6d174..c1176e40e42 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -553,6 +553,7 @@ def test_insert_select_schema_inference(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test.native.zst')") assert int(result) == 1 + def test_cluster_join(started_cluster): result = node1.query( """ From b32800420c6eb133a03d3abeb04d24b80700f486 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 7 Apr 2022 13:30:11 +0300 Subject: [PATCH 288/372] Update stress --- docker/test/stress/stress | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 86f8edf5980..10c6088af75 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -10,7 +10,7 @@ import logging import time -def get_options(i): +def get_options(i, backward_compatibility_check): options = [] client_options = [] if 0 < i: @@ -19,7 +19,7 @@ def get_options(i): if i % 3 == 1: options.append("--db-engine=Ordinary") - if i % 3 == 2: + if i % 3 == 2 and not backward_compatibility_check: options.append('''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)) client_options.append('allow_experimental_database_replicated=1') @@ -57,7 +57,7 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t pipes = [] for i in range(0, len(output_paths)): f = open(output_paths[i], 'w') - full_command = "{} {} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option, backward_compatibility_check_option) + full_command = "{} {} {} {} {}".format(cmd, get_options(i, backward_compatibility_check), global_time_limit_option, skip_tests_option, backward_compatibility_check_option) logging.info("Run func tests '%s'", full_command) p = Popen(full_command, shell=True, stdout=f, stderr=f) pipes.append(p) From 42d4a84a6f845e169194fcac64872541c27c22ab Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 7 Apr 2022 10:20:38 +0200 Subject: [PATCH 289/372] More tests for corner cases --- src/Functions/makeDate.cpp | 41 +++++++++++++++--- .../02245_make_datetime64.reference | 30 +++++++++++-- .../0_stateless/02245_make_datetime64.sql | 43 +++++++++++++++---- 3 files changed, 94 insertions(+), 20 deletions(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 675d22d7f00..dbf29322787 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -196,16 +196,28 @@ protected: { /// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String /// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39" - if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || + if (unlikely(std::isnan(year) || std::isnan(month) || std::isnan(day_of_month) || + std::isnan(hour) || std::isnan(minute) || std::isnan(second) || + year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 || hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99)) - return lut.makeDateTime(DATE_LUT_MIN_YEAR-1, 1, 1, 0, 0, 0); + return minDateTime(lut); if (unlikely(year > DATE_LUT_MAX_YEAR)) - return lut.makeDateTime(DATE_LUT_MAX_YEAR+1, 1, 1, 23, 59, 59); + return maxDateTime(lut); return lut.makeDateTime(year, month, day_of_month, hour, minute, second); } + static Int64 minDateTime(const DateLUTImpl & lut) + { + return lut.makeDateTime(DATE_LUT_MIN_YEAR - 1, 1, 1, 0, 0, 0); + } + + static Int64 maxDateTime(const DateLUTImpl & lut) + { + return lut.makeDateTime(DATE_LUT_MAX_YEAR + 1, 1, 1, 23, 59, 59); + } + std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const { std::string timezone; @@ -364,6 +376,8 @@ public: const auto & date_lut = DateLUT::instance(timezone); const auto max_fraction = pow(10, precision) - 1; + const auto min_date_time = minDateTime(date_lut); + const auto max_date_time = maxDateTime(date_lut); for (size_t i = 0; i < input_rows_count; ++i) { @@ -376,11 +390,24 @@ public: auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); - auto fraction = fraction_data ? (*fraction_data)[i] : 0; - if (unlikely(fraction < 0)) + double fraction = 0; + if (unlikely(date_time == min_date_time)) fraction = 0; - else if (unlikely(fraction > max_fraction)) - fraction = max_fraction; + else if (unlikely(date_time == max_date_time)) + fraction = 999999999ll; + else + { + fraction = fraction_data ? (*fraction_data)[i] : 0; + if (unlikely(std::isnan(fraction))) + { + date_time = min_date_time; + fraction = 0; + } + else if (unlikely(fraction < 0)) + fraction = 0; + else if (unlikely(fraction > max_fraction)) + fraction = max_fraction; + } result_data[i] = DecimalUtils::decimalFromComponents(date_time, fraction, precision); } diff --git a/tests/queries/0_stateless/02245_make_datetime64.reference b/tests/queries/0_stateless/02245_make_datetime64.reference index bcdf7dacccc..eea768f1dc7 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.reference +++ b/tests/queries/0_stateless/02245_make_datetime64.reference @@ -9,7 +9,7 @@ DateTime64(6) DateTime64(7, \'CET\') DateTime64(7, \'UTC\') 1925-01-01 00:00:00.000000000 -1925-01-01 00:00:00.999999999 +1925-01-01 00:00:00.000000000 2283-11-11 23:59:59.99999999 2262-04-11 23:47:16.854775807 2262-04-11 23:47:16.85477581 @@ -30,8 +30,23 @@ DateTime64(7, \'UTC\') 1984-01-02 01:00:00.000000000 1984-01-01 01:10:00.000000000 1984-01-01 00:01:10.000000000 -1984-01-01 00:00:00.000000000 -1983-03-01 00:00:00.000000000 +1984-01-01 02:03:04.000000005 +1984-02-29 02:03:04.000000005 +1983-03-01 02:03:04.000000005 +1984-03-01 02:03:04.000000005 +1983-03-02 02:03:04.000000005 +1984-03-02 02:03:04.000000005 +1983-03-03 02:03:04.000000005 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1925-01-01 00:00:00.000000000 +1984-01-01 02:03:04.000000000 +1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 @@ -39,7 +54,14 @@ DateTime64(7, \'UTC\') 1925-01-01 00:00:00.000000000 1925-01-01 00:00:00.000000000 1984-01-01 00:00:00.000000000 -2283-11-11 23:59:59.000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +1984-01-01 00:00:00.000000000 +2283-11-11 23:59:59.999 1925-01-01 00:00:00.000 1925-01-01 00:00:00.000 1925-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index 7bd3c1df2da..2e2d81fa363 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -40,15 +40,40 @@ select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } -select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1983, 2, 29, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(-1984, 1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, -1, 1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, -1, 0, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, -1, 0, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, -1, 0, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, -1, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, 0, -1, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 29, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 29, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 30, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 30, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 31, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 31, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 2, 32, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1983, 2, 32, 2, 3, 4, 5, 9, 'UTC'); + +select makeDateTime64(-1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, -1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, -1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, -1, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, -1, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, -1, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, -1, 9, 'UTC'); + +select makeDateTime64(NaN, 1, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, NaN, 1, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, NaN, 2, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, NaN, 3, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, NaN, 4, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, NaN, 5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 2, 3, 4, NaN, 9, 'UTC'); + +select makeDateTime64(1984.5, 1, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1.5, 1, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1.5, 0, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0.5, 0, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0.5, 0, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0.5, 0, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0.5, 9, 'UTC'); +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9.5, 'UTC'); select makeDateTime64(65537, 8, 24, 21, 4, 0); select makeDateTime64(1991, 65537, 24, 21, 4, 0); From a998d73ee22dcf02d0388d3adaf13c3af6742c9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Apr 2022 12:37:11 +0200 Subject: [PATCH 290/372] Update StorageKafka.cpp --- src/Storages/Kafka/StorageKafka.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 79c967ce9c0..127911c9f60 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -661,7 +661,7 @@ bool StorageKafka::streamToViews() // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) - size_t rows = 0; + std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); From 69160fa3e33f15bfc6df201c62e814ed1876c622 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Mar 2022 13:42:04 +0000 Subject: [PATCH 291/372] remove dead code in needRewrite function --- src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index d6a00ba89b4..aa2585bdbfc 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -182,7 +182,6 @@ struct RewriteTablesVisitorData } }; -template bool needRewrite(ASTSelectQuery & select, std::vector & table_expressions) { if (!select.tables()) @@ -233,8 +232,6 @@ bool needRewrite(ASTSelectQuery & select, std::vector table_expressions; - if (!needRewrite<2>(select, table_expressions)) + if (!needRewrite(select, table_expressions)) return; auto & src_tables = select.tables()->children; From 846e4b94fb65cb574494364d7760664c70cafec5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 18:32:54 +0200 Subject: [PATCH 292/372] Tiny improvements to git and version helpers --- tests/ci/git_helper.py | 2 +- tests/ci/version_helper.py | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 5c02ea53cf8..50414ffb470 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -93,7 +93,7 @@ class Git: if value == "": return if not self._tag_pattern.match(value): - raise Exception(f"last tag {value} doesn't match the pattern") + raise ValueError(f"last tag {value} doesn't match the pattern") @property def latest_tag(self) -> str: diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 7db96cfde7c..ecccc250e33 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -150,6 +150,9 @@ class ClickHouseVersion: return False + def __le__(self, other: "ClickHouseVersion") -> bool: + return self == other or self < other + class VersionType: LTS = "lts" From eb62b1884582445ada00c4bea4198c4d9cb3b952 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Apr 2022 18:41:18 +0200 Subject: [PATCH 293/372] Improve and fix edge cases for docker_server.py - Allow define version as file - Add inline cache - Fix auto_release_type function --- tests/ci/docker_server.py | 22 +++++++++++++++++----- tests/ci/docker_test.py | 19 +++++++++++++++---- 2 files changed, 32 insertions(+), 9 deletions(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 789abc30b0c..a274cbbf170 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -16,6 +16,7 @@ from commit_status_helper import post_commit_status from docker_images_check import DockerImage from env_helper import CI, GITHUB_RUN_URL, RUNNER_TEMP, S3_BUILDS_BUCKET from get_robot_token import get_best_robot_token, get_parameter_from_ssm +from git_helper import removeprefix from pr_info import PRInfo from s3_helper import S3Helper from stopwatch import Stopwatch @@ -25,6 +26,7 @@ from version_helper import ( get_tagged_versions, get_version_from_repo, get_version_from_string, + get_version_from_tag, ) TEMP_PATH = p.join(RUNNER_TEMP, "docker_images_check") @@ -49,7 +51,8 @@ def parse_args() -> argparse.Namespace: "--version", type=version_arg, default=get_version_from_repo().string, - help="a version to build", + help="a version to build, automaticaly got from version_helper, accepts either " + "tag ('refs/tags/' is removed automatically) or a normal 22.2.2.2 format", ) parser.add_argument( "--release-type", @@ -112,10 +115,19 @@ def parse_args() -> argparse.Namespace: def version_arg(version: str) -> ClickHouseVersion: + version = removeprefix(version, "refs/tags/") try: return get_version_from_string(version) - except ValueError as e: - raise argparse.ArgumentTypeError(e) + except ValueError: + pass + try: + return get_version_from_tag(version) + except ValueError: + pass + + raise argparse.ArgumentTypeError( + f"version {version} does not match tag of plain version" + ) def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: @@ -125,7 +137,7 @@ def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: git_versions = get_tagged_versions() reference_version = git_versions[0] for i in reversed(range(len(git_versions))): - if git_versions[i] < version: + if git_versions[i] <= version: if i == len(git_versions) - 1: return "latest" reference_version = git_versions[i + 1] @@ -209,7 +221,7 @@ def build_and_push_image( result = [] if os != "ubuntu": tag += f"-{os}" - init_args = ["docker", "buildx", "build"] + init_args = ["docker", "buildx", "build", "--build-arg BUILDKIT_INLINE_CACHE=1"] if push: init_args.append("--push") init_args.append("--output=type=image,push-by-digest=true") diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 0d1c554026f..23a007ccac0 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -9,7 +9,7 @@ from pr_info import PRInfo import docker_images_check as di with patch("git_helper.Git"): - from version_helper import get_version_from_string, get_tagged_versions + from version_helper import get_version_from_string import docker_server as ds # di.logging.basicConfig(level=di.logging.INFO) @@ -251,7 +251,8 @@ class TestDockerServer(unittest.TestCase): get_version_from_string("2.2.1.1"), get_version_from_string("2.2.2.1"), ] - cases = ( + + cases_less = ( (get_version_from_string("1.0.1.1"), "minor"), (get_version_from_string("1.1.2.1"), "minor"), (get_version_from_string("1.3.1.1"), "major"), @@ -260,8 +261,18 @@ class TestDockerServer(unittest.TestCase): (get_version_from_string("2.2.3.1"), "latest"), (get_version_from_string("2.3.1.1"), "latest"), ) - _ = get_tagged_versions() - for case in cases: + for case in cases_less: + release = ds.auto_release_type(case[0], "auto") + self.assertEqual(case[1], release) + + cases_equal = ( + (get_version_from_string("1.1.1.1"), "minor"), + (get_version_from_string("1.2.1.1"), "major"), + (get_version_from_string("2.1.1.1"), "minor"), + (get_version_from_string("2.2.1.1"), "patch"), + (get_version_from_string("2.2.2.1"), "latest"), + ) + for case in cases_equal: release = ds.auto_release_type(case[0], "auto") self.assertEqual(case[1], release) From 9c5ad2a1581e02f3ed876391ef983611a4225c60 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 12:23:15 +0200 Subject: [PATCH 294/372] Fix action for docker images build --- .github/workflows/release.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 29e3d0c4358..ea2e1ed33fb 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -52,8 +52,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type auto - python3 docker_server.py --release-type auto --no-ubuntu \ + python3 docker_server.py --release-type auto --version "${{ github.ref }}" + python3 docker_server.py --release-type auto --version "${{ github.ref }}" --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() From 58d381de5c1703aa99ecd5a81c6c0807d44269f0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 12:25:00 +0200 Subject: [PATCH 295/372] A temporary fix for artifactory push before multiple architectures --- tests/ci/push_to_artifactory.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/push_to_artifactory.py b/tests/ci/push_to_artifactory.py index ccbf1918602..be977bdd907 100755 --- a/tests/ci/push_to_artifactory.py +++ b/tests/ci/push_to_artifactory.py @@ -40,13 +40,12 @@ class Packages: "_".join((name, version, arch + ".deb")) for name, arch in self.packages ) - rev = "2" self.rpm = tuple( - "-".join((name, version, rev + "." + self.rpm_arch[arch] + ".rpm")) + "-".join((name, version + "." + self.rpm_arch[arch] + ".rpm")) for name, arch in self.packages ) - self.tgz = tuple(f"{name}-{version}.tgz" for name, _ in self.packages) + self.tgz = tuple(f"{name}-{version}-amd64.tgz" for name, _ in self.packages) def arch(self, deb_pkg: str) -> str: if deb_pkg not in self.deb: From 11fe15be9e96ef68dce1843c3221db83b9406110 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 12:37:46 +0200 Subject: [PATCH 296/372] Add python unit tests to backport workflow --- .github/workflows/backport_branches.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 75f8a63368d..ea8eef40933 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -9,6 +9,18 @@ on: # yamllint disable-line rule:truthy branches: - 'backport/**' jobs: + PythonUnitTests: + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Python unit tests + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: From 82583b93491a48e0c5a7600e80540bafca3b1240 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Apr 2022 13:07:18 +0200 Subject: [PATCH 297/372] Update 02245_format_string_stack_overflow --- .../queries/0_stateless/02245_format_string_stack_overflow.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql index 9376b12aa1e..40053fd0d9b 100644 --- a/tests/queries/0_stateless/02245_format_string_stack_overflow.sql +++ b/tests/queries/0_stateless/02245_format_string_stack_overflow.sql @@ -1,2 +1,2 @@ --- Tags: no-backward-compatibility-check:22.3.2.2 +-- Tags: no-backward-compatibility-check:22.3 select format('{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}{0}', toString(number)) str from numbers(1); From 80503b29836621b82b8f0b690a2131b9c3fcd00a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 7 Apr 2022 13:16:19 +0200 Subject: [PATCH 298/372] Update clickhouse-test --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9ca820652d9..3efb37cc27d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -535,7 +535,9 @@ class TestCase: server_version = str(clickhouse_execute(args, "SELECT version()").decode()) # If server version is less or equal from the version specified in tag, we should skip this test. - if list(map(int, server_version.split('.'))) <= list(map(int, version_from_tag.split('.'))): + version_from_tag_split = list(map(int, version_from_tag.split('.'))) + server_version_split = list(map(int, server_version.split('.'))) + if server_version_split[:len(version_from_tag_split)] <= version_from_tag_split: return True return False From eb7ac4c3a67115f4517b0ca5c955ca1915838258 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Apr 2022 11:32:32 +0000 Subject: [PATCH 299/372] Handle empty expression list in cross join --- .../JoinToSubqueryTransformVisitor.cpp | 4 ++ .../02265_cross_join_empty_list.reference | 52 +++++++++++++++++++ .../02265_cross_join_empty_list.sql | 6 +++ 3 files changed, 62 insertions(+) create mode 100644 tests/queries/0_stateless/02265_cross_join_empty_list.reference create mode 100644 tests/queries/0_stateless/02265_cross_join_empty_list.sql diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index aa2585bdbfc..c43302e0de9 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -377,7 +377,11 @@ private: static void visit(ASTSelectQuery & select, ASTPtr &, Data & data) { if (!data.done) + { + if (data.expression_list->children.empty()) + data.expression_list->children.emplace_back(std::make_shared()); select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list)); + } data.done = true; } }; diff --git a/tests/queries/0_stateless/02265_cross_join_empty_list.reference b/tests/queries/0_stateless/02265_cross_join_empty_list.reference new file mode 100644 index 00000000000..fef5e889a1e --- /dev/null +++ b/tests/queries/0_stateless/02265_cross_join_empty_list.reference @@ -0,0 +1,52 @@ +24 +24 +24 +24 24 24 +0 0 0 +0 0 1 +0 0 2 +0 0 3 +0 1 0 +0 1 1 +0 1 2 +0 1 3 +0 2 0 +0 2 1 +0 2 2 +0 2 3 +1 0 0 +1 0 1 +1 0 2 +1 0 3 +1 1 0 +1 1 1 +1 1 2 +1 1 3 +1 2 0 +1 2 1 +1 2 2 +1 2 3 +0 0 0 +0 0 1 +0 0 2 +0 0 3 +0 1 0 +0 1 1 +0 1 2 +0 1 3 +0 2 0 +0 2 1 +0 2 2 +0 2 3 +1 0 0 +1 0 1 +1 0 2 +1 0 3 +1 1 0 +1 1 1 +1 1 2 +1 1 3 +1 2 0 +1 2 1 +1 2 2 +1 2 3 diff --git a/tests/queries/0_stateless/02265_cross_join_empty_list.sql b/tests/queries/0_stateless/02265_cross_join_empty_list.sql new file mode 100644 index 00000000000..346a047351d --- /dev/null +++ b/tests/queries/0_stateless/02265_cross_join_empty_list.sql @@ -0,0 +1,6 @@ +SELECT count(1) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3; +SELECT count(*) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3; +SELECT count() FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3; +SELECT count(n1.number), count(n2.number), count(n3.number) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3; +SELECT * FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 ORDER BY n1.number, n2.number, n3.number; +SELECT n1.number, n2.number, n3.number FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 ORDER BY n1.number, n2.number, n3.number; From a7145cf087f107f0e0b8a53d70dbd89588beddb5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 13:48:52 +0200 Subject: [PATCH 300/372] Move version_arg to version_helper, add tests --- tests/ci/docker_server.py | 20 +------------------- tests/ci/version_helper.py | 16 +++++++++++++++- tests/ci/version_test.py | 33 +++++++++++++++++++++++++++++++++ 3 files changed, 49 insertions(+), 20 deletions(-) create mode 100644 tests/ci/version_test.py diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index a274cbbf170..0ddafc4b582 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -16,7 +16,6 @@ from commit_status_helper import post_commit_status from docker_images_check import DockerImage from env_helper import CI, GITHUB_RUN_URL, RUNNER_TEMP, S3_BUILDS_BUCKET from get_robot_token import get_best_robot_token, get_parameter_from_ssm -from git_helper import removeprefix from pr_info import PRInfo from s3_helper import S3Helper from stopwatch import Stopwatch @@ -25,8 +24,7 @@ from version_helper import ( ClickHouseVersion, get_tagged_versions, get_version_from_repo, - get_version_from_string, - get_version_from_tag, + version_arg, ) TEMP_PATH = p.join(RUNNER_TEMP, "docker_images_check") @@ -114,22 +112,6 @@ def parse_args() -> argparse.Namespace: return parser.parse_args() -def version_arg(version: str) -> ClickHouseVersion: - version = removeprefix(version, "refs/tags/") - try: - return get_version_from_string(version) - except ValueError: - pass - try: - return get_version_from_tag(version) - except ValueError: - pass - - raise argparse.ArgumentTypeError( - f"version {version} does not match tag of plain version" - ) - - def auto_release_type(version: ClickHouseVersion, release_type: str) -> str: if release_type != "auto": return release_type diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index ecccc250e33..9c67191e4c3 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 import logging import os.path as p -from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter +from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter, ArgumentTypeError from typing import Dict, List, Tuple, Union from git_helper import Git, removeprefix @@ -220,6 +220,20 @@ def get_version_from_tag(tag: str) -> ClickHouseVersion: return get_version_from_string(tag) +def version_arg(version: str) -> ClickHouseVersion: + version = removeprefix(version, "refs/tags/") + try: + return get_version_from_string(version) + except ValueError: + pass + try: + return get_version_from_tag(version) + except ValueError: + pass + + raise ArgumentTypeError(f"version {version} does not match tag of plain version") + + def get_tagged_versions() -> List[ClickHouseVersion]: versions = [] for tag in git.get_tags(): diff --git a/tests/ci/version_test.py b/tests/ci/version_test.py new file mode 100644 index 00000000000..86a2d58c3c8 --- /dev/null +++ b/tests/ci/version_test.py @@ -0,0 +1,33 @@ +#!/usr/bin/env python + +import unittest +from argparse import ArgumentTypeError + +import version_helper as vh + + +class TestFunctions(unittest.TestCase): + def test_version_arg(self): + cases = ( + ("0.0.0.0", vh.get_version_from_string("0.0.0.0")), + ("1.1.1.2", vh.get_version_from_string("1.1.1.2")), + ("v1.1.1.2-lts", vh.get_version_from_string("1.1.1.2")), + ("v1.1.1.2-prestable", vh.get_version_from_string("1.1.1.2")), + ("v1.1.1.2-stable", vh.get_version_from_string("1.1.1.2")), + ("v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), + ("refs/tags/v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), + ) + for case in cases: + version = vh.version_arg(case[0]) + self.assertEqual(case[1], version) + error_cases = ( + "0.0.0", + "1.1.1.a", + "1.1.1.1.1", + "1.1.1.2-testing", + "v1.1.1.2-testin", + "refs/tags/v1.1.1.2-testin", + ) + for case in error_cases: + with self.assertRaises(ArgumentTypeError): + version = vh.version_arg(case[0]) From a071bde59e2aa1290f5aeddb44cfcedc44c79dd6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 7 Apr 2022 19:56:31 +0800 Subject: [PATCH 301/372] Use storage_snapshot for projection analysis --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index eacec8f50e5..80ad6a4fd67 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5176,7 +5176,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); } - auto parts = getDataPartsVector(); + + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + const auto & parts = snapshot_data.parts; // If minmax_count_projection is a valid candidate, check its completeness. if (minmax_count_projection_candidate) From ebfdadd6f8491fba128af05194213d951935df88 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Apr 2022 12:05:21 +0000 Subject: [PATCH 302/372] Fix firmatting. --- tests/integration/test_storage_kafka/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index a74a045b9a3..a27b5a134e4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1146,6 +1146,7 @@ def test_kafka_read_consumers_in_parallel(kafka_cluster): kafka_create_topic(admin_client, topic_name, num_partitions=8) cancel = threading.Event() + def produce(): while not cancel.is_set(): messages = [] @@ -1188,14 +1189,14 @@ def test_kafka_read_consumers_in_parallel(kafka_cluster): "kafka.*Polled batch of [0-9]+.*read_consumers_in_parallel", repetitions=64, look_behind_lines=100, - timeout=30 # we should get 64 polls in ~8 seconds, but when read sequentially it will take more than 64 sec + timeout=30, # we should get 64 polls in ~8 seconds, but when read sequentially it will take more than 64 sec ) cancel.set() kafka_thread.join() instance.query( - """ + """ DROP TABLE test.consumer; DROP TABLE test.view; DROP TABLE test.kafka; From 1c783ed88abbad3b144d942aa3a035faf5b27d6c Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Apr 2022 12:17:48 +0000 Subject: [PATCH 303/372] Resolve conflicts --- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 11 +++-------- .../02149_schema_inference.reference | 8 ++++---- .../02240_tskv_schema_inference_bug.reference | 8 ++++---- ...2247_names_order_in_json_and_tskv.reference | 18 +++++++++--------- 4 files changed, 20 insertions(+), 25 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 798cb69d759..854aefc7562 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -181,15 +181,10 @@ bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & return true; } -JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) +JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader( + ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( - in_, - format_settings_ - with_names_, - with_types_, - &reader, - nullptr, - format_settings_.json.read_bools_as_numbers) + in_, format_settings_, with_names_, with_types_, &reader, nullptr, format_settings_.json.read_bools_as_numbers) , reader(in_, yield_strings_, format_settings_) { } diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index fe87552e776..2d7dd5caca7 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -75,11 +75,11 @@ c Array(Nullable(Float64)) TSKV a Nullable(Float64) b Nullable(String) -c Array(Nullable(Float64)) -1 s1 \N +c Array(Nullable(Float64)) +1 s1 [] 2 } [2] -\N \N \N -\N \N \N +\N \N [] +\N \N [] \N \N [3] Values c1 Nullable(Float64) diff --git a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference index 5467b32ffb0..d0ced74f8f6 100644 --- a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference +++ b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.reference @@ -1,8 +1,8 @@ a Nullable(Float64) b Nullable(String) -c Array(Nullable(Float64)) -1 s1 \N +c Array(Nullable(Float64)) +1 s1 [] 2 } [2] -\N \N \N -\N \N \N +\N \N [] +\N \N [] \N \N [3] diff --git a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference index 49a285dc11a..300846c17a0 100644 --- a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference +++ b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.reference @@ -1,15 +1,15 @@ -a Nullable(String) +a Nullable(Float64) b Nullable(String) -c Nullable(String) -1 s1 \N +c Array(Nullable(Float64)) +1 s1 [] 2 } [2] -\N \N \N -\N \N \N +\N \N [] +\N \N [] \N \N [3] -b Nullable(String) -a Nullable(String) -c Nullable(String) -e Nullable(String) +b Nullable(Float64) +a Nullable(Float64) +c Nullable(Float64) +e Nullable(Float64) 1 \N \N \N \N 2 3 \N \N \N \N \N From e9de38c52bc876b43ab44458f5cd1cede1a7ceba Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 20:45:07 +0800 Subject: [PATCH 304/372] fix bug --- src/Storages/Hive/StorageHive.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 659bf44e3f4..90e5dd59722 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -308,6 +308,8 @@ StorageHive::StorageHive( storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment_); + storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); + setInMemoryMetadata(storage_metadata); } @@ -517,7 +519,15 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( { auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); if (hive_file) + { + LOG_TRACE( + log, + "Append hive file {} from partition {}, prune_level:{}", + hive_file->getPath(), + boost::join(partition.values, ","), + pruneLevelToString(prune_level)); hive_files.push_back(hive_file); + } } return hive_files; } @@ -535,11 +545,11 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( ContextPtr context_, PruneLevel prune_level) const { - LOG_TRACE(log, "create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + LOG_TRACE(log, "Create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' - if (startsWith(filename, ".") == 0) + if (startsWith(filename, ".")) return {}; auto hive_file = createHiveFile( @@ -553,12 +563,12 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( storage_settings, context_); - /// Load file level minmax index and apply if (prune_level >= PruneLevel::File) { const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { + /// Load file level minmax index and apply hive_file->loadFileMinMaxIndex(); if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) @@ -574,9 +584,9 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( if (prune_level >= PruneLevel::Split) { - /// Load sub-file level minmax index and apply if (hive_file->useSplitMinMaxIndex()) { + /// Load sub-file level minmax index and apply std::unordered_set skip_splits; hive_file->loadSplitMinMaxIndex(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); From 87507ec9e8e428d00395ecc6b1e27d3d4119cbdc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 7 Apr 2022 20:52:54 +0800 Subject: [PATCH 305/372] fix conflicts --- src/Storages/Hive/HiveFile.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 1d965350e16..7b85a2f4f91 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -190,10 +190,6 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c return idx; } -<<<<<<< HEAD -======= - ->>>>>>> master void HiveORCFile::loadFileMinMaxIndex() { if (!reader) From c5ffbc688e27385cc5b79f1eb0444f03bc4f254b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Apr 2022 10:52:44 +0300 Subject: [PATCH 306/372] Do not allow SETTINGS after FORMAT for INSERT queries Parsing SETTINGS after FORMAT, that has been introduced in [1], can interpret SETTING as some values, which is misleading. [1]: https://github.com/ClickHouse/ClickHouse/pull/4174/files#diff-ba7bd0657630b1cd94cf6ed364bd857338096f49f66dc82918438d6745753775R106 Note, that we are touching only INSERT queries, not SELECT, since this is a backward incompatible change, and in case of modifying SELECT it can break too much. Signed-off-by: Azat Khuzhin Fixes: #35100 Fixes: #20343 --- src/Parsers/ASTInsertQuery.cpp | 59 ++++++++++++------- src/Parsers/ParserInsertQuery.cpp | 24 ++++---- .../test_format_schema_on_server/test.py | 2 +- tests/integration/test_storage_s3/test.py | 10 ++-- .../00825_protobuf_format_array_3dim.sh | 2 +- .../00825_protobuf_format_array_of_arrays.sh | 2 +- .../00825_protobuf_format_enum_mapping.sh | 2 +- .../0_stateless/00825_protobuf_format_map.sh | 2 +- .../00825_protobuf_format_nested_in_nested.sh | 2 +- .../00825_protobuf_format_nested_optional.sh | 2 +- ...825_protobuf_format_no_length_delimiter.sh | 2 +- .../00825_protobuf_format_persons.sh | 8 +-- ...rotobuf_format_skipped_column_in_nested.sh | 2 +- .../00825_protobuf_format_splitted_nested.sh | 2 +- .../00825_protobuf_format_squares.sh | 2 +- .../00825_protobuf_format_table_default.sh | 2 +- .../00851_http_insert_json_defaults.sh | 8 +-- .../00900_orc_arrow_parquet_nested.sh | 8 +-- .../00938_template_input_format.sh | 20 ++++--- .../01014_format_custom_separated.sh | 5 +- .../0_stateless/01085_regexp_input_format.sh | 8 +-- ...1086_regexp_input_format_skip_unmatched.sh | 2 +- .../01183_custom_separated_format_http.sh | 6 +- .../0_stateless/01593_insert_settings.sql | 5 +- .../0_stateless/02010_lc_native.python | 10 ++-- .../queries/0_stateless/02030_capnp_format.sh | 21 ++++--- ...7_custom_separated_with_names_and_types.sh | 45 +++++++++----- .../0_stateless/02127_plus_before_float.sh | 2 +- .../0_stateless/02129_skip_quoted_fields.sh | 2 +- .../02134_async_inserts_formats.sh | 8 +-- .../02147_arrow_duplicate_columns.sh | 2 +- .../02155_csv_with_strings_with_slash.sh | 4 +- .../02180_insert_into_values_settings.sql | 2 +- .../02193_async_insert_tcp_client_1.sql | 6 +- .../02193_async_insert_tcp_client_2.sh | 4 +- .../02240_protobuflist_format_persons.sh | 8 +-- .../0_stateless/02241_parquet_bad_column.sh | 2 +- .../02242_case_insensitive_column_matching.sh | 6 +- .../02242_case_insensitive_nested.sh | 2 +- .../02263_format_insert_settings.reference | 58 ++++++++++++++++++ .../02263_format_insert_settings.sh | 38 ++++++++++++ ...7_parallel_parsing_with_names_and_types.sh | 4 +- 42 files changed, 272 insertions(+), 139 deletions(-) create mode 100644 tests/queries/0_stateless/02263_format_insert_settings.reference create mode 100755 tests/queries/0_stateless/02263_format_insert_settings.sh diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index 40e14c918ff..3fa6a6ed1a9 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -94,33 +94,48 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s << quoteString(compression->as().value.safeGet()); } - if (select) - { - settings.ostr << " "; - select->formatImpl(settings, state, frame); - } - else if (watch) - { - settings.ostr << " "; - watch->formatImpl(settings, state, frame); - } - else - { - if (!format.empty()) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORMAT " << (settings.hilite ? hilite_none : "") << format; - } - else if (!infile) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " VALUES" << (settings.hilite ? hilite_none : ""); - } - } - if (settings_ast) { settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : ""); settings_ast->formatImpl(settings, state, frame); } + + /// Compatibility for INSERT w/o SETTINGS to format in oneline, i.e.: + /// + /// INSERT INTO foo VALUES + /// + /// But + /// + /// INSERT INTO foo + /// SETTINGS max_threads=1 + /// VALUES + /// + char delim = settings_ast ? settings.nl_or_ws : ' '; + + if (select) + { + settings.ostr << delim; + select->formatImpl(settings, state, frame); + } + else if (watch) + { + settings.ostr << delim; + watch->formatImpl(settings, state, frame); + } + + if (!select && !watch) + { + if (!format.empty()) + { + settings.ostr << delim + << (settings.hilite ? hilite_keyword : "") << "FORMAT " << (settings.hilite ? hilite_none : "") << format; + } + else if (!infile) + { + settings.ostr << delim + << (settings.hilite ? hilite_keyword : "") << "VALUES" << (settings.hilite ? hilite_none : ""); + } + } } void ASTInsertQuery::updateTreeHashImpl(SipHash & hash_state) const diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index 44db07278c2..b77c0bf5709 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -130,8 +130,17 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - Pos before_values = pos; + /// Read SETTINGS if they are defined + if (s_settings.ignore(pos, expected)) + { + /// Settings are written like SET query, so parse them with ParserSetQuery + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, settings_ast, expected)) + return false; + } + String format_str; + Pos before_values = pos; /// VALUES or FORMAT or SELECT or WITH or WATCH. /// After FROM INFILE we expect FORMAT, SELECT, WITH or nothing. @@ -177,19 +186,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// Read SETTINGS if they are defined - if (s_settings.ignore(pos, expected)) - { - /// Settings are written like SET query, so parse them with ParserSetQuery - ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, settings_ast, expected)) - return false; - /// In case of INSERT INTO ... VALUES SETTINGS ... (...), (...), ... - /// we should move data pointer after all settings. - if (data != nullptr) - data = pos->begin; - } - if (select) { /// Copy SETTINGS from the INSERT ... SELECT ... SETTINGS diff --git a/tests/integration/test_format_schema_on_server/test.py b/tests/integration/test_format_schema_on_server/test.py index 7001d53ccf2..0b7d8837ad3 100644 --- a/tests/integration/test_format_schema_on_server/test.py +++ b/tests/integration/test_format_schema_on_server/test.py @@ -29,7 +29,7 @@ def create_simple_table(): def test_protobuf_format_input(started_cluster): create_simple_table() instance.http_query( - "INSERT INTO test.simple FORMAT Protobuf SETTINGS format_schema='simple:KeyValuePair'", + "INSERT INTO test.simple SETTINGS format_schema='simple:KeyValuePair' FORMAT Protobuf", "\x07\x08\x01\x12\x03abc\x07\x08\x02\x12\x03def", ) assert instance.query("SELECT * from test.simple") == "1\tabc\n2\tdef\n" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 71371f13d1e..e32ddd2782b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -162,7 +162,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" put_query = f"""insert into table function s3('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{filename}', - {maybe_auth}'CSV', '{table_format}', '{compression}') values settings s3_truncate_on_insert=1 {values}""" + {maybe_auth}'CSV', '{table_format}', '{compression}') settings s3_truncate_on_insert=1 values {values}""" try: run_query(instance, put_query) @@ -362,7 +362,7 @@ def test_put_csv(started_cluster, maybe_auth, positive): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV settings s3_truncate_on_insert=1".format( + put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') settings s3_truncate_on_insert=1 format CSV".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, @@ -392,7 +392,7 @@ def test_put_get_with_redirect(started_cluster): values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" values_csv = "1,1,1\n1,1,1\n11,11,11\n" filename = "test.csv" - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format( started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, @@ -431,7 +431,7 @@ def test_put_with_zero_redirect(started_cluster): filename = "test.csv" # Should work without redirect - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, @@ -442,7 +442,7 @@ def test_put_with_zero_redirect(started_cluster): run_query(instance, query) # Should not work with redirect - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') settings s3_truncate_on_insert=1 values {}".format( started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh index 5c2804bdcae..3cd842a10ba 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh @@ -31,7 +31,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_3dim:ABC' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM array_3dim_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh b/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh index bd208195acc..76c5a63c4f2 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh @@ -36,7 +36,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM array_of_arrays_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh index 8d9e2689e26..1258230610d 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh @@ -33,7 +33,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_map.sh b/tests/queries/0_stateless/00825_protobuf_format_map.sh index 2a84772bc9f..81d1cf2e305 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_map.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_map.sh @@ -34,7 +34,7 @@ hexdump -C $BINARY_FILE_PATH # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO map_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_map:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO map_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_map:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM map_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh index f1567128cf4..b0a16c2fbba 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh @@ -30,7 +30,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO nested_in_nested_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO nested_in_nested_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM nested_in_nested_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh b/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh index 1b94ebd79f2..cf9c47f5ea9 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh @@ -37,7 +37,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_optional:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_optional:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM nested_optional_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh index a1bbdc318d5..0f168c38395 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh @@ -38,7 +38,7 @@ echo echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_no_length_delimiter_protobuf_00825 AS no_length_delimiter_protobuf_00825" -$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_no_length_delimiter_protobuf_00825 FORMAT ProtobufSingle SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_no_length_delimiter_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message' FORMAT ProtobufSingle" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_no_length_delimiter_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_persons.sh b/tests/queries/0_stateless/00825_protobuf_format_persons.sh index 465b27aa683..df8b149d7be 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_persons.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_persons.sh @@ -68,7 +68,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_00825 AS persons_00825" -$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:Person' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_00825 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -82,7 +82,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_00825 AS persons_00825" -$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:AltPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:AltPerson' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_00825 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -96,7 +96,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_00825 AS persons_00825" -$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:StrPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:StrPerson' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_00825 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -110,7 +110,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_00825 AS persons_00825" -$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_00825 ORDER BY name" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh index ed35df5e98b..1c1dde82b4a 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh @@ -48,7 +48,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO table_skipped_column_in_nested_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO table_skipped_column_in_nested_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM table_skipped_column_in_nested_00825 ORDER BY unused1" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_splitted_nested.sh b/tests/queries/0_stateless/00825_protobuf_format_splitted_nested.sh index 0cf33c91465..f4525ba609c 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_splitted_nested.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_splitted_nested.sh @@ -67,7 +67,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO splitted_nested_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_splitted_nested:Some'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO splitted_nested_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_splitted_nested:Some' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM splitted_nested_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_squares.sh b/tests/queries/0_stateless/00825_protobuf_format_squares.sh index cc54a62e07f..f04cf60d78b 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_squares.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_squares.sh @@ -28,7 +28,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO squares_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO squares_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM squares_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_table_default.sh b/tests/queries/0_stateless/00825_protobuf_format_table_default.sh index 09271e81ab1..5701ee2cb4d 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_table_default.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_table_default.sh @@ -34,7 +34,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_table_default:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_table_default:Message' FORMAT Protobuf" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM table_default_protobuf_00825 ORDER BY x,y,z" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00851_http_insert_json_defaults.sh b/tests/queries/0_stateless/00851_http_insert_json_defaults.sh index 8838200271f..46523173410 100755 --- a/tests/queries/0_stateless/00851_http_insert_json_defaults.sh +++ b/tests/queries/0_stateless/00851_http_insert_json_defaults.sh @@ -8,10 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS defaults" $CLICKHOUSE_CLIENT --query="CREATE TABLE defaults (x UInt32, y UInt32, a DEFAULT x + y, b Float32 DEFAULT round(log(1 + x + y), 5), c UInt32 DEFAULT 42, e MATERIALIZED x + y, f ALIAS x + y) ENGINE = Memory" -echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20defaults%20FORMAT%20JSONEachRow%20SETTINGS%20input_format_defaults_for_omitted_fields=1" --data-binary @- -echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @- -echo -ne '{"x":3, "y":3, "a":3, "b":3, "c":3}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @- -echo -ne '{"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+FORMAT+JSONEachRow+SETTINGS+input_format_defaults_for_omitted_fields=1" --data-binary @- +echo -ne '{"x":1, "y":1}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT%20INTO%20defaults%20SETTINGS%20input_format_defaults_for_omitted_fields=1%20FORMAT%20JSONEachRow" --data-binary @- +echo -ne '{"x":2, "y":2, "c":2}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @- +echo -ne '{"x":3, "y":3, "a":3, "b":3, "c":3}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @- +echo -ne '{"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6}\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database=${CLICKHOUSE_DATABASE}&query=INSERT+INTO+defaults+SETTINGS+input_format_defaults_for_omitted_fields=1+FORMAT+JSONEachRow" --data-binary @- $CLICKHOUSE_CLIENT --query="SELECT * FROM defaults ORDER BY x, y FORMAT JSONEachRow" $CLICKHOUSE_CLIENT --query="DROP TABLE defaults" diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh index a5294f06272..e07c8fcff09 100755 --- a/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_nested.sh @@ -20,17 +20,15 @@ for ((i = 0; i < 3; i++)) do echo ${formats[i]} ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_table" - cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1" + cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1 FORMAT ${formats[i]}" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_table" ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_nested_table" - cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1" - + cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1 FORMAT ${formats[i]}" + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_nested_table" - - done ${CLICKHOUSE_CLIENT} --query="DROP TABLE nested_table" diff --git a/tests/queries/0_stateless/00938_template_input_format.sh b/tests/queries/0_stateless/00938_template_input_format.sh index 9218f4bebca..e99f59614da 100755 --- a/tests/queries/0_stateless/00938_template_input_format.sh +++ b/tests/queries/0_stateless/00938_template_input_format.sh @@ -22,10 +22,11 @@ cv bn m\", d: 2016-01-01 ; n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ; n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ; n: 789, s1: zx\\ncv\\tbn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'" - $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \ + $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 SETTINGS \ format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \ format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \ -format_template_rows_between_delimiter = ';\n'"; +format_template_rows_between_delimiter = ';\n' \ +FORMAT Template"; $CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV"; @@ -33,10 +34,11 @@ echo "==== parse json (sophisticated template) ====" echo -ne '{${:}"meta"${:}:${:}[${:}{${:}"name"${:}:${:}"s1"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s2"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s3"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"s4"${:},${:}"type"${:}:${:}"String"${:}}${:},${:}{${:}"name"${:}:${:}"n"${:},${:}"type"${:}:${:}"UInt64"${:}}${:},${:}{${:}"name"${:}:${:}"d"${:},${:}"type"${:}:${:}"Date"${:}}${:}]${:},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:}${:CSV}${:},${:}"statistics"${:}:${:}{${:}"elapsed"${:}:${:}${:CSV}${:},${:}"rows_read"${:}:${:}${:CSV}${:},${:}"bytes_read"${:}:${:}${:CSV}${:}}${:}}' > "$CURDIR"/00938_template_input_format_resultset.tmp echo -ne '{${:}"s1"${:}:${:}${s1:JSON}${:},${:}"s2"${:}:${:}${s2:JSON}${:},${:}"s3"${:}:${:}${s3:JSON}${:},${:}"s4"${:}:${:}${s4:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > "$CURDIR"/00938_template_input_format_row.tmp -$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \ +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 SETTINGS \ format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \ format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \ -format_template_rows_between_delimiter = ','"; +format_template_rows_between_delimiter = ',' \ +FORMAT TemplateIgnoreSpaces"; $CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template2"; @@ -45,10 +47,11 @@ echo "==== parse json ====" echo -ne '{${:}"meta"${:}:${:JSON},${:}"data"${:}:${:}[${data}]${:},${:}"rows"${:}:${:JSON},${:}"statistics"${:}:${:JSON}${:}}' > "$CURDIR"/00938_template_input_format_resultset.tmp echo -ne '{${:}"s1"${:}:${:}${s3:JSON}${:},${:}"s2"${:}:${:}${:JSON}${:},${:}"s3"${:}:${:}${s1:JSON}${:},${:}"s4"${:}:${:}${:JSON}${:},${:}"n"${:}:${:}${n:JSON}${:},${:}"d"${:}:${:}${d:JSON}${:}${:}}' > "$CURDIR"/00938_template_input_format_row.tmp -$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \ +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 SETTINGS \ format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \ format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \ -format_template_rows_between_delimiter = ','"; +format_template_rows_between_delimiter = ',' \ +FORMAT TemplateIgnoreSpaces"; $CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV"; @@ -66,10 +69,11 @@ cv bn m\", d: 2016-01-01 ; n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ; n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ; n: 789, s1: zx\cv\bn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'" - $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \ + $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 SETTINGS \ format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \ format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \ -format_template_rows_between_delimiter = ';\n'"; +format_template_rows_between_delimiter = ';\n' \ +FORMAT Template"; $CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV"; diff --git a/tests/queries/0_stateless/01014_format_custom_separated.sh b/tests/queries/0_stateless/01014_format_custom_separated.sh index 42599bcc944..4e88419d125 100755 --- a/tests/queries/0_stateless/01014_format_custom_separated.sh +++ b/tests/queries/0_stateless/01014_format_custom_separated.sh @@ -23,12 +23,13 @@ echo '0, "2019-09-24", "hello" 1, 2019-09-25, "world" 2, "2019-09-26", custom 3, 2019-09-27, separated -end' | $CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated FORMAT CustomSeparated SETTINGS \ +end' | $CLICKHOUSE_CLIENT --query="INSERT INTO custom_separated SETTINGS \ format_custom_escaping_rule = 'CSV', \ format_custom_field_delimiter = ', ', \ format_custom_row_after_delimiter = '\n', \ format_custom_row_between_delimiter = '', \ -format_custom_result_after_delimiter = 'end\n'" +format_custom_result_after_delimiter = 'end\n' +FORMAT CustomSeparated" $CLICKHOUSE_CLIENT --query="SELECT * FROM custom_separated ORDER BY n FORMAT CSV" diff --git a/tests/queries/0_stateless/01085_regexp_input_format.sh b/tests/queries/0_stateless/01085_regexp_input_format.sh index 5736d031c08..217a2fbe8b7 100755 --- a/tests/queries/0_stateless/01085_regexp_input_format.sh +++ b/tests/queries/0_stateless/01085_regexp_input_format.sh @@ -9,19 +9,19 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, array Array(UInt32), echo 'id: 1 array: [1,2,3] string: str1 date: 2020-01-01 id: 2 array: [1,2,3] string: str2 date: 2020-01-02 -id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped'"; +id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped' FORMAT Regexp "; echo 'id: 4 array: "[1,2,3]" string: "str4" date: "2020-01-04" id: 5 array: "[1,2,3]" string: "str5" date: "2020-01-05" -id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV'"; +id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV' FORMAT Regexp"; echo "id: 7 array: [1,2,3] string: 'str7' date: '2020-01-07' id: 8 array: [1,2,3] string: 'str8' date: '2020-01-08' -id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted'"; +id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted' FORMAT Regexp"; echo 'id: 10 array: [1,2,3] string: "str10" date: "2020-01-10" id: 11 array: [1,2,3] string: "str11" date: "2020-01-11" -id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON'"; +id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON' FORMAT Regexp"; $CLICKHOUSE_CLIENT --query="SELECT * FROM regexp ORDER BY id"; $CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; diff --git a/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh b/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh index c96aed7d3ee..8db27891006 100755 --- a/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh +++ b/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGIN echo 'id: 1 string: str1 id: 2 string: str2 id=3, string=str3 -id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1"; +id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1 FORMAT Regexp"; $CLICKHOUSE_CLIENT --query="SELECT * FROM regexp"; $CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; diff --git a/tests/queries/0_stateless/01183_custom_separated_format_http.sh b/tests/queries/0_stateless/01183_custom_separated_format_http.sh index f981ef5b890..8eaa22f4ecc 100755 --- a/tests/queries/0_stateless/01183_custom_separated_format_http.sh +++ b/tests/queries/0_stateless/01183_custom_separated_format_http.sh @@ -6,9 +6,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo 'DROP TABLE IF EXISTS mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- echo 'CREATE TABLE mydb (datetime String, d1 String, d2 String ) ENGINE=Memory' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- -echo "2021-Jan^d1^d2" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @- -echo -n "" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @- +echo "2021-Jan^d1^d2" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @- +echo -n "" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @- echo 'SELECT * FROM mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- -printf "2021-Jan^d1^d2\n%.0s" {1..999999} | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20FORMAT%20CustomSeparated%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27" --data-binary @- +printf "2021-Jan^d1^d2\n%.0s" {1..999999} | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT%20INTO%20mydb%20SETTINGS%20format_custom_escaping_rule%3D%27CSV%27%2C%20format_custom_field_delimiter%20%3D%20%27%5E%27%20FORMAT%20CustomSeparated" --data-binary @- echo 'SELECT count(*), countDistinct(datetime, d1, d2) FROM mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- echo 'DROP TABLE mydb' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- diff --git a/tests/queries/0_stateless/01593_insert_settings.sql b/tests/queries/0_stateless/01593_insert_settings.sql index 7ef49f54049..88a58b2152e 100644 --- a/tests/queries/0_stateless/01593_insert_settings.sql +++ b/tests/queries/0_stateless/01593_insert_settings.sql @@ -2,9 +2,8 @@ drop table if exists data_01593; create table data_01593 (key Int) engine=MergeTree() order by key partition by key; insert into data_01593 select * from numbers_mt(10); --- TOO_MANY_PARTS error -insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1; -- { serverError 252 } +insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1; -- { serverError TOO_MANY_PARTS } -- settings for INSERT is prefered -insert into data_01593 select * from numbers_mt(10) settings max_partitions_per_insert_block=1 settings max_partitions_per_insert_block=100; +insert into data_01593 settings max_partitions_per_insert_block=100 select * from numbers_mt(10) settings max_partitions_per_insert_block=1; drop table data_01593; diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index 71965512e64..e6d6f9e1317 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -143,7 +143,7 @@ def sendQuery(s, query): writeStringBinary('', ba) # No interserver secret writeVarUInt(2, ba) # Stage - Complete ba.append(0) # No compression - writeStringBinary(query + ' settings input_format_defaults_for_omitted_fields=0', ba) # query, finally + writeStringBinary(query, ba) # query, finally s.sendall(ba) @@ -205,7 +205,7 @@ def insertValidLowCardinalityRow(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) # external tables sendEmptyBlock(s) @@ -241,7 +241,7 @@ def insertLowCardinalityRowWithIndexOverflow(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) # external tables sendEmptyBlock(s) @@ -275,7 +275,7 @@ def insertLowCardinalityRowWithIncorrectDictType(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) # external tables sendEmptyBlock(s) @@ -308,7 +308,7 @@ def insertLowCardinalityRowWithIncorrectAdditionalKeys(): s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) sendHello(s) receiveHello(s) - sendQuery(s, 'insert into {}.tab format TSV'.format(CLICKHOUSE_DATABASE)) + sendQuery(s, 'insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV'.format(CLICKHOUSE_DATABASE)) # external tables sendEmptyBlock(s) diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index aa2fe6c1b35..cdc1587bccd 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -19,7 +19,8 @@ cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_simple_types"; $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_simple_types (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixed FixedString(5), data String, date Date, datetime DateTime, datetime64 DateTime64(3)) ENGINE=Memory" $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types values (-1, 1, -1000, 1000, -10000000, 1000000, -1000000000, 1000000000, 123.123, 123123123.123123123, 'Some string', 'fixed', 'Some data', '2000-01-06', '2000-06-01 19:42:42', '2000-04-01 11:21:33.123')" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_simple_types" @@ -27,7 +28,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_simple_types" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_tuples" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_tuples (value UInt64, tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)), tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64)))) ENGINE=Memory"; $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples VALUES (1, (2, (3, 4)), (((5))))" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_tuples" @@ -35,7 +37,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_tuples" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_lists" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_lists (value UInt64, list1 Array(UInt64), list2 Array(Array(Array(UInt64)))) ENGINE=Memory"; $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists VALUES (1, [1, 2, 3], [[[1, 2, 3], [4, 5, 6]], [[7, 8, 9], []], []])" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_lists" @@ -43,7 +46,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_lists" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_lists_and_tuples" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_lists_and_tuples (value UInt64, nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64))))))) ENGINE=Memory"; $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples VALUES (1, ((2, [[3, 4], [5, 6], []]), [([[(7, 8), (9, 10)], [(11, 12), (13, 14)], []], [([15, 16, 17]), ([])])]))" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_lists_and_tuples" @@ -51,7 +55,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_lists_and_tuples" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_table" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_table (nested Nested(value UInt64, array Array(UInt64), tuple Tuple(one UInt64, two UInt64))) ENGINE=Memory"; $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table VALUES ([1, 2, 3], [[4, 5, 6], [], [7, 8]], [(9, 10), (11, 12), (13, 14)])" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_table" @@ -59,7 +64,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_table" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nullable" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nullable (nullable Nullable(UInt64), array Array(Nullable(UInt64)), tuple Tuple(nullable Nullable(UInt64))) ENGINE=Memory"; $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable VALUES (1, [1, Null, 2], (1)), (Null, [Null, Null, 42], (Null))" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nullable" @@ -78,7 +84,8 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_low_cardinality" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_low_cardinality (lc1 LowCardinality(String), lc2 LowCardinality(Nullable(String)), lc3 Array(LowCardinality(Nullable(String)))) ENGINE=Memory" $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality VALUES ('one', 'two', ['one', Null, 'two', Null]), ('two', Null, [Null])" -$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" | \ + $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message' FORMAT CapnProto" $CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality" $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_low_cardinality" diff --git a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh index 1285758866d..400bf2a56fa 100755 --- a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh +++ b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh @@ -14,7 +14,8 @@ for format in CustomSeparated CustomSeparatedWithNames CustomSeparatedWithNamesA do echo $format $CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" - $CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02117 FORMAT $format $CUSTOM_SETTINGS" + $CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" done @@ -23,66 +24,80 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE test_02117" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_02117 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | \ + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" TMP_FILE=$CURDIR/test_02117 $CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" > $TMP_FILE -cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | \ + grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" > $TMP_FILE -cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +cat $TMP_FILE | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 $CUSTOM_SETTINGS FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | \ + grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "DROP TABLE test_02117" rm $TMP_FILE diff --git a/tests/queries/0_stateless/02127_plus_before_float.sh b/tests/queries/0_stateless/02127_plus_before_float.sh index b464bedb837..2f0195410eb 100755 --- a/tests/queries/0_stateless/02127_plus_before_float.sh +++ b/tests/queries/0_stateless/02127_plus_before_float.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "create table test_02127 (x Float32, y Float64) engine=Mem for escaping_rule in Quoted JSON Escaped CSV Raw do -echo -e "+42.42\t+42.42" | $CLICKHOUSE_CLIENT -q "insert into test_02127 format CustomSeparated settings format_custom_escaping_rule='$escaping_rule'" +echo -e "+42.42\t+42.42" | $CLICKHOUSE_CLIENT -q "insert into test_02127 settings format_custom_escaping_rule='$escaping_rule' format CustomSeparated" done diff --git a/tests/queries/0_stateless/02129_skip_quoted_fields.sh b/tests/queries/0_stateless/02129_skip_quoted_fields.sh index c1baeb5b8f2..ac702d3c750 100755 --- a/tests/queries/0_stateless/02129_skip_quoted_fields.sh +++ b/tests/queries/0_stateless/02129_skip_quoted_fields.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_02129" $CLICKHOUSE_CLIENT -q "create table test_02129 (x UInt64, y UInt64) engine=Memory()" -QUERY="insert into test_02129 format CustomSeparatedWithNames settings input_format_skip_unknown_fields=1, format_custom_escaping_rule='Quoted'" +QUERY="insert into test_02129 settings input_format_skip_unknown_fields=1, format_custom_escaping_rule='Quoted' format CustomSeparatedWithNames" # Skip string echo -e "'x'\t'trash'\t'y'\n1\t'Some string'\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" diff --git a/tests/queries/0_stateless/02134_async_inserts_formats.sh b/tests/queries/0_stateless/02134_async_inserts_formats.sh index bd102fefe9f..631809e5dc2 100755 --- a/tests/queries/0_stateless/02134_async_inserts_formats.sh +++ b/tests/queries/0_stateless/02134_async_inserts_formats.sh @@ -9,23 +9,23 @@ url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id" -${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparated settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparated 1,\"a\" 2,\"b\" " & -${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparated settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparated 3,\"a\" 4,\"b\" " & -${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparatedWithNames settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparatedWithNames \"id\",\"s\" 5,\"a\" 6,\"b\" " & -${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts FORMAT CustomSeparatedWithNames settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO async_inserts settings format_custom_escaping_rule='CSV', format_custom_field_delimiter=',' FORMAT CustomSeparatedWithNames \"id\",\"s\" 7,\"a\" 8,\"b\" diff --git a/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh index 938b45fee98..548b2ca868b 100755 --- a/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh +++ b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh @@ -26,6 +26,6 @@ GZDATA="H4sIAHTzuWEAA9VTuw3CMBB9+RCsyIULhFIwAC0SJQWZACkNi1CAxCCMwCCMQMEIKdkgPJ8P ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t1" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t1 ( x Int64, y Int64, z Int64 ) ENGINE = Memory" -echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 FORMAT Arrow settings input_format_arrow_allow_missing_columns = true" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||: +echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 settings input_format_arrow_allow_missing_columns = true FORMAT Arrow" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t1" diff --git a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh index ab2577e6138..08d380bf559 100755 --- a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh +++ b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh @@ -10,13 +10,13 @@ ${CLICKHOUSE_CLIENT} --query="create table test_02155_csv (A Int64, S String, D echo "input_format_null_as_default = 1" -cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 1" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 1 FORMAT CSV" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE test_02155_csv" echo "input_format_null_as_default = 0" -cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 0" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv SETTINGS input_format_null_as_default = 0 FORMAT CSV" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" diff --git a/tests/queries/0_stateless/02180_insert_into_values_settings.sql b/tests/queries/0_stateless/02180_insert_into_values_settings.sql index 0a1468070c1..a499ab15f26 100644 --- a/tests/queries/0_stateless/02180_insert_into_values_settings.sql +++ b/tests/queries/0_stateless/02180_insert_into_values_settings.sql @@ -1,4 +1,4 @@ drop table if exists t; create table t (x Bool) engine=Memory(); -insert into t values settings bool_true_representation='да' ('да'); +insert into t settings bool_true_representation='да' values ('да'); drop table t; diff --git a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql index 795a27883e6..7e68beb4b6f 100644 --- a/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql +++ b/tests/queries/0_stateless/02193_async_insert_tcp_client_1.sql @@ -1,12 +1,14 @@ +SET log_queries = 1; + DROP TABLE IF EXISTS t_async_insert_02193_1; CREATE TABLE t_async_insert_02193_1 (id UInt32, s String) ENGINE = Memory; -INSERT INTO t_async_insert_02193_1 FORMAT CSV SETTINGS async_insert = 1 +INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV 1,aaa ; -INSERT INTO t_async_insert_02193_1 FORMAT Values SETTINGS async_insert = 1 (2, 'bbb'); +INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb'); SET async_insert = 1; diff --git a/tests/queries/0_stateless/02193_async_insert_tcp_client_2.sh b/tests/queries/0_stateless/02193_async_insert_tcp_client_2.sh index e620b21ac72..8aeb53d3b87 100755 --- a/tests/queries/0_stateless/02193_async_insert_tcp_client_2.sh +++ b/tests/queries/0_stateless/02193_async_insert_tcp_client_2.sh @@ -9,8 +9,8 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_async_insert_02193_2" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_async_insert_02193_2 (id UInt32, s String) ENGINE = Memory" -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 FORMAT CSV SETTINGS async_insert = 1 1,aaa" -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 FORMAT Values SETTINGS async_insert = 1 (2, 'bbb')" +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 SETTINGS async_insert = 1 FORMAT CSV 1,aaa" +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb')" ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_async_insert_02193_2 VALUES (3, 'ccc')" --async_insert=1 ${CLICKHOUSE_CLIENT} -q 'INSERT INTO t_async_insert_02193_2 FORMAT JSONEachRow {"id": 4, "s": "ddd"}' --async_insert=1 diff --git a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh index dec14b54eb2..637e01b9e63 100755 --- a/tests/queries/0_stateless/02240_protobuflist_format_persons.sh +++ b/tests/queries/0_stateless/02240_protobuflist_format_persons.sh @@ -72,7 +72,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_02240 AS persons_02240" -$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist1_format_persons:Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist1_format_persons:Person' FORMAT ProtobufList" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_02240 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -86,7 +86,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_02240 AS persons_02240" -$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist2_format_persons:AltPerson' FORMAT ProtobufList" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_02240 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -100,7 +100,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format # echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_02240 AS persons_02240" -$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist3_format_persons:StrPerson' FORMAT ProtobufList" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_02240 ORDER BY name" rm "$BINARY_FILE_PATH" @@ -114,7 +114,7 @@ $CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format echo echo "Roundtrip:" $CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_02240 AS persons_02240" -$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_02240 FORMAT ProtobufList SETTINGS format_schema='$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_02240 SETTINGS format_schema='$SCHEMADIR/02240_protobuflist_format_persons_syntax2:Syntax2Person' FORMAT ProtobufList" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_02240 ORDER BY name" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/02241_parquet_bad_column.sh b/tests/queries/0_stateless/02241_parquet_bad_column.sh index 9efd11cbbe1..cfe8c2d0dbe 100755 --- a/tests/queries/0_stateless/02241_parquet_bad_column.sh +++ b/tests/queries/0_stateless/02241_parquet_bad_column.sh @@ -22,7 +22,7 @@ for case_insensitive in "true" "false"; do original_width Nullable(UInt32), original_height Nullable(UInt32)) engine=Memory" - cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT -q "insert into test_02241 format Parquet SETTINGS input_format_parquet_case_insensitive_column_matching=$case_insensitive" + cat $CUR_DIR/data_parquet_bad_column/metadata_0.parquet | $CLICKHOUSE_CLIENT -q "insert into test_02241 SETTINGS input_format_parquet_case_insensitive_column_matching=$case_insensitive format Parquet" $CLICKHOUSE_CLIENT -q "select count() from test_02241" $CLICKHOUSE_CLIENT -q "drop table test_02241" diff --git a/tests/queries/0_stateless/02242_case_insensitive_column_matching.sh b/tests/queries/0_stateless/02242_case_insensitive_column_matching.sh index 8ebf2952ab3..42652615d7d 100755 --- a/tests/queries/0_stateless/02242_case_insensitive_column_matching.sh +++ b/tests/queries/0_stateless/02242_case_insensitive_column_matching.sh @@ -9,7 +9,7 @@ echo "Parquet" DATA_FILE=$CUR_DIR/data_parquet/case_insensitive_column_matching.parquet ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (iD String, scOre Int32) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet SETTINGS input_format_parquet_case_insensitive_column_matching=true" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load SETTINGS input_format_parquet_case_insensitive_column_matching=true FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" ${CLICKHOUSE_CLIENT} --query="drop table parquet_load" @@ -17,7 +17,7 @@ echo "ORC" DATA_FILE=$CUR_DIR/data_orc/case_insensitive_column_matching.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (iD String, sCorE Int32) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_load FORMAT ORC SETTINGS input_format_orc_case_insensitive_column_matching=true" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_load SETTINGS input_format_orc_case_insensitive_column_matching=true FORMAT ORC" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_load" ${CLICKHOUSE_CLIENT} --query="drop table orc_load" @@ -25,6 +25,6 @@ echo "Arrow" DATA_FILE=$CUR_DIR/data_arrow/case_insensitive_column_matching.arrow ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_load (iD String, sCorE Int32) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_load FORMAT Arrow SETTINGS input_format_arrow_case_insensitive_column_matching=true" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_load SETTINGS input_format_arrow_case_insensitive_column_matching=true FORMAT Arrow" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_load" ${CLICKHOUSE_CLIENT} --query="drop table arrow_load" diff --git a/tests/queries/0_stateless/02242_case_insensitive_nested.sh b/tests/queries/0_stateless/02242_case_insensitive_nested.sh index c22f5695dc3..05d7bf4fc8e 100755 --- a/tests/queries/0_stateless/02242_case_insensitive_nested.sh +++ b/tests/queries/0_stateless/02242_case_insensitive_nested.sh @@ -17,7 +17,7 @@ for ((i = 0; i < 3; i++)) do echo ${formats[i]} ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE nested_table" - cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table FORMAT ${formats[i]} SETTINGS input_format_${format_files[i]}_import_nested = 1, input_format_${format_files[i]}_case_insensitive_column_matching = true" + cat $CUR_DIR/data_orc_arrow_parquet_nested/nested_table.${format_files[i]} | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nested_table SETTINGS input_format_${format_files[i]}_import_nested = 1, input_format_${format_files[i]}_case_insensitive_column_matching = true FORMAT ${formats[i]}" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM nested_table" diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference new file mode 100644 index 00000000000..1c189b2cc4b --- /dev/null +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -0,0 +1,58 @@ +insert into foo settings max_threads=1 +Syntax error (query): failed at position 40 (end of query): +insert into foo format tsv settings max_threads=1 +Can't format ASTInsertQuery with data, since data will be lost. +[multi] insert into foo values +INSERT INTO foo FORMAT Values +[oneline] insert into foo values +INSERT INTO foo FORMAT Values +[multi] insert into foo select 1 +INSERT INTO foo SELECT 1 +[oneline] insert into foo select 1 +INSERT INTO foo SELECT 1 +[multi] insert into foo watch bar +INSERT INTO foo WATCH bar +[oneline] insert into foo watch bar +INSERT INTO foo WATCH bar +[multi] insert into foo format tsv +INSERT INTO foo FORMAT tsv +[oneline] insert into foo format tsv +INSERT INTO foo FORMAT tsv +[multi] insert into foo settings max_threads=1 values +INSERT INTO foo +SETTINGS max_threads = 1 +FORMAT Values +[oneline] insert into foo settings max_threads=1 values +INSERT INTO foo SETTINGS max_threads = 1 FORMAT Values +[multi] insert into foo settings max_threads=1 select 1 +INSERT INTO foo +SETTINGS max_threads = 1 +SELECT 1 +[oneline] insert into foo settings max_threads=1 select 1 +INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 +[multi] insert into foo settings max_threads=1 watch bar +INSERT INTO foo +SETTINGS max_threads = 1 +WATCH bar +[oneline] insert into foo settings max_threads=1 watch bar +INSERT INTO foo SETTINGS max_threads = 1 WATCH bar +[multi] insert into foo settings max_threads=1 format tsv +INSERT INTO foo +SETTINGS max_threads = 1 +FORMAT tsv +[oneline] insert into foo settings max_threads=1 format tsv +INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv +[multi] insert into foo select 1 settings max_threads=1 +INSERT INTO foo +SETTINGS max_threads = 1 +SELECT 1 +SETTINGS max_threads = 1 +[oneline] insert into foo select 1 settings max_threads=1 +INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 SETTINGS max_threads = 1 +[multi] insert into foo settings max_threads=1 select 1 settings max_threads=1 +INSERT INTO foo +SETTINGS max_threads = 1 +SELECT 1 +SETTINGS max_threads = 1 +[oneline] insert into foo settings max_threads=1 select 1 settings max_threads=1 +INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 SETTINGS max_threads = 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh new file mode 100755 index 00000000000..830faad9f86 --- /dev/null +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function run_format() +{ + local q="$1" && shift + + echo "$q" + $CLICKHOUSE_FORMAT <<<"$q" +} +function run_format_both() +{ + local q="$1" && shift + + echo "[multi] $q" + $CLICKHOUSE_FORMAT <<<"$q" + echo "[oneline] $q" + $CLICKHOUSE_FORMAT --oneline <<<"$q" +} + +# NOTE: that those queries may work slow, due to stack trace obtaining +run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") + +run_format_both 'insert into foo values' +run_format_both 'insert into foo select 1' +run_format_both 'insert into foo watch bar' +run_format_both 'insert into foo format tsv' + +run_format_both 'insert into foo settings max_threads=1 values' +run_format_both 'insert into foo settings max_threads=1 select 1' +run_format_both 'insert into foo settings max_threads=1 watch bar' +run_format_both 'insert into foo settings max_threads=1 format tsv' +run_format_both 'insert into foo select 1 settings max_threads=1' +run_format_both 'insert into foo settings max_threads=1 select 1 settings max_threads=1' diff --git a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh index 33562918f67..433d51a3036 100755 --- a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh +++ b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh @@ -15,7 +15,7 @@ do echo "$format, false"; $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ - $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0" + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" @@ -25,7 +25,7 @@ do echo "$format, true"; $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ - $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0" + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names SETTINGS input_format_null_as_default=0 FORMAT $format" $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" From 33d99c8ffb02abf9b7eaa1a72da4466464ab648a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Apr 2022 11:09:22 +0300 Subject: [PATCH 307/372] Introduce compatiblity mode for SETTINGS after FORMAT Add allow_settings_after_format_in_insert setting, OFF by default. Signed-off-by: Azat Khuzhin v2: s/parser_settings_after_format_compact/allow_settings_after_format_in_insert/ (suggested by vitlibar) v3: replace ParserSettings with a flag (requested by vitlibar) --- docs/en/operations/settings/settings.md | 27 +++++++++++++++++++ programs/format/Format.cpp | 4 ++- src/Client/ClientBase.cpp | 2 +- src/Core/Settings.h | 1 + src/Interpreters/DDLTask.cpp | 5 ++-- src/Interpreters/executeQuery.cpp | 2 +- src/Parsers/ParserExplainQuery.cpp | 4 +-- src/Parsers/ParserExplainQuery.h | 6 ++++- src/Parsers/ParserInsertQuery.cpp | 25 +++++++++++++++++ src/Parsers/ParserInsertQuery.h | 6 ++++- src/Parsers/ParserQuery.cpp | 4 +-- src/Parsers/ParserQuery.h | 6 ++++- src/Parsers/ParserQueryWithOutput.cpp | 2 +- src/Parsers/ParserQueryWithOutput.h | 8 +++++- src/Parsers/parseQuery.cpp | 5 ++-- src/Parsers/parseQuery.h | 3 ++- src/Server/GRPCServer.cpp | 2 +- src/Server/PostgreSQLHandler.cpp | 5 +++- .../System/StorageSystemDDLWorkerQueue.cpp | 11 +++++--- .../02263_format_insert_settings.reference | 11 ++++++++ .../02263_format_insert_settings.sh | 17 +++++++++--- 21 files changed, 131 insertions(+), 25 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f9996cbfb0b..07abd77fed0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -519,6 +519,33 @@ Possible values: Default value: `1`. +## allow_settings_after_format_in_insert {#allow_settings_after_format_in_insert} + +Control whether `SETTINGS` after `FORMAT` in `INSERT` queries is allowed or not. It is not recommended to use this, since this may interpret part of `SETTINGS` as values. + +Example: + +```sql +INSERT INTO FUNCTION null('foo String') SETTINGS max_threads=1 VALUES ('bar'); +``` + +But the following query will work only with `allow_settings_after_format_in_insert`: + +```sql +SET allow_settings_after_format_in_insert=1; +INSERT INTO FUNCTION null('foo String') VALUES ('bar') SETTINGS max_threads=1; +``` + +Possible values: + +- 0 — Disallow. +- 1 — Allow. + +Default value: `0`. + +!!! note "Warning" + Use this setting only for backward compatibility if your use cases depend on old syntax. + ## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} Enables or disables skipping insertion of extra data. diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 835afcdb2ed..50d85cdd43d 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -54,6 +54,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ("multiquery,n", "allow multiple queries in the same file") ("obfuscate", "obfuscate instead of formatting") ("backslash", "add a backslash at the end of each line of the formatted query") + ("allow_settings_after_format_in_insert", "Allow SETTINGS after FORMAT, but note, that this is not always safe") ("seed", po::value(), "seed (arbitrary string) that determines the result of obfuscation") ; @@ -83,6 +84,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) bool multiple = options.count("multiquery"); bool obfuscate = options.count("obfuscate"); bool backslash = options.count("backslash"); + bool allow_settings_after_format_in_insert = options.count("allow_settings_after_format_in_insert"); if (quiet && (hilite || oneline || obfuscate)) { @@ -154,7 +156,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) const char * pos = query.data(); const char * end = pos + query.size(); - ParserQuery parser(end); + ParserQuery parser(end, allow_settings_after_format_in_insert); do { ASTPtr res = parseQueryAndMovePosition( diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index dced4e60dd2..391d758e6dc 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -268,7 +268,7 @@ void ClientBase::setupSignalHandler() ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const { - ParserQuery parser(end); + ParserQuery parser(end, global_context->getSettings().allow_settings_after_format_in_insert); ASTPtr res; const auto & settings = global_context->getSettingsRef(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2cbfe97cde5..0015fa73784 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -465,6 +465,7 @@ class IColumn; M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ + M(Bool, allow_settings_after_format_in_insert, false, "Allow SETTINGS after FORMAT, but note, that this is not always safe (note: this is a compatibility setting).", 0) \ M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ M(Seconds, periodic_live_view_refresh, DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC, "Interval after which periodically refreshed live view is forced to refresh.", 0) \ M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \ diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index a490d7bed43..476da294789 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -142,10 +142,11 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) { const char * begin = entry.query.data(); const char * end = begin + entry.query.size(); + const auto & settings = context->getSettingsRef(); - ParserQuery parser_query(end); + ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert); String description; - query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth); + query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth); } ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a3349f12f8f..dbd7063f3b3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -423,7 +423,7 @@ static std::tuple executeQueryImpl( String query_table; try { - ParserQuery parser(end); + ParserQuery parser(end, settings.allow_settings_after_format_in_insert); /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index e072f6a14d7..63314452447 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -58,11 +58,11 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserCreateTableQuery create_p; ParserSelectWithUnionQuery select_p; - ParserInsertQuery insert_p(end); + ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert); ASTPtr query; if (kind == ASTExplainQuery::ExplainKind::ParsedAST) { - ParserQuery p(end); + ParserQuery p(end, allow_settings_after_format_in_insert); if (p.parse(pos, query, expected)) explain_query->setExplainedQuery(std::move(query)); else diff --git a/src/Parsers/ParserExplainQuery.h b/src/Parsers/ParserExplainQuery.h index ba30e97a58f..1a415a04dde 100644 --- a/src/Parsers/ParserExplainQuery.h +++ b/src/Parsers/ParserExplainQuery.h @@ -10,11 +10,15 @@ class ParserExplainQuery : public IParserBase { protected: const char * end; + bool allow_settings_after_format_in_insert; const char * getName() const override { return "EXPLAIN"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - explicit ParserExplainQuery(const char* end_) : end(end_) {} + explicit ParserExplainQuery(const char* end_, bool allow_settings_after_format_in_insert_) + : end(end_) + , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + {} }; } diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index b77c0bf5709..b0ca361155f 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -186,6 +186,31 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } + /// Read SETTINGS after FORMAT. + /// + /// Note, that part of SETTINGS can be interpreted as values, + /// hence it is done only under option. + /// + /// Refs: https://github.com/ClickHouse/ClickHouse/issues/35100 + if (allow_settings_after_format_in_insert && s_settings.ignore(pos, expected)) + { + if (settings_ast) + throw Exception("You have SETTINGS before and after FORMAT, " + "this is not allowed. " + "Consider switching to SETTINGS before FORMAT " + "and disable allow_settings_after_format_in_insert.", + ErrorCodes::SYNTAX_ERROR); + + /// Settings are written like SET query, so parse them with ParserSetQuery + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, settings_ast, expected)) + return false; + /// In case of INSERT INTO ... VALUES SETTINGS ... (...), (...), ... + /// we should move data pointer after all settings. + if (data != nullptr) + data = pos->begin; + } + if (select) { /// Copy SETTINGS from the INSERT ... SELECT ... SETTINGS diff --git a/src/Parsers/ParserInsertQuery.h b/src/Parsers/ParserInsertQuery.h index f98e433551d..0d7ce25e09d 100644 --- a/src/Parsers/ParserInsertQuery.h +++ b/src/Parsers/ParserInsertQuery.h @@ -26,11 +26,15 @@ class ParserInsertQuery : public IParserBase { private: const char * end; + bool allow_settings_after_format_in_insert; const char * getName() const override { return "INSERT query"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - explicit ParserInsertQuery(const char * end_) : end(end_) {} + explicit ParserInsertQuery(const char * end_, bool allow_settings_after_format_in_insert_) + : end(end_) + , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + {} }; /** Insert accepts an identifier and an asterisk with variants. diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 7677efd9415..78d8854f298 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -30,8 +30,8 @@ namespace DB bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ParserQueryWithOutput query_with_output_p(end); - ParserInsertQuery insert_p(end); + ParserQueryWithOutput query_with_output_p(end, allow_settings_after_format_in_insert); + ParserInsertQuery insert_p(end, allow_settings_after_format_in_insert); ParserUseQuery use_p; ParserSetQuery set_p; ParserSystemQuery system_p; diff --git a/src/Parsers/ParserQuery.h b/src/Parsers/ParserQuery.h index be72a436be8..a2d4e6e04df 100644 --- a/src/Parsers/ParserQuery.h +++ b/src/Parsers/ParserQuery.h @@ -10,12 +10,16 @@ class ParserQuery : public IParserBase { private: const char * end; + bool allow_settings_after_format_in_insert; const char * getName() const override { return "Query"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - explicit ParserQuery(const char * end_) : end(end_) {} + explicit ParserQuery(const char * end_, bool allow_settings_after_format_in_insert_ = false) + : end(end_) + , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + {} }; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index f1e007948f9..6041f986a49 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -49,7 +49,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowCreateAccessEntityQuery show_create_access_entity_p; ParserShowGrantsQuery show_grants_p; ParserShowPrivilegesQuery show_privileges_p; - ParserExplainQuery explain_p(end); + ParserExplainQuery explain_p(end, allow_settings_after_format_in_insert); ASTPtr query; diff --git a/src/Parsers/ParserQueryWithOutput.h b/src/Parsers/ParserQueryWithOutput.h index 1fd7bec1eea..dba420a077a 100644 --- a/src/Parsers/ParserQueryWithOutput.h +++ b/src/Parsers/ParserQueryWithOutput.h @@ -12,10 +12,16 @@ class ParserQueryWithOutput : public IParserBase { protected: const char * end; + bool allow_settings_after_format_in_insert; + const char * getName() const override { return "Query with output"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + public: - explicit ParserQueryWithOutput(const char * end_) : end(end_) {} + explicit ParserQueryWithOutput(const char * end_, bool allow_settings_after_format_in_insert_ = false) + : end(end_) + , allow_settings_after_format_in_insert(allow_settings_after_format_in_insert_) + {} }; } diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index ed09d648477..af8c9dc58a6 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -386,7 +386,8 @@ std::pair splitMultipartQuery( const std::string & queries, std::vector & queries_list, size_t max_query_size, - size_t max_parser_depth) + size_t max_parser_depth, + bool allow_settings_after_format_in_insert) { ASTPtr ast; @@ -394,7 +395,7 @@ std::pair splitMultipartQuery( const char * pos = begin; /// parser moves pos from begin to the end of current query const char * end = begin + queries.size(); - ParserQuery parser(end); + ParserQuery parser(end, allow_settings_after_format_in_insert); queries_list.clear(); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index d8d7426872b..cc077bbdab2 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -61,6 +61,7 @@ std::pair splitMultipartQuery( const std::string & queries, std::vector & queries_list, size_t max_query_size, - size_t max_parser_depth); + size_t max_parser_depth, + bool allow_settings_after_format_in_insert); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index eeaf5b32a92..7578f8afc1d 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -868,7 +868,7 @@ namespace query_text = std::move(*(query_info.mutable_query())); const char * begin = query_text.data(); const char * end = begin + query_text.size(); - ParserQuery parser(end); + ParserQuery parser(end, settings.allow_settings_after_format_in_insert); ast = parseQuery(parser, begin, end, "", settings.max_query_size, settings.max_parser_depth); /// Choose input format. diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 04e43ed63aa..489c47b3c31 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -275,7 +275,10 @@ void PostgreSQLHandler::processQuery() const auto & settings = session->sessionContext()->getSettingsRef(); std::vector queries; - auto parse_res = splitMultipartQuery(query->query, queries, settings.max_query_size, settings.max_parser_depth); + auto parse_res = splitMultipartQuery(query->query, queries, + settings.max_query_size, + settings.max_parser_depth, + settings.allow_settings_after_format_in_insert); if (!parse_res.second) throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 1df8b43515e..111ea343398 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -71,13 +71,18 @@ static String clusterNameFromDDLQuery(ContextPtr context, const DDLTask & task) { const char * begin = task.entry.query.data(); const char * end = begin + task.entry.query.size(); - String cluster_name; - ParserQuery parser_query(end); + const auto & settings = context->getSettingsRef(); + String description = fmt::format("from {}", task.entry_path); + ParserQuery parser_query(end, settings.allow_settings_after_format_in_insert); ASTPtr query = parseQuery(parser_query, begin, end, description, - context->getSettingsRef().max_query_size, context->getSettingsRef().max_parser_depth); + settings.max_query_size, + settings.max_parser_depth); + + String cluster_name; if (const auto * query_on_cluster = dynamic_cast(query.get())) cluster_name = query_on_cluster->cluster; + return cluster_name; } diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 1c189b2cc4b..721e7960875 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -2,6 +2,17 @@ insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): insert into foo format tsv settings max_threads=1 Can't format ASTInsertQuery with data, since data will be lost. +[multi] insert into foo format tsv settings max_threads=1 +INSERT INTO foo +SETTINGS max_threads = 1 +FORMAT tsv +[oneline] insert into foo format tsv settings max_threads=1 +INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv +insert into foo settings max_threads=1 format tsv settings max_threads=1 +You have SETTINGS before and after FORMAT +Cannot parse input: expected '\n' before: 'settings max_threads=1 1' +1 +You have SETTINGS before and after FORMAT [multi] insert into foo values INSERT INTO foo FORMAT Values [oneline] insert into foo values diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 830faad9f86..3d5f780a38c 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -9,21 +9,32 @@ function run_format() local q="$1" && shift echo "$q" - $CLICKHOUSE_FORMAT <<<"$q" + $CLICKHOUSE_FORMAT "$@" <<<"$q" } function run_format_both() { local q="$1" && shift echo "[multi] $q" - $CLICKHOUSE_FORMAT <<<"$q" + $CLICKHOUSE_FORMAT "$@" <<<"$q" echo "[oneline] $q" - $CLICKHOUSE_FORMAT --oneline <<<"$q" + $CLICKHOUSE_FORMAT --oneline "$@" <<<"$q" } # NOTE: that those queries may work slow, due to stack trace obtaining run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +# compatibility run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") +run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert +run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +# and via server (since this is a separate code path) +$CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' +$CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' +$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'") +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1' +$CLICKHOUSE_CLIENT -q 'select * from data_02263' +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +$CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' run_format_both 'insert into foo select 1' From 44895e5287c27bea160f2371264f9a32ad8b2714 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 16:28:43 +0200 Subject: [PATCH 308/372] Improve build cache for docker images - Save all layer's cache - Use cache with fallback to a latest tags --- tests/ci/docker_images_check.py | 3 ++- tests/ci/docker_test.py | 15 +++++++++------ 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 43671e3a2f7..57227ef307e 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -237,9 +237,10 @@ def build_and_push_one_image( "docker buildx build --builder default " f"--label build-url={GITHUB_RUN_URL} " f"{from_tag_arg}" - f"--build-arg BUILDKIT_INLINE_CACHE=1 " f"--tag {image.repo}:{version_string} " f"--cache-from type=registry,ref={image.repo}:{version_string} " + f"--cache-from type=registry,ref={image.repo}:latest " + f"--cache-to type=inline,mode=max " f"{push_arg}" f"--progress plain {image.full_path}" ) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 0d1c554026f..8a0c4a0dfce 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -124,8 +124,9 @@ class TestDockerImageCheck(unittest.TestCase): self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version --cache-from " - "type=registry,ref=name:version --push --progress plain path", + "--tag name:version --cache-from type=registry,ref=name:version " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --push --progress plain path", mock_popen.call_args.args, ) self.assertTrue(result) @@ -141,8 +142,9 @@ class TestDockerImageCheck(unittest.TestCase): self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version2 " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " - "type=registry,ref=name:version2 --progress plain path", + "--tag name:version2 --cache-from type=registry,ref=name:version2 " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --progress plain path", mock_popen.call_args.args, ) self.assertTrue(result) @@ -157,8 +159,9 @@ class TestDockerImageCheck(unittest.TestCase): mock_machine.assert_not_called() self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " - "type=registry,ref=name:version2 --progress plain path", + "--tag name:version2 --cache-from type=registry,ref=name:version2 " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --progress plain path", mock_popen.call_args.args, ) self.assertFalse(result) From 8111010815cb77f177254a79beda2ac433b5487a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 17:08:56 +0200 Subject: [PATCH 309/372] Rebuild a builder for test --- docker/docs/builder/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 906312a19a2..061251aa7f0 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -1,4 +1,3 @@ -# rebuild in #33610 # docker build -t clickhouse/docs-builder . FROM ubuntu:20.04 From 9fa6e6e7dfd3499e147718520b66f49d8fb46d79 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 7 Apr 2022 17:42:33 +0200 Subject: [PATCH 310/372] Make backward compatibility check always green --- docker/test/stress/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index ba85999caa5..27d5f9c4be4 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -330,7 +330,7 @@ then -e "UNFINISHED" \ -e "Renaming unexpected part" \ /var/log/clickhouse-server/clickhouse-server.backward.*.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_error_messages.txt if it's empty @@ -346,7 +346,7 @@ then # OOM zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ - && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Logical errors @@ -366,7 +366,7 @@ then # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_fatal_messages.txt if it's empty From 2e2e2b91904e2e6c50cf10aad9431607d9a160a9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Apr 2022 18:17:43 +0200 Subject: [PATCH 311/372] fix a race condition --- src/Interpreters/MergeTreeTransaction.cpp | 26 +++++++++++++------ src/Interpreters/MergeTreeTransaction.h | 3 +++ .../01172_transaction_counters.reference | 3 ++- 3 files changed, 23 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index dd235d23bfe..7c1feb579e2 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -248,6 +248,24 @@ bool MergeTreeTransaction::rollback() noexcept 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(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true); + } + + for (const auto & part : parts_to_activate) + if (part->version.getCreationTID() != tid) + const_cast(part->storage).restoreAndActivatePart(part); + /// Kind of optimization: cleanup thread can remove these parts immediately for (const auto & part : parts_to_remove) { @@ -264,14 +282,6 @@ bool MergeTreeTransaction::rollback() noexcept part->version.unlockRemovalTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name}); } - /// 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) - const_cast(part->storage).removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part}, true); - - for (const auto & part : parts_to_activate) - if (part->version.getCreationTID() != tid) - const_cast(part->storage).restoreAndActivatePart(part); assert([&]() { diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index 301434ac712..7ebea450dd0 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -16,6 +16,9 @@ class IMergeTreeDataPart; using DataPartPtr = std::shared_ptr; using DataPartsVector = std::vector; +/// 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, private boost::noncopyable { friend class TransactionLog; diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 3a167e76817..1aabf8a2a38 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -6,7 +6,7 @@ 3 all_1_1_0 0 3 all_3_3_0 1 4 all_1_1_0 1 (0,0,'00000000-0000-0000-0000-000000000000') 0 -4 all_2_2_0 18446744073709551615 (0,0,'00000000-0000-0000-0000-000000000000') 0 +4 all_2_2_0 18446744073709551615 (1,1,'00000000-0000-0000-0000-000000000000') 0 4 all_3_3_0 0 (0,0,'00000000-0000-0000-0000-000000000000') 0 5 1 6 all_1_1_0 0 @@ -19,6 +19,7 @@ 1 1 AddPart 1 1 1 1 all_1_1_0 2 1 Begin 1 1 1 1 2 1 AddPart 1 1 1 1 all_2_2_0 +1 1 LockPart 1 1 1 1 all_2_2_0 2 1 Rollback 1 1 1 1 3 1 Begin 1 1 1 1 3 1 AddPart 1 1 1 1 all_3_3_0 From dd4bd4abba54830e4bbd159aec5b1df1c2b1617c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 18:21:13 +0200 Subject: [PATCH 312/372] Simplify and fix coverity --- .github/workflows/nightly.yml | 15 ++++++++------- docker/packager/binary/build.sh | 4 +++- docker/packager/packager | 11 ++++++----- tests/ci/build_check.py | 7 ------- 4 files changed, 17 insertions(+), 20 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 1e70213adf5..836421f34dd 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -79,13 +79,14 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse + BUILD_NAME=coverity CACHES_PATH=${{runner.temp}}/../ccaches CHECK_NAME=ClickHouse build check (actions) - BUILD_NAME=coverity + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + TEMP_PATH=${{runner.temp}}/build_check EOF + echo "COVERITY_TOKEN=${{ secrets.COVERITY_TOKEN }}" >> "$GITHUB_ENV" - name: Download changed images uses: actions/download-artifact@v2 with: @@ -105,12 +106,12 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" "${{ secrets.COV_TOKEN }}" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" - name: Upload Coverity Analysis if: ${{ success() || failure() }} run: | - curl --form token='${{ secrets.COV_TOKEN }}' \ - --form email='${{ secrets.ROBOT_CLICKHOUSE_EMAIL }}' \ + curl --form token="${COVERITY_TOKEN}" \ + --form email='security+coverity@clickhouse.com' \ --form file="@$TEMP_PATH/$BUILD_NAME/clickhouse-scan.tgz" \ --form version="${GITHUB_REF#refs/heads/}-${GITHUB_SHA::6}" \ --form description="Nighly Scan: $(date +'%Y-%m-%dT%H:%M:%S')" \ diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 269d3eb52c6..2bedb50dd40 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -27,7 +27,9 @@ cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUI if [ "coverity" == "$COMBINED_OUTPUT" ] then - wget --post-data "token=$COV_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /opt/cov-analysis --strip-components 1 + mkdir -p /opt/cov-analysis + + wget --post-data "token=$COVERITY_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /opt/cov-analysis --strip-components 1 export PATH=$PATH:/opt/cov-analysis/bin cov-configure --config ./coverity.config --template --comptype clangcc --compiler "$CC" SCAN_WRAPPER="cov-build --config ./coverity.config --dir cov-int" diff --git a/docker/packager/packager b/docker/packager/packager index 1a79b497fa2..9a72a16bd70 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -86,7 +86,6 @@ def parse_env_variables( additional_pkgs, with_coverage, with_binaries, - coverity_scan, ): DARWIN_SUFFIX = "-darwin" DARWIN_ARM_SUFFIX = "-darwin-aarch64" @@ -179,7 +178,7 @@ def parse_env_variables( cmake_flags.append("-DENABLE_TESTS=0") elif package_type == "coverity": result.append("COMBINED_OUTPUT=coverity") - result.append("COV_TOKEN={}".format(cov_token)) + result.append('COVERITY_TOKEN="$COVERITY_TOKEN"') elif split_binary: result.append("COMBINED_OUTPUT=shared_build") @@ -328,13 +327,16 @@ if __name__ == "__main__": parser.add_argument( "--docker-image-version", default="latest", help="docker image tag to use" ) - parser.add_argument("--cov_token", default="") args = parser.parse_args() if not os.path.isabs(args.output_dir): args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir)) - image_type = "binary" if args.package_type in ("performance", "coverity") else args.package_type + image_type = ( + "binary" + if args.package_type in ("performance", "coverity") + else args.package_type + ) image_name = "clickhouse/binary-builder" if not os.path.isabs(args.clickhouse_repo_path): @@ -376,7 +378,6 @@ if __name__ == "__main__": args.additional_pkgs, args.with_coverage, args.with_binaries, - args.cov_token, ) run_docker_image_with_env( diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 74ebebf1e6f..e9d660156e1 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -55,7 +55,6 @@ def get_packager_cmd( image_version: str, ccache_path: str, official: bool, - cov_token: str, ) -> str: package_type = build_config["package_type"] comp = build_config["compiler"] @@ -88,8 +87,6 @@ def get_packager_cmd( if official: cmd += " --official" - if cov_token: - cmd += " --cov-token={}".format(cov_token) return cmd @@ -206,9 +203,6 @@ def main(): build_check_name = sys.argv[1] build_name = sys.argv[2] - cov_token = "" - if len(sys.argv) > 3: - cov_token = sys.argv[3] build_config = get_build_config(build_check_name, build_name) @@ -303,7 +297,6 @@ def main(): image_version, ccache_path, official_flag, - cov_token, ) logging.info("Going to run packager with %s", packager_cmd) From a5b1a0620c4433d1af58409472351174741f4aa0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 18:22:29 +0200 Subject: [PATCH 313/372] Add default IMAGES_PATH value --- tests/ci/env_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 242d6bf5e40..6462baad729 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -15,7 +15,7 @@ GITHUB_RUN_ID = os.getenv("GITHUB_RUN_ID", "0") GITHUB_SERVER_URL = os.getenv("GITHUB_SERVER_URL", "https://github.com") GITHUB_WORKSPACE = os.getenv("GITHUB_WORKSPACE", git_root) GITHUB_RUN_URL = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}" -IMAGES_PATH = os.getenv("IMAGES_PATH") +IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH) REPORTS_PATH = os.getenv("REPORTS_PATH", p.abspath(p.join(module_dir, "./reports"))) REPO_COPY = os.getenv("REPO_COPY", git_root) RUNNER_TEMP = os.getenv("RUNNER_TEMP", p.abspath(p.join(module_dir, "./tmp"))) From 5dce2f18b5cb8ad9285f35df1122d3a63bc3dcd4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Apr 2022 18:46:46 +0200 Subject: [PATCH 314/372] Better --- src/Common/FileCache.cpp | 8 +- src/Common/FileCache.h | 8 +- src/Common/FileCacheSettings.h | 2 +- src/Common/FileSegment.cpp | 111 +++++++++++------- src/Common/FileSegment.h | 25 +++- src/Disks/DiskCacheWrapper.cpp | 2 + src/Disks/DiskWebServer.h | 2 +- src/Disks/IDiskRemote.cpp | 14 +-- src/Disks/IDiskRemote.h | 4 +- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 12 +- src/Disks/IO/CachedReadBufferFromRemoteFS.h | 2 - src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 20 ---- src/Disks/IO/ThreadPoolRemoteFSReader.h | 1 - src/Disks/S3/DiskS3.cpp | 7 +- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/WriteBufferFromS3.cpp | 33 ++---- src/IO/WriteBufferFromS3.h | 4 +- src/IO/WriteSettings.h | 1 + src/Interpreters/threadPoolCallbackRunner.cpp | 9 +- src/Interpreters/threadPoolCallbackRunner.h | 2 +- .../MergedColumnOnlyOutputStream.cpp | 2 +- .../System/StorageSystemFilesystemCache.cpp | 10 +- .../System/StorageSystemFilesystemCache.h | 4 +- 24 files changed, 139 insertions(+), 148 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 2e05e7a7202..5d5851d0b84 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -57,7 +57,7 @@ String IFileCache::getPathInLocalCache(const Key & key) return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str; } -bool IFileCache::shouldBypassCache() +bool IFileCache::isReadOnly() { return !CurrentThread::isInitialized() || !CurrentThread::get().getQueryContext() @@ -708,7 +708,7 @@ bool LRUFileCache::isLastFileSegmentHolder( return cell->file_segment.use_count() == 2; } -FileSegmentsHolder LRUFileCache::getAll() +FileSegments LRUFileCache::getSnapshot() const { std::lock_guard cache_lock(mutex); @@ -717,10 +717,10 @@ FileSegmentsHolder LRUFileCache::getAll() for (const auto & [key, cells_by_offset] : files) { for (const auto & [offset, cell] : cells_by_offset) - file_segments.push_back(cell.file_segment); + file_segments.push_back(FileSegment::getSnapshot(cell.file_segment)); } - return FileSegmentsHolder(std::move(file_segments)); + return file_segments; } std::vector LRUFileCache::tryGetCachePaths(const Key & key) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 089bdb633c0..e706376bc89 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -44,7 +44,7 @@ public: virtual void tryRemoveAll() = 0; - static bool shouldBypassCache(); + static bool isReadOnly(); /// Cache capacity in bytes. size_t capacity() const { return max_size; } @@ -72,10 +72,10 @@ public: */ virtual FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) = 0; - virtual FileSegmentsHolder getAll() = 0; - virtual FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) = 0; + virtual FileSegments getSnapshot() const = 0; + /// For debug. virtual String dumpStructure(const Key & key) = 0; @@ -124,7 +124,7 @@ public: FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override; - FileSegmentsHolder getAll() override; + FileSegments getSnapshot() const override; FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) override; diff --git a/src/Common/FileCacheSettings.h b/src/Common/FileCacheSettings.h index 53c28400c86..0b34e1e3d82 100644 --- a/src/Common/FileCacheSettings.h +++ b/src/Common/FileCacheSettings.h @@ -2,7 +2,7 @@ #include -namespace Poco { namespace Util { class AbstractConfiguration; }} +namespace Poco { namespace Util { class AbstractConfiguration; } } namespace DB { diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index d8e7a994df4..4def08c6817 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -73,6 +73,12 @@ size_t FileSegment::getDownloadOffset() const return range().left + getDownloadedSize(segment_lock); } +size_t FileSegment::getDownloadedSize() const +{ + std::lock_guard segment_lock(mutex); + return getDownloadedSize(segment_lock); +} + size_t FileSegment::getDownloadedSize(std::lock_guard & /* segment_lock */) const { if (download_state == State::DOWNLOADED) @@ -84,24 +90,15 @@ size_t FileSegment::getDownloadedSize(std::lock_guard & /* segment_l String FileSegment::getCallerId() { - return getCallerIdImpl(false); + return getCallerIdImpl(); } -String FileSegment::getCallerIdImpl(bool allow_non_strict_checking) +String FileSegment::getCallerIdImpl() { - if (IFileCache::shouldBypassCache()) - { - /// getCallerId() can be called from completeImpl(), which can be called from complete(). - /// complete() is called from destructor of CachedReadBufferFromRemoteFS when there is no query id anymore. - /// Allow non strict checking in this case. This works correctly as if getCallerIdImpl() is called from destructor, - /// then we know that caller is not a downloader, because downloader is reset each nextImpl() call either - /// manually or via SCOPE_EXIT. - - if (allow_non_strict_checking) - return "None"; - - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Cannot use cache without query id"); - } + if (!CurrentThread::isInitialized() + || !CurrentThread::get().getQueryContext() + || CurrentThread::getQueryId().size == 0) + return "None:" + toString(getThreadId()); return CurrentThread::getQueryId().toString() + ":" + toString(getThreadId()); } @@ -244,15 +241,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) { std::lock_guard segment_lock(mutex); - auto info = getInfoForLogImpl(segment_lock); - e.addMessage("while writing into cache, info: " + info); + wrapWithCacheInfo(e, "while writing into cache", segment_lock); - LOG_ERROR(log, "Failed to write to cache. File segment info: {}", info); - - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; - - cache_writer->finalize(); - cache_writer.reset(); + setDownloadFailed(segment_lock); cv.notify_all(); @@ -265,7 +256,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) void FileSegment::writeInMemory(const char * from, size_t size) { if (!size) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed"); + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Attempt to write zero size cache file"); if (availableSize() < size) throw Exception( @@ -284,14 +275,13 @@ void FileSegment::writeInMemory(const char * from, size_t size) { cache_writer->write(from, size); } - catch (...) + catch (Exception & e) { - LOG_ERROR(log, "Failed to write to cache. File segment info: {}", getInfoForLogImpl(segment_lock)); + wrapWithCacheInfo(e, "while writing into cache", segment_lock); - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + setDownloadFailed(segment_lock); - cache_writer->finalize(); - cache_writer.reset(); + cv.notify_all(); throw; } @@ -313,23 +303,23 @@ size_t FileSegment::finalizeWrite() { cache_writer->next(); } - catch (...) + catch (Exception & e) { - download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + wrapWithCacheInfo(e, "while writing into cache", segment_lock); - cache_writer->finalize(); - cache_writer.reset(); + setDownloadFailed(segment_lock); + + cv.notify_all(); throw; } downloaded_size += size; - cache_writer.reset(); - downloader_id.clear(); - download_state = State::DOWNLOADED; if (downloaded_size != range().size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} == {}", downloaded_size, range().size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected downloaded size to equal file segment size ({} == {})", downloaded_size, range().size()); + + setDownloaded(segment_lock); return size; } @@ -398,6 +388,20 @@ void FileSegment::setDownloaded(std::lock_guard & /* segment_lock */ { download_state = State::DOWNLOADED; is_downloaded = true; + downloader_id.clear(); + + if (cache_writer) + { + cache_writer->finalize(); + cache_writer.reset(); + remote_file_reader.reset(); + } +} + +void FileSegment::setDownloadFailed(std::lock_guard & /* segment_lock */) +{ + download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; + downloader_id.clear(); if (cache_writer) { @@ -455,7 +459,7 @@ void FileSegment::complete(State state) } catch (...) { - if (!downloader_id.empty() && downloader_id == getCallerIdImpl(true)) + if (!downloader_id.empty() && downloader_id == getCallerIdImpl()) downloader_id.clear(); cv.notify_all(); @@ -480,7 +484,7 @@ void FileSegment::complete(std::lock_guard & cache_lock) /// Segment state can be changed from DOWNLOADING or EMPTY only if the caller is the /// downloader or the only owner of the segment. - bool can_update_segment_state = downloader_id == getCallerIdImpl(true) + bool can_update_segment_state = downloader_id == getCallerIdImpl() || cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); if (can_update_segment_state) @@ -489,11 +493,11 @@ void FileSegment::complete(std::lock_guard & cache_lock) try { - completeImpl(cache_lock, segment_lock, /* allow_non_strict_checking */true); + completeImpl(cache_lock, segment_lock); } catch (...) { - if (!downloader_id.empty() && downloader_id == getCallerIdImpl(true)) + if (!downloader_id.empty() && downloader_id == getCallerIdImpl()) downloader_id.clear(); cv.notify_all(); @@ -503,7 +507,7 @@ void FileSegment::complete(std::lock_guard & cache_lock) cv.notify_all(); } -void FileSegment::completeImpl(std::lock_guard & cache_lock, std::lock_guard & segment_lock, bool allow_non_strict_checking) +void FileSegment::completeImpl(std::lock_guard & cache_lock, std::lock_guard & segment_lock) { bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); @@ -539,7 +543,7 @@ void FileSegment::completeImpl(std::lock_guard & cache_lock, std::lo } } - if (!downloader_id.empty() && (downloader_id == getCallerIdImpl(allow_non_strict_checking) || is_last_holder)) + if (!downloader_id.empty() && (downloader_id == getCallerIdImpl() || is_last_holder)) { LOG_TEST(log, "Clearing downloader id: {}, current state: {}", downloader_id, stateToString(download_state)); downloader_id.clear(); @@ -566,6 +570,11 @@ String FileSegment::getInfoForLogImpl(std::lock_guard & segment_lock return info.str(); } +void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard & segment_lock) const +{ + e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogImpl(segment_lock))); +} + String FileSegment::stateToString(FileSegment::State state) { switch (state) @@ -599,6 +608,22 @@ void FileSegment::assertCorrectnessImpl(std::lock_guard & /* segment assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(cache->getPathInLocalCache(key(), offset())) > 0); } +FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment) +{ + auto snapshot = std::make_shared( + file_segment->offset(), + file_segment->range().size(), + file_segment->key(), + nullptr, + file_segment->state()); + + snapshot->hits_count = file_segment->getHitsCount(); + snapshot->ref_count = file_segment.use_count(); + snapshot->downloaded_size = file_segment->getDownloadedSize(); + + return snapshot; +} + FileSegmentsHolder::~FileSegmentsHolder() { /// In CacheableReadBufferFromRemoteFS file segment's downloader removes file segments from diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index c9e4146c726..a02d8e85a46 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -97,6 +97,11 @@ public: void write(const char * from, size_t size, size_t offset_); + /** + * writeInMemory and finalizeWrite are used together to write a single file with delay. + * Both can be called only once, one after another. Used for writing cache via threadpool + * on wrote operations. TODO: this solution is temporary, until adding a separate cache layer. + */ void writeInMemory(const char * from, size_t size); size_t finalizeWrite(); @@ -121,18 +126,24 @@ public: size_t getDownloadOffset() const; + size_t getDownloadedSize() const; + void completeBatchAndResetDownloader(); void complete(State state); String getInfoForLog() const; - size_t hits() const { return hits_num; } + size_t getHitsCount() const { return hits_count; } - void hit() { ++hits_num; } + size_t getRefCount() const { return ref_count; } + + void incrementHitsCount() { ++hits_count; } void assertCorrectness() const; + static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment); + private: size_t availableSize() const { return reserved_size - downloaded_size; } @@ -141,6 +152,9 @@ private: void assertCorrectnessImpl(std::lock_guard & segment_lock) const; void setDownloaded(std::lock_guard & segment_lock); + void setDownloadFailed(std::lock_guard & segment_lock); + + void wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard & segment_lock) const; bool lastFileSegmentHolder() const; @@ -152,9 +166,9 @@ private: void completeImpl( std::lock_guard & cache_lock, - std::lock_guard & segment_lock, bool allow_non_strict_checking = false); + std::lock_guard & segment_lock); - static String getCallerIdImpl(bool allow_non_strict_checking = false); + static String getCallerIdImpl(); void resetDownloaderImpl(std::lock_guard & segment_lock); @@ -188,7 +202,8 @@ private: bool detached = false; std::atomic is_downloaded{false}; - std::atomic hits_num = 0; /// cache hits. + std::atomic hits_count = 0; /// cache hits. + std::atomic ref_count = 0; /// Used for getting snapshot state }; struct FileSegmentsHolder : private boost::noncopyable diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 178caa0c496..a86f13f55af 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -206,6 +206,8 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode return DiskDecorator::writeFile(path, buf_size, mode, settings); WriteSettings current_settings = settings; + /// There are two different cache implementations. Disable second one if the first is enabled. + /// The firts will soon be removed, this disabling is temporary. current_settings.enable_filesystem_cache_on_write_operations = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); diff --git a/src/Disks/DiskWebServer.h b/src/Disks/DiskWebServer.h index 94ba32939da..6341b582174 100644 --- a/src/Disks/DiskWebServer.h +++ b/src/Disks/DiskWebServer.h @@ -77,7 +77,7 @@ public: UInt64 getTotalSpace() const final override { return std::numeric_limits::max(); } UInt64 getAvailableSpace() const final override { return std::numeric_limits::max(); } -UInt64 getUnreservedSpace() const final override { return std::numeric_limits::max(); } + UInt64 getUnreservedSpace() const final override { return std::numeric_limits::max(); } /// Read-only part diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 0f430e69a5e..fb1c0ddc378 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -343,9 +343,9 @@ void IDiskRemote::removeMetadataRecursive(const String & path, RemoteFSPathKeepe } } -std::vector IDiskRemote::getRemotePaths(const String & path) const +std::vector IDiskRemote::getRemotePaths(const String & local_path) const { - auto metadata = readMetadata(path); + auto metadata = readMetadata(local_path); std::vector remote_paths; for (const auto & [remote_path, _] : metadata.remote_fs_objects) @@ -354,16 +354,16 @@ std::vector IDiskRemote::getRemotePaths(const String & path) const return remote_paths; } -void IDiskRemote::getRemotePathsRecursive(const String & path, std::vector & paths_map) +void IDiskRemote::getRemotePathsRecursive(const String & local_path, std::vector & paths_map) { - if (metadata_disk->isFile(path)) + if (metadata_disk->isFile(local_path)) { - paths_map.emplace_back(path, getRemotePaths(path)); + paths_map.emplace_back(local_path, getRemotePaths(local_path)); } else { - for (auto it = iterateDirectory(path); it->isValid(); it->next()) - IDiskRemote::getRemotePathsRecursive(fs::path(path) / it->name(), paths_map); + for (auto it = iterateDirectory(local_path); it->isValid(); it->next()) + IDiskRemote::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map); } } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 6b16a1f753c..a8a299391bf 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -68,9 +68,9 @@ public: String getCacheBasePath() const final override; - std::vector getRemotePaths(const String & path) const final override; + std::vector getRemotePaths(const String & local_path) const final override; - void getRemotePathsRecursive(const String & path, std::vector & paths_map) override; + void getRemotePathsRecursive(const String & local_path, std::vector & paths_map) override; /// Methods for working with metadata. For some operations (like hardlink /// creation) metadata can be updated concurrently from multiple threads diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index a2e60a1937e..b9f7457447e 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -389,7 +389,7 @@ bool CachedReadBufferFromRemoteFS::completeFileSegmentAndGetNext() implementation_buffer = getImplementationBuffer(*current_file_segment_it); if (read_type == ReadType::CACHED) - (*current_file_segment_it)->hit(); + (*current_file_segment_it)->incrementHitsCount(); LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString()); return true; @@ -573,8 +573,6 @@ bool CachedReadBufferFromRemoteFS::nextImpl() bool CachedReadBufferFromRemoteFS::nextImplStep() { - assertCacheAllowed(); - last_caller_id = FileSegment::getCallerId(); if (!initialized) @@ -623,7 +621,7 @@ bool CachedReadBufferFromRemoteFS::nextImplStep() implementation_buffer = getImplementationBuffer(*current_file_segment_it); if (read_type == ReadType::CACHED) - (*current_file_segment_it)->hit(); + (*current_file_segment_it)->incrementHitsCount(); } assert(!internal_buffer.empty()); @@ -820,12 +818,6 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() return std::nullopt; } -void CachedReadBufferFromRemoteFS::assertCacheAllowed() const -{ - if (IFileCache::shouldBypassCache() && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache used when not allowed"); -} - String CachedReadBufferFromRemoteFS::getInfoForLog() { auto implementation_buffer_read_range_str = diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.h b/src/Disks/IO/CachedReadBufferFromRemoteFS.h index 5d632e62c0f..5fc9ec39246 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.h +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.h @@ -50,8 +50,6 @@ private: bool nextImplStep(); - void assertCacheAllowed() const; - enum class ReadType { CACHED, diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 18c61e1d704..7014b21e8b4 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.enable_filesystem_cache && !IFileCache::shouldBypassCache(); + bool with_cache = cache && settings.enable_filesystem_cache; auto remote_file_reader_creator = [=, this]() { diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 15d0eece624..b1ae42d03d6 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -1,6 +1,5 @@ #include "ThreadPoolRemoteFSReader.h" -#include #include #include #include @@ -51,25 +50,6 @@ std::future ThreadPoolRemoteFSReader::submit(Reques if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); - if (!query_context) - { - if (!shared_query_context) - { - ContextPtr global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; - if (global_context) - { - shared_query_context = Context::createCopy(global_context); - shared_query_context->makeQueryContext(); - } - } - - if (shared_query_context) - { - shared_query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); - query_context = shared_query_context; - } - } - auto task = std::make_shared>([request, running_group, query_context] { ThreadStatus thread_status; diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h index a2a1e77c834..b2d5f11724a 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.h +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.h @@ -15,7 +15,6 @@ class ThreadPoolRemoteFSReader : public IAsynchronousReader private: ThreadPool pool; - ContextMutablePtr shared_query_context; public: ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 07d27f67d1e..d879953bd9e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -230,7 +230,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co ReadSettings disk_read_settings{read_settings}; if (cache) { - if (IFileCache::shouldBypassCache()) + if (IFileCache::isReadOnly()) disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; disk_read_settings.remote_fs_cache = cache; @@ -272,7 +272,8 @@ std::unique_ptr DiskS3::writeFile(const String & path, LOG_TRACE(log, "{} to file by path: {}. S3 path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + blob_name); - bool cache_on_insert = fs::path(path).extension() != ".tmp" + bool cache_on_write = cache + && fs::path(path).extension() != ".tmp" && write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; @@ -285,7 +286,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, settings->s3_upload_part_size_multiply_parts_count_threshold, settings->s3_max_single_part_upload_size, std::move(object_metadata), - buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), blob_name, cache_on_insert ? cache : nullptr); + buf_size, threadPoolCallbackRunner(getThreadPoolWriter()), blob_name, cache_on_write ? cache : nullptr); auto create_metadata_callback = [this, path, blob_name, mode] (size_t count) { diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 64550e9430b..f036d6a08c8 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -33,7 +33,7 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock & /*buffer auto worker = read_workers.emplace_back(std::make_shared(std::move(reader))); - schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }, nullptr); + schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }); return true; } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 86f4366ec8d..c85f3989531 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -100,28 +100,6 @@ void WriteBufferFromS3::nextImpl() ? CurrentThread::get().getThreadGroup() : MainThreadStatus::getInstance().getThreadGroup(); - if (CurrentThread::isInitialized()) - query_context = CurrentThread::get().getQueryContext(); - - if (!query_context) - { - if (!shared_query_context) - { - ContextPtr global_context = CurrentThread::isInitialized() ? CurrentThread::get().getGlobalContext() : nullptr; - if (global_context) - { - shared_query_context = Context::createCopy(global_context); - shared_query_context->makeQueryContext(); - } - } - - if (shared_query_context) - { - shared_query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); - query_context = shared_query_context; - } - } - if (cacheEnabled()) { if (blob_name.empty()) @@ -132,8 +110,10 @@ void WriteBufferFromS3::nextImpl() current_download_offset += size; size_t remaining_size = size; - for (const auto & file_segment : file_segments_holder->file_segments) + auto & file_segments = file_segments_holder->file_segments; + for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end(); ++file_segment_it) { + auto & file_segment = *file_segment_it; size_t current_size = std::min(file_segment->range().size(), remaining_size); remaining_size -= current_size; @@ -143,6 +123,7 @@ void WriteBufferFromS3::nextImpl() } else { + file_segments.erase(file_segment_it, file_segments.end()); break; } } @@ -190,7 +171,7 @@ WriteBufferFromS3::~WriteBufferFromS3() bool WriteBufferFromS3::cacheEnabled() const { - return cache != nullptr && !IFileCache::shouldBypassCache(); + return cache != nullptr; } void WriteBufferFromS3::preFinalize() @@ -317,7 +298,7 @@ void WriteBufferFromS3::writePart() /// Releasing lock and condvar notification. bg_tasks_condvar.notify_one(); } - }, query_context); + }); } else { @@ -454,7 +435,7 @@ void WriteBufferFromS3::makeSinglepartUpload() /// Releasing lock and condvar notification. bg_tasks_condvar.notify_one(); } - }, query_context); + }); } else { diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index d1e51b0c7f9..8e91bbc04da 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -33,7 +33,7 @@ namespace Aws::S3::Model namespace DB { -using ScheduleFunc = std::function, ContextPtr)>; +using ScheduleFunc = std::function)>; class WriteBufferFromFile; /** @@ -128,8 +128,6 @@ private: size_t current_download_offset = 0; std::optional file_segments_holder; static void finalizeCacheIfNeeded(std::optional &); - ContextMutablePtr shared_query_context; - ContextPtr query_context; }; } diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index af26452e8e6..3464bb31664 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -3,6 +3,7 @@ namespace DB { +/// Settings to be passed to IDisk::writeFile() struct WriteSettings { bool enable_filesystem_cache_on_write_operations = false; diff --git a/src/Interpreters/threadPoolCallbackRunner.cpp b/src/Interpreters/threadPoolCallbackRunner.cpp index 9eeea986d09..288079e49d2 100644 --- a/src/Interpreters/threadPoolCallbackRunner.cpp +++ b/src/Interpreters/threadPoolCallbackRunner.cpp @@ -9,19 +9,14 @@ namespace DB CallbackRunner threadPoolCallbackRunner(ThreadPool & pool) { - return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback, ContextPtr query_context) mutable + return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback) mutable { pool->scheduleOrThrow( - [&, callback = std::move(callback), thread_group, query_context]() + [&, callback = std::move(callback), thread_group]() { if (thread_group) CurrentThread::attachTo(thread_group); - std::optional query_scope; - - if (query_context && !CurrentThread::get().getQueryContext()) - query_scope.emplace(query_context); - SCOPE_EXIT_SAFE({ if (thread_group) CurrentThread::detachQueryIfNotDetached(); diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 8d9d5d4d45b..59d06f2f1bc 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -7,7 +7,7 @@ namespace DB { /// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously -using CallbackRunner = std::function, ContextPtr)>; +using CallbackRunner = std::function)>; /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()' CallbackRunner threadPoolCallbackRunner(ThreadPool & pool); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 005d8093bba..4fb993bfcc7 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -27,7 +27,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeWriterSettings writer_settings( global_settings, - WriteSettings{}, + data_part->storage.getContext()->getWriteSettings(), storage_settings, index_granularity_info ? index_granularity_info->is_adaptive : data_part->storage.canUseAdaptiveGranularity(), /* rewrite_primary_key = */false); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 08a62c47f27..f3ead8a95f0 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -22,6 +22,7 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"state", std::make_shared()}, {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, + {"downloaded_size", std::make_shared()}, }; } @@ -37,9 +38,9 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & [cache_base_path, cache_data] : caches) { const auto & cache = cache_data.cache; - auto holder = cache->getAll(); + auto file_segments = cache->getSnapshot(); - for (const auto & file_segment : holder.file_segments) + for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); res_columns[1]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset())); @@ -49,8 +50,9 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex res_columns[3]->insert(range.right); res_columns[4]->insert(range.size()); res_columns[5]->insert(FileSegment::stateToString(file_segment->state())); - res_columns[6]->insert(file_segment->hits()); - res_columns[7]->insert(file_segment.use_count()); + res_columns[6]->insert(file_segment->getHitsCount()); + res_columns[7]->insert(file_segment->getRefCount()); + res_columns[8]->insert(file_segment->getDownloadedSize()); } } } diff --git a/src/Storages/System/StorageSystemFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h index 0f0bd81e760..1d9d28d7b50 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -7,12 +7,14 @@ namespace DB { /** + * Usgae example. How to get mapping from local paths to remote paths: * SELECT * cache_path, * cache_hits, * remote_path, * local_path, - * file_segment_range, + * file_segment_range_begin, + * file_segment_range_end, * size, * state * FROM From 88134c86a531d97b961312005a6a2c8f26a69cbc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Apr 2022 20:32:47 +0200 Subject: [PATCH 315/372] Fix typo --- src/Core/SettingsEnums.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1168013488e..7fe54c12665 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -165,7 +165,7 @@ DECLARE_SETTING_ENUM(DistributedDDLOutputMode) enum class HandleKafkaErrorMode { - DEFAULT = 0, // Ignore errors whit threshold. + DEFAULT = 0, // Ignore errors with threshold. STREAM, // Put errors to stream in the virtual column named ``_error. /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likey system.kafka_errors. This is not implemented now. */ /*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */ From 5b43df0b14582a82f71dc7a8e700f3ef0131414d Mon Sep 17 00:00:00 2001 From: Andrii R <35810911+Blaumaus@users.noreply.github.com> Date: Thu, 7 Apr 2022 19:12:08 +0000 Subject: [PATCH 316/372] Added Swetrix Analytics --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 20d6b20feb6..98eea85bbfa 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -158,6 +158,7 @@ toc_title: Adopters | Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | | Suning | E-Commerce | User behaviour analytics | — | — | [Blog article](https://www.sohu.com/a/434152235_411876) | | Superwall | Monetization Tooling | Main product | — | — | [Word of mouth, Jan 2022](https://github.com/ClickHouse/ClickHouse/pull/33573) | +| Swetrix | Analytics | Main Product | — | — | [Source code](https://github.com/swetrix/swetrix-api) | | Teralytics | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) | | Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | From 2623b05e59eb143624f7f3e646dcf89f33e85e53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Apr 2022 22:58:29 +0200 Subject: [PATCH 317/372] Fix flaky test --- .../01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index 23df052a8d6..a29d0661621 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash # Tags: long, zookeeper, no-parallel +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 64dcddc6e3c58d6aa2bca27d3ce4d699a15b1414 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 7 Apr 2022 17:41:05 -0400 Subject: [PATCH 318/372] fixed ASTInterpolateElement::clone, fixed QueryNormalizer to exclude ASTInterpolateElement::children --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/QueryNormalizer.cpp | 3 ++- src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 +- src/Parsers/ASTInterpolateElement.h | 3 ++- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2c3c2dcf23d..d4f0f3994a7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -888,7 +888,7 @@ static InterpolateDescriptionPtr getInterpolateDescription( col_set.insert(column.name); } for (const auto & column : result_block) - if( col_set.count(column.name) == 0) + if (col_set.count(column.name) == 0) source_columns.emplace_back(column.name, column.type); } diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 7c820622c37..3c0a965dfa4 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -134,7 +135,7 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & static bool needVisitChild(const ASTPtr & child) { - return !(child->as() || child->as()); + return !(child->as() || child->as() || child->as()); } /// special visitChildren() for ASTSelectQuery diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index c46d0ac7bf0..cc9a2d86d28 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -47,7 +47,7 @@ bool RequiredSourceColumnsMatcher::needChildVisit(const ASTPtr & node, const AST return false; /// Processed. Do not need children. - if (node->as() || node->as() || node->as()) + if (node->as() || node->as() || node->as() || node->as()) return false; if (const auto * f = node->as()) diff --git a/src/Parsers/ASTInterpolateElement.h b/src/Parsers/ASTInterpolateElement.h index 70178de053f..a278755de88 100644 --- a/src/Parsers/ASTInterpolateElement.h +++ b/src/Parsers/ASTInterpolateElement.h @@ -17,8 +17,9 @@ public: ASTPtr clone() const override { auto clone = std::make_shared(*this); - clone->cloneChildren(); clone->expr = clone->expr->clone(); + clone->children.clear(); + clone->children.push_back(clone->expr); return clone; } From 0b7af1f26c85877094478cdbaa52c9fae451159a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Apr 2022 01:58:55 +0200 Subject: [PATCH 319/372] Fix checks --- src/Common/FileCache.cpp | 2 +- src/Common/FileSegment.cpp | 5 +++-- src/Common/FileSegment.h | 2 +- src/Disks/DiskCacheWrapper.cpp | 2 +- .../0_stateless/02240_system_remote_filesystem_cache.sql | 2 +- .../0_stateless/02241_remote_filesystem_cache_on_insert.sql | 2 +- 6 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 5d5851d0b84..05d32f5ffe4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -717,7 +717,7 @@ FileSegments LRUFileCache::getSnapshot() const for (const auto & [key, cells_by_offset] : files) { for (const auto & [offset, cell] : cells_by_offset) - file_segments.push_back(FileSegment::getSnapshot(cell.file_segment)); + file_segments.push_back(FileSegment::getSnapshot(cell.file_segment, cache_lock)); } return file_segments; diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 4def08c6817..ba4129a0ef6 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -608,18 +608,19 @@ void FileSegment::assertCorrectnessImpl(std::lock_guard & /* segment assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(cache->getPathInLocalCache(key(), offset())) > 0); } -FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment) +FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & /* cache_lock */) { auto snapshot = std::make_shared( file_segment->offset(), file_segment->range().size(), file_segment->key(), nullptr, - file_segment->state()); + State::EMPTY); snapshot->hits_count = file_segment->getHitsCount(); snapshot->ref_count = file_segment.use_count(); snapshot->downloaded_size = file_segment->getDownloadedSize(); + snapshot->download_state = file_segment->state(); return snapshot; } diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index a02d8e85a46..615fd9a56de 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -142,7 +142,7 @@ public: void assertCorrectness() const; - static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment); + static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & cache_lock); private: size_t availableSize() const { return reserved_size - downloaded_size; } diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index a86f13f55af..cc2c330975a 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -207,7 +207,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode WriteSettings current_settings = settings; /// There are two different cache implementations. Disable second one if the first is enabled. - /// The firts will soon be removed, this disabling is temporary. + /// The first will soon be removed, this disabling is temporary. current_settings.enable_filesystem_cache_on_write_operations = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index 757f792b931..aa469779130 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-parallel, no-fasttest, no-s3-storage -- { echo } diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index 7b0ff2c70ca..c3ab1de3693 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-parallel, no-fasttest, no-s3-storage -- { echo } From 2e6f0db8255abaaa2da459d7dd2c2b4b7f9dd7a9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 15:12:24 +0800 Subject: [PATCH 320/372] first commit --- src/Storages/Hive/HiveCommon.cpp | 6 ++--- src/Storages/Hive/HiveCommon.h | 18 ++++++--------- src/Storages/Hive/HiveFile.cpp | 22 +++++++++++++++--- src/Storages/Hive/HiveFile.h | 38 ++++++++++++++++++------------- src/Storages/Hive/StorageHive.cpp | 17 ++++++++++---- src/Storages/Hive/StorageHive.h | 4 +++- 6 files changed, 66 insertions(+), 39 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 60f471c1568..510ac2a98d6 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -128,7 +128,7 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( } metadata = std::make_shared( - db_name, table_name, table, std::move(new_partition_infos), getContext()); + db_name, table_name, table, std::move(new_partition_infos)); table_metadata_cache.set(cache_key, metadata); } return metadata; @@ -232,7 +232,7 @@ using namespace apache::thrift::protocol; using namespace apache::thrift::transport; using namespace Apache::Hadoop::Hive; -HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name, ContextPtr context) +HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name) { std::lock_guard lock(mutex); @@ -243,7 +243,7 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na { return createThriftHiveMetastoreClient(name); }; - auto client = std::make_shared(builder, context->getGlobalContext()); + auto client = std::make_shared(builder); clients[name] = client; return client; } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index b8075457a02..bce7d291351 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -37,7 +37,7 @@ protected: private: ThriftHiveMetastoreClientBuilder builder; }; -class HiveMetastoreClient : public WithContext +class HiveMetastoreClient { public: @@ -68,17 +68,15 @@ public: /// Used for speeding up metadata query process. - struct HiveTableMetadata : public WithContext + struct HiveTableMetadata { public: HiveTableMetadata( const String & db_name_, const String & table_name_, std::shared_ptr table_, - const std::map & partition_infos_, - ContextPtr context_) - : WithContext(context_) - , db_name(db_name_) + const std::map & partition_infos_) + : db_name(db_name_) , table_name(table_name_) , table(table_) , partition_infos(partition_infos_) @@ -117,14 +115,12 @@ public: using HiveTableMetadataPtr = std::shared_ptr; - explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_, ContextPtr context_) - : WithContext(context_) - , table_metadata_cache(1000) + explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_) + : table_metadata_cache(1000) , client_pool(builder_) { } - HiveTableMetadataPtr getTableMetadata(const String & db_name, const String & table_name); // Access hive table information by hive client std::shared_ptr getHiveTable(const String & db_name, const String & table_name); @@ -150,7 +146,7 @@ class HiveMetastoreClientFactory final : private boost::noncopyable public: static HiveMetastoreClientFactory & instance(); - HiveMetastoreClientPtr getOrCreate(const String & name, ContextPtr context); + HiveMetastoreClientPtr getOrCreate(const String & name); static std::shared_ptr createThriftHiveMetastoreClient(const String & name); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 7b85a2f4f91..02c92770274 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -84,6 +84,22 @@ std::optional IHiveFile::getRows() return rows; } +void IHiveFile::loadFileMinMaxIndex() +{ + if (file_minmax_idx_loaded) + return; + loadFileMinMaxIndexImpl(); + file_minmax_idx_loaded = true; +} + +void IHiveFile::loadSplitMinMaxIndexes() +{ + if (split_minmax_idxes_loaded) + return; + loadSplitMinMaxIndexesImpl(); + split_minmax_idxes_loaded = true; +} + Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { if (!col_stats || col_stats->hasNull()) @@ -190,7 +206,7 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c return idx; } -void HiveORCFile::loadFileMinMaxIndex() +void HiveORCFile::loadFileMinMaxIndexImpl() { if (!reader) { @@ -208,7 +224,7 @@ bool HiveORCFile::useSplitMinMaxIndex() const } -void HiveORCFile::loadSplitMinMaxIndex() +void HiveORCFile::loadSplitMinMaxIndexesImpl() { if (!reader) { @@ -257,7 +273,7 @@ void HiveParquetFile::prepareReader() THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); } -void HiveParquetFile::loadSplitMinMaxIndex() +void HiveParquetFile::loadSplitMinMaxIndexesImpl() { if (!reader) prepareReader(); diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index f7f50063268..0faf2a190b7 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -123,22 +123,25 @@ public: virtual FileFormat getFormat() const = 0; + /// If hive query could use file level minmax index? virtual bool useFileMinMaxIndex() const { return false; } + void loadFileMinMaxIndex(); - virtual void loadFileMinMaxIndex() - { - throw Exception("Method loadFileMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); - } - - /// If hive query could use contains sub-file level minmax index? + /// If hive query could use sub-file level minmax index? virtual bool useSplitMinMaxIndex() const { return false; } - - virtual void loadSplitMinMaxIndex() - { - throw Exception("Method loadSplitMinMaxIndex is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); - } + void loadSplitMinMaxIndexes(); protected: + virtual void loadFileMinMaxIndexImpl() + { + throw Exception("Method loadFileMinMaxIndexImpl is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } + + virtual void loadSplitMinMaxIndexesImpl() + { + throw Exception("Method loadSplitMinMaxIndexesImpl is not supported by hive file:" + getFormatName(), ErrorCodes::NOT_IMPLEMENTED); + } + virtual std::optional getRowsImpl() = 0; FieldVector partition_values; @@ -149,8 +152,13 @@ protected: std::optional rows; NamesAndTypesList index_names_and_types; + MinMaxIndexPtr file_minmax_idx; + std::atomic file_minmax_idx_loaded{false}; + std::vector split_minmax_idxes; + std::atomic split_minmax_idxes_loaded{false}; + /// Skip splits for this file after applying minmax index (if any) std::unordered_set skip_splits; std::shared_ptr storage_settings; @@ -199,14 +207,13 @@ public: FileFormat getFormat() const override { return FileFormat::ORC; } bool useFileMinMaxIndex() const override; - void loadFileMinMaxIndex() override; - bool useSplitMinMaxIndex() const override; - void loadSplitMinMaxIndex() override; private: static Range buildRange(const orc::ColumnStatistics * col_stats); + void loadFileMinMaxIndexImpl() override; + void loadSplitMinMaxIndexesImpl() override; std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); void prepareReader(); void prepareColumnMapping(); @@ -235,11 +242,10 @@ public: } FileFormat getFormat() const override { return FileFormat::PARQUET; } - bool useSplitMinMaxIndex() const override; - void loadSplitMinMaxIndex() override; private: + void loadSplitMinMaxIndexesImpl() override; std::optional getRowsImpl() override; void prepareReader(); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 90e5dd59722..a91ecf6e280 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -319,7 +319,7 @@ void StorageHive::lazyInitialize() if (has_initialized) return; - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getHiveTable(hive_database, hive_table); hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->sd.location); @@ -415,7 +415,7 @@ ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) } -HiveFilePtr createHiveFile( +static HiveFilePtr createHiveFile( const String & format_name, const FieldVector & fields, const String & namenode_url, @@ -562,6 +562,13 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( hivefile_name_types, storage_settings, context_); + /* + { + std::lock_guard lock{init_mutex}; + hive_files_by_path[file_info.path] = hive_file; + std::cout << "size:" << hive_files_by_path.size() << std::endl; + } + */ if (prune_level >= PruneLevel::File) { @@ -588,7 +595,7 @@ HiveFilePtr StorageHive::createHiveFileIfNeeded( { /// Load sub-file level minmax index and apply std::unordered_set skip_splits; - hive_file->loadSplitMinMaxIndex(); + hive_file->loadSplitMinMaxIndexes(); const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) { @@ -654,7 +661,7 @@ Pipe StorageHive::read( HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); /// Collect Hive files to read @@ -788,7 +795,7 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu if (!isColumnOriented()) return {}; - auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, getContext()->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 0f381803156..b9841d28252 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -131,7 +131,7 @@ private: String hive_database; String hive_table; - std::mutex init_mutex; + mutable std::mutex init_mutex; bool has_initialized = false; /// Hive table meta @@ -155,6 +155,8 @@ private: std::shared_ptr storage_settings; + // mutable std::map hive_files_by_path; + Poco::Logger * log = &Poco::Logger::get("StorageHive"); }; From 213b7bb222322bd1d7c207868ce2a2827c3cf5f6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Apr 2022 08:48:20 +0300 Subject: [PATCH 321/372] clickhouse-client: fix query cancellation if any result was not received yet This should fix issues with queries left after tests like in: - https://s3.amazonaws.com/clickhouse-test-reports/35865/10b9f38d8215cb57783125efe51a8c7aa48590a5/stateless_tests__debug__actions__[2/3].html - https://s3.amazonaws.com/clickhouse-test-reports/35865/10b9f38d8215cb57783125efe51a8c7aa48590a5/stateless_tests__debug__actions__[3/3].html Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5409971c5de..93ed1780e3b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -225,17 +225,16 @@ std::atomic_flag exit_on_signal; class QueryInterruptHandler : private boost::noncopyable { public: - QueryInterruptHandler() { exit_on_signal.clear(); } - - ~QueryInterruptHandler() { exit_on_signal.test_and_set(); } - + static void start() { exit_on_signal.clear(); } + /// Return true if the query was stopped. + static bool stop() { return exit_on_signal.test_and_set(); } static bool cancelled() { return exit_on_signal.test(); } }; /// This signal handler is set only for SIGINT. void interruptSignalHandler(int signum) { - if (exit_on_signal.test_and_set()) + if (QueryInterruptHandler::stop()) safeExit(128 + signum); } @@ -254,7 +253,7 @@ ClientBase::ClientBase() = default; void ClientBase::setupSignalHandler() { - exit_on_signal.test_and_set(); + QueryInterruptHandler::stop(); struct sigaction new_act; memset(&new_act, 0, sizeof(new_act)); @@ -685,6 +684,9 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa { try { + QueryInterruptHandler::start(); + SCOPE_EXIT({ QueryInterruptHandler::stop(); }); + connection->sendQuery( connection_parameters.timeouts, query, @@ -724,8 +726,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa /// Also checks if query execution should be cancelled. void ClientBase::receiveResult(ASTPtr parsed_query) { - QueryInterruptHandler query_interrupt_handler; - // TODO: get the poll_interval from commandline. const auto receive_timeout = connection_parameters.timeouts.receive_timeout; constexpr size_t default_poll_interval = 1000000; /// in microseconds @@ -760,7 +760,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) }; /// handler received sigint - if (query_interrupt_handler.cancelled()) + if (QueryInterruptHandler::cancelled()) { cancel_query(); } From 811178da54721dbfc9b658063c5717667ed0569e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Apr 2022 12:28:25 +0200 Subject: [PATCH 322/372] Add debug and fix cancel_rerun lambda --- .../ci/cancel_and_rerun_workflow_lambda/app.py | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index ab5702569d4..8deed464012 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from collections import namedtuple +from typing import Any, Dict import json import time @@ -21,6 +22,8 @@ API_URL = "https://api.github.com/repos/ClickHouse/ClickHouse" MAX_RETRY = 5 +DEBUG_INFO = {} # type: Dict[str, Any] + def get_installation_id(jwt_token): headers = { @@ -110,6 +113,10 @@ def get_workflows_description_for_pull_request(pull_request_event): workflow_descriptions = [] for workflow in workflows_data: + DEBUG_INFO["workflow"] = workflow + # Some time workflow["head_repository"]["full_name"] is None + if workflow["head_repository"] is None: + continue # unfortunately we cannot filter workflows from forks in request to API # so doing it manually if ( @@ -162,7 +169,8 @@ def exec_workflow_url(urls_to_cancel, token): def main(event): token = get_token_from_aws() - event_data = json.loads(event["body"]) + DEBUG_INFO["event_body"] = event["body"] + event_data = event["body"] print("Got event for PR", event_data["number"]) action = event_data["action"] @@ -210,4 +218,9 @@ def main(event): def handler(event, _): - main(event) + try: + main(event) + except Exception: + for name, value in DEBUG_INFO.items(): + print(f"Value of {name}: ", value) + raise From 1322fab333d64516012ae25f9b3c19ab75d1a573 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Apr 2022 12:51:58 +0200 Subject: [PATCH 323/372] Fix style in run_check.py --- tests/ci/run_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index bda3cf40bba..2f622bd3a42 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -255,7 +255,9 @@ if __name__ == "__main__": elif SUBMODULE_CHANGED_LABEL in pr_info.labels: pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL) - print("change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove)) + print( + "change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove) + ) if pr_labels_to_add: post_labels(gh, pr_info, pr_labels_to_add) From fc3e3251b97f24b451773f74dafb66486ca2285b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Apr 2022 13:50:24 +0200 Subject: [PATCH 324/372] Fix --- .../test_storage_postgresql_replica/test.py | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index d2ac5d1a109..aca33816d75 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -50,20 +50,17 @@ queries = [ def check_tables_are_synchronized( table_name, order_by="key", postgres_database="postgres_database" ): - expected = instance.query( - "select * from {}.{} order by {};".format( - postgres_database, table_name, order_by + while True: + expected = instance.query( + "select * from {}.{} order by {};".format( + postgres_database, table_name, order_by + ) ) - ) - result = instance.query( - "select * from test.{} order by {};".format(table_name, order_by) - ) - - while result != expected: - time.sleep(0.5) result = instance.query( "select * from test.{} order by {};".format(table_name, order_by) ) + if result == expected: + break assert result == expected @@ -103,7 +100,7 @@ def create_clickhouse_postgres_db(ip, port, name="postgres_database"): ) -def create_materialized_table(ip, port, table_name='postgresql_replica'): +def create_materialized_table(ip, port, table_name="postgresql_replica"): instance.query( f""" CREATE TABLE test.{table_name} (key Int64, value Int64) @@ -709,12 +706,12 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): table_name = "postgresql_replica_697" create_postgres_table(cursor, table_name) - instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT -1, 1" - ) + instance.query(f"INSERT INTO postgres_database.{table_name} SELECT -1, 1") instance.query(f"DROP TABLE IF EXISTS test.{table_name} NO DELAY") create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, table_name=table_name + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + table_name=table_name, ) n = 1 @@ -722,7 +719,7 @@ def test_abrupt_server_restart_while_heavy_replication(started_cluster): sleep(1) n += 1 if n > 10: - break; + break for query in queries: cursor.execute(query.format(table_name)) From 46457c4d0c80a451209bed0f531a43330920a077 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 8 Apr 2022 14:25:40 +0200 Subject: [PATCH 325/372] Improve removing of backup files after error. --- src/Backups/ArchiveBackup.cpp | 2 +- src/Backups/ArchiveBackup.h | 2 +- src/Backups/BackupImpl.cpp | 5 ++++- src/Backups/BackupImpl.h | 3 ++- src/Backups/DirectoryBackup.cpp | 7 ++++--- src/Backups/DirectoryBackup.h | 2 +- 6 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp index 5d4837fff2e..f7a13418fa5 100644 --- a/src/Backups/ArchiveBackup.cpp +++ b/src/Backups/ArchiveBackup.cpp @@ -65,7 +65,7 @@ void ArchiveBackup::openImpl(OpenMode open_mode_) } } -void ArchiveBackup::closeImpl(bool writing_finalized_) +void ArchiveBackup::closeImpl(const Strings &, bool writing_finalized_) { /// mutex is already locked if (writer && writer->isWritingFile()) diff --git a/src/Backups/ArchiveBackup.h b/src/Backups/ArchiveBackup.h index 9649c0c1843..d947fa16beb 100644 --- a/src/Backups/ArchiveBackup.h +++ b/src/Backups/ArchiveBackup.h @@ -35,7 +35,7 @@ public: private: bool backupExists() const override; void openImpl(OpenMode open_mode_) override; - void closeImpl(bool writing_finalized_) override; + void closeImpl(const Strings & written_files_, bool writing_finalized_) override; bool supportsWritingInMultipleThreads() const override { return false; } std::unique_ptr readFileImpl(const String & file_name) const override; std::unique_ptr writeFileImpl(const String & file_name) override; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index e4fc894837a..e9dcb2c5b61 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -107,6 +107,7 @@ void BackupImpl::open(OpenMode open_mode_) timestamp = std::time(nullptr); uuid = UUIDHelpers::generateV4(); writing_finalized = false; + written_files.clear(); } if (open_mode_ == OpenMode::READ) @@ -145,7 +146,7 @@ void BackupImpl::close() if (open_mode == OpenMode::NONE) return; - closeImpl(writing_finalized); + closeImpl(written_files, writing_finalized); uuid = UUIDHelpers::Nil; timestamp = 0; @@ -213,6 +214,7 @@ void BackupImpl::writeBackupMetadata() std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM config->save(stream); String str = stream.str(); + written_files.push_back(".backup"); auto out = writeFileImpl(".backup"); out->write(str.data(), str.size()); } @@ -526,6 +528,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) } /// Copy the entry's data after `copy_pos`. + written_files.push_back(file_name); auto out = writeFileImpl(file_name); copyData(*read_buffer, *out); diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index d1fc3c3248c..597b025d0ef 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -47,7 +47,7 @@ protected: virtual void openImpl(OpenMode open_mode_) = 0; OpenMode getOpenModeNoLock() const { return open_mode; } - virtual void closeImpl(bool writing_finalized_) = 0; + virtual void closeImpl(const Strings & written_files_, bool writing_finalized_) = 0; /// Read a file from the backup. /// Low level: the function doesn't check base backup or checksums. @@ -86,6 +86,7 @@ private: std::optional base_backup_uuid; std::map file_infos; /// Should be ordered alphabetically, see listFiles(). std::unordered_map file_checksums; + Strings written_files; bool writing_finalized = false; }; diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp index dc4d098dbe9..b5420c6c894 100644 --- a/src/Backups/DirectoryBackup.cpp +++ b/src/Backups/DirectoryBackup.cpp @@ -56,13 +56,14 @@ void DirectoryBackup::openImpl(OpenMode open_mode_) disk->createDirectories(dir_path); } -void DirectoryBackup::closeImpl(bool writing_finalized_) +void DirectoryBackup::closeImpl(const Strings & written_files_, bool writing_finalized_) { - if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_ && disk->isDirectory(dir_path)) + if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_ && !written_files_.empty()) { /// Creating of the backup wasn't finished correctly, /// so the backup cannot be used and it's better to remove its files. - disk->removeRecursive(dir_path); + for (const String & file_name : written_files_) + disk->removeFileIfExists(file_name); } } diff --git a/src/Backups/DirectoryBackup.h b/src/Backups/DirectoryBackup.h index 7d9b5cc4557..c8cd5dfc009 100644 --- a/src/Backups/DirectoryBackup.h +++ b/src/Backups/DirectoryBackup.h @@ -25,7 +25,7 @@ public: private: bool backupExists() const override; void openImpl(OpenMode open_mode_) override; - void closeImpl(bool writing_finalized_) override; + void closeImpl(const Strings & written_files_, bool writing_finalized_) override; std::unique_ptr readFileImpl(const String & file_name) const override; std::unique_ptr writeFileImpl(const String & file_name) override; From 0db8808f2e3aec6f60a4aee2efd85c62d8421ead Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Apr 2022 15:06:43 +0200 Subject: [PATCH 326/372] Remove unused field from buffers --- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 2 +- src/Disks/DiskWebServer.cpp | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 3 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 17 +++++------------ src/Disks/S3/DiskS3.cpp | 2 +- 6 files changed, 10 insertions(+), 18 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index 97e98fb3a3e..c512ac4326b 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -71,7 +71,7 @@ std::unique_ptr DiskAzureBlobStorage::readFile( LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path)); auto reader_impl = std::make_unique( - path, blob_container_client, metadata, settings->max_single_read_retries, + blob_container_client, metadata, settings->max_single_read_retries, settings->max_single_download_retries, read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index f3039d9af2e..61fd6885411 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -168,7 +168,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p RemoteMetadata meta(path, remote_path); meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); - auto web_impl = std::make_unique(path, url, meta, getContext(), read_settings); + auto web_impl = std::make_unique(url, meta, getContext(), read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index f78ecd2669a..0d7fff29758 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -82,7 +82,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size()); - auto hdfs_impl = std::make_unique(path, config, remote_fs_root_path, metadata, read_settings); + auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings); auto buf = std::make_unique(std::move(hdfs_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 7014b21e8b4..08b39cade79 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -83,11 +83,10 @@ SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const #endif -ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const ReadSettings & settings_, const String & path_) +ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const ReadSettings & settings_) : ReadBuffer(nullptr, 0) , metadata(metadata_) , settings(settings_) - , canonical_path(path_) , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) { } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 25bfe0b7e16..57b409bc740 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -27,8 +27,7 @@ friend class ReadIndirectBufferFromRemoteFS; public: ReadBufferFromRemoteFSGather( const RemoteMetadata & metadata_, - const ReadSettings & settings_, - const String & path_); + const ReadSettings & settings_); String getFileName() const; @@ -89,8 +88,6 @@ private: */ size_t bytes_to_ignore = 0; - String canonical_path; - Poco::Logger * log; }; @@ -101,13 +98,12 @@ class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromS3Gather( - const String & path_, std::shared_ptr client_ptr_, const String & bucket_, IDiskRemote::Metadata metadata_, size_t max_single_read_retries_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_, path_) + : ReadBufferFromRemoteFSGather(metadata_, settings_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) @@ -130,13 +126,12 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS { public: ReadBufferFromAzureBlobStorageGather( - const String & path_, std::shared_ptr blob_container_client_, IDiskRemote::Metadata metadata_, size_t max_single_read_retries_, size_t max_single_download_retries_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_, path_) + : ReadBufferFromRemoteFSGather(metadata_, settings_) , blob_container_client(blob_container_client_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) @@ -157,12 +152,11 @@ class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromWebServerGather( - const String & path_, const String & uri_, RemoteMetadata metadata_, ContextPtr context_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_, path_) + : ReadBufferFromRemoteFSGather(metadata_, settings_) , uri(uri_) , context(context_) { @@ -182,12 +176,11 @@ class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromHDFSGather( - const String & path_, const Poco::Util::AbstractConfiguration & config_, const String & hdfs_uri_, IDiskRemote::Metadata metadata_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_, path_) + : ReadBufferFromRemoteFSGather(metadata_, settings_) , config(config_) { const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index d879953bd9e..4b79c36d210 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -237,7 +237,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co } auto s3_impl = std::make_unique( - path, settings->client, bucket, metadata, + settings->client, bucket, metadata, settings->s3_max_single_read_retries, disk_read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) From ad9fdc1d798a2ceb1dfcadee08c195582c3ad73d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Apr 2022 15:30:06 +0200 Subject: [PATCH 327/372] Don't use metadata path in write buffers --- src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 4 ++-- src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp | 4 ++-- src/Disks/IO/WriteIndirectBufferFromRemoteFS.h | 6 +++--- src/Disks/S3/DiskS3.cpp | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index c512ac4326b..7b027837bb0 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -109,7 +109,7 @@ std::unique_ptr DiskAzureBlobStorage::writeFile( readOrCreateUpdateAndStoreMetadata(path, mode, false, [blob_path, count] (Metadata & metadata) { metadata.addObject(blob_path, count); return true; }); }; - return std::make_unique(std::move(buffer), std::move(create_metadata_callback), path); + return std::make_unique(std::move(buffer), std::move(create_metadata_callback), blob_path); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 0d7fff29758..49854d99460 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -92,7 +92,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path { /// Path to store new HDFS object. auto file_name = getRandomName(); - auto hdfs_path = remote_fs_root_path + file_name; + auto hdfs_path = fs::path(remote_fs_root_path) / file_name; LOG_TRACE(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), hdfs_path); @@ -106,7 +106,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path readOrCreateUpdateAndStoreMetadata(path, mode, false, [file_name, count] (Metadata & metadata) { metadata.addObject(file_name, count); return true; }); }; - return std::make_unique(std::move(hdfs_buffer), std::move(create_metadata_callback), path); + return std::make_unique(std::move(hdfs_buffer), std::move(create_metadata_callback), hdfs_path); } diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp index 9b604341da9..dca2fb17ba7 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp @@ -12,10 +12,10 @@ namespace DB WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( std::unique_ptr impl_, CreateMetadataCallback && create_callback_, - const String & metadata_file_path_) + const String & remote_path_) : WriteBufferFromFileDecorator(std::move(impl_)) , create_metadata_callback(std::move(create_callback_)) - , metadata_file_path(metadata_file_path_) + , remote_path(remote_path_) { } diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h index 25a93e2fe07..84bd2b99c7e 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h @@ -18,17 +18,17 @@ public: WriteIndirectBufferFromRemoteFS( std::unique_ptr impl_, CreateMetadataCallback && create_callback_, - const String & metadata_file_path_); + const String & remote_path_); ~WriteIndirectBufferFromRemoteFS() override; - String getFileName() const override { return metadata_file_path; } + String getFileName() const override { return remote_path; } private: void finalizeImpl() override; CreateMetadataCallback create_metadata_callback; - String metadata_file_path; + String remote_path; }; } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4b79c36d210..90cec1d5dc9 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -280,7 +280,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, auto s3_buffer = std::make_unique( settings->client, bucket, - remote_fs_root_path + blob_name, + fs::path(remote_fs_root_path) / blob_name, settings->s3_min_upload_part_size, settings->s3_upload_part_size_multiply_factor, settings->s3_upload_part_size_multiply_parts_count_threshold, @@ -293,7 +293,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, readOrCreateUpdateAndStoreMetadata(path, mode, false, [blob_name, count] (Metadata & metadata) { metadata.addObject(blob_name, count); return true; }); }; - return std::make_unique(std::move(s3_buffer), std::move(create_metadata_callback), path); + return std::make_unique(std::move(s3_buffer), std::move(create_metadata_callback), fs::path(remote_fs_root_path) / blob_name); } void DiskS3::createHardLink(const String & src_path, const String & dst_path) From 60e36abfe9000fc50bcb60defb2e3d6cfc6270fc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Apr 2022 16:03:00 +0200 Subject: [PATCH 328/372] Get rid of metadata in buffers --- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 4 +-- src/Disks/DiskWebServer.cpp | 4 +-- src/Disks/HDFS/DiskHDFS.cpp | 6 ++-- src/Disks/IDiskRemote.cpp | 5 ++-- src/Disks/IDiskRemote.h | 23 ++++++++++++--- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 26 ++++++++++------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 29 ++++++++++++------- src/Disks/S3/DiskS3.cpp | 2 +- 8 files changed, 64 insertions(+), 35 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index 7b027837bb0..78b9b9e3446 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -71,8 +71,8 @@ std::unique_ptr DiskAzureBlobStorage::readFile( LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path)); auto reader_impl = std::make_unique( - blob_container_client, metadata, settings->max_single_read_retries, - settings->max_single_download_retries, read_settings); + blob_container_client, metadata.remote_fs_root_path, metadata.remote_fs_objects, + settings->max_single_read_retries, settings->max_single_download_retries, read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 61fd6885411..2f8929982e3 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -166,9 +166,9 @@ std::unique_ptr DiskWebServer::readFile(const String & p remote_path = remote_path.string().substr(url.size()); RemoteMetadata meta(path, remote_path); - meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); + meta.remote_fs_objects.emplace_back(remote_path, iter->second.size); - auto web_impl = std::make_unique(url, meta, getContext(), read_settings); + auto web_impl = std::make_unique(url, meta.remote_fs_root_path, meta.remote_fs_objects, getContext(), read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 49854d99460..a3817a85a36 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -82,7 +82,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size()); - auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings); + auto hdfs_impl = std::make_unique(config, remote_fs_root_path, remote_fs_root_path, metadata.remote_fs_objects, read_settings); auto buf = std::make_unique(std::move(hdfs_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } @@ -91,8 +91,8 @@ std::unique_ptr DiskHDFS::readFile(const String & path, std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) { /// Path to store new HDFS object. - auto file_name = getRandomName(); - auto hdfs_path = fs::path(remote_fs_root_path) / file_name; + std::string file_name = getRandomName(); + std::string hdfs_path = fs::path(remote_fs_root_path) / file_name; LOG_TRACE(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), hdfs_path); diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index fb1c0ddc378..b475ae1ee94 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -122,7 +122,8 @@ void IDiskRemote::Metadata::load() remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size()); } assertChar('\n', *buf); - remote_fs_objects[i] = {remote_fs_object_path, remote_fs_object_size}; + remote_fs_objects[i].relative_path = remote_fs_object_path; + remote_fs_objects[i].bytes_size = remote_fs_object_size; } readIntText(ref_count, *buf); @@ -638,7 +639,7 @@ String IDiskRemote::getUniqueId(const String & path) const auto metadata = readMetadata(path); String id; if (!metadata.remote_fs_objects.empty()) - id = metadata.remote_fs_root_path + metadata.remote_fs_objects[0].first; + id = metadata.remote_fs_root_path + metadata.remote_fs_objects[0].relative_path; return id; } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index a8a299391bf..aa78468c7bb 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -13,7 +13,6 @@ #include #include - namespace CurrentMetrics { extern const Metric DiskSpaceReservedForMerge; @@ -22,6 +21,24 @@ namespace CurrentMetrics namespace DB { +/// Path to blob with it's size +struct BlobPathWithSize +{ + std::string relative_path; + uint64_t bytes_size; + + BlobPathWithSize() = default; + BlobPathWithSize(const BlobPathWithSize & other) = default; + + BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_) + : relative_path(relative_path_) + , bytes_size(bytes_size_) + {} +}; + +/// List of blobs with their sizes +using BlobsPathToSize = std::vector; + /// Helper class to collect paths into chunks of maximum size. /// For s3 it is Aws::vector, for hdfs it is std::vector. class RemoteFSPathKeeper @@ -191,10 +208,8 @@ using RemoteDiskPtr = std::shared_ptr; /// Minimum info, required to be passed to ReadIndirectBufferFromRemoteFS struct RemoteMetadata { - using PathAndSize = std::pair; - /// Remote FS objects paths and their sizes. - std::vector remote_fs_objects; + std::vector remote_fs_objects; /// URI const String & remote_fs_root_path; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 08b39cade79..16a57b83771 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -43,7 +43,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S auto remote_file_reader_creator = [=, this]() { return std::make_unique( - client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, + client_ptr, bucket, fs::path(common_path_prefix) / path, max_single_read_retries, settings, /* use_external_buffer */true, /* offset */ 0, read_until_position, /* restricted_seek */true); }; @@ -83,9 +83,13 @@ SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const #endif -ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const ReadSettings & settings_) +ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, + const ReadSettings & settings_) : ReadBuffer(nullptr, 0) - , metadata(metadata_) + , common_path_prefix(common_path_prefix_) + , blobs_to_read(blobs_to_read_) , settings(settings_) , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) { @@ -118,9 +122,9 @@ void ReadBufferFromRemoteFSGather::initialize() { /// One clickhouse file can be split into multiple files in remote fs. auto current_buf_offset = file_offset_of_buffer_end; - for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) + for (size_t i = 0; i < blobs_to_read.size(); ++i) { - const auto & [file_path, size] = metadata.remote_fs_objects[i]; + const auto & [file_path, size] = blobs_to_read[i]; if (size > current_buf_offset) { @@ -137,7 +141,7 @@ void ReadBufferFromRemoteFSGather::initialize() current_buf_offset -= size; } - current_buf_idx = metadata.remote_fs_objects.size(); + current_buf_idx = blobs_to_read.size(); current_buf = nullptr; } @@ -167,12 +171,12 @@ bool ReadBufferFromRemoteFSGather::nextImpl() bool ReadBufferFromRemoteFSGather::moveToNextBuffer() { /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) + if (current_buf_idx + 1 >= blobs_to_read.size()) return false; ++current_buf_idx; - const auto & [path, size] = metadata.remote_fs_objects[current_buf_idx]; + const auto & [path, size] = blobs_to_read[current_buf_idx]; current_buf = createImplementationBuffer(path, size); return true; @@ -201,7 +205,7 @@ bool ReadBufferFromRemoteFSGather::readImpl() if (!result) result = current_buf->next(); - if (metadata.remote_fs_objects.size() == 1) + if (blobs_to_read.size() == 1) { file_offset_of_buffer_end = current_buf->getFileOffsetOfBufferEnd(); } @@ -254,8 +258,8 @@ String ReadBufferFromRemoteFSGather::getFileName() const size_t ReadBufferFromRemoteFSGather::getFileSize() const { size_t size = 0; - for (const auto & object : metadata.remote_fs_objects) - size += object.second; + for (const auto & object : blobs_to_read) + size += object.bytes_size; return size; } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 57b409bc740..f8a6209eb59 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -16,6 +16,8 @@ namespace Poco { class Logger; } namespace DB { + + /** * Remote disk might need to split one clickhouse file into multiple files in remote fs. * This class works like a proxy to allow transition from one file into multiple. @@ -26,7 +28,8 @@ friend class ReadIndirectBufferFromRemoteFS; public: ReadBufferFromRemoteFSGather( - const RemoteMetadata & metadata_, + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, const ReadSettings & settings_); String getFileName() const; @@ -56,7 +59,9 @@ public: protected: virtual SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t file_size) = 0; - RemoteMetadata metadata; + std::string common_path_prefix; + + BlobsPathToSize blobs_to_read; ReadSettings settings; @@ -100,10 +105,11 @@ public: ReadBufferFromS3Gather( std::shared_ptr client_ptr_, const String & bucket_, - IDiskRemote::Metadata metadata_, + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, size_t max_single_read_retries_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_) + : ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) @@ -127,11 +133,12 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS public: ReadBufferFromAzureBlobStorageGather( std::shared_ptr blob_container_client_, - IDiskRemote::Metadata metadata_, + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, size_t max_single_read_retries_, size_t max_single_download_retries_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_) + : ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_) , blob_container_client(blob_container_client_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) @@ -153,10 +160,11 @@ class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather public: ReadBufferFromWebServerGather( const String & uri_, - RemoteMetadata metadata_, + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, ContextPtr context_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_) + : ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_) , uri(uri_) , context(context_) { @@ -178,9 +186,10 @@ public: ReadBufferFromHDFSGather( const Poco::Util::AbstractConfiguration & config_, const String & hdfs_uri_, - IDiskRemote::Metadata metadata_, + const std::string & common_path_prefix_, + const BlobsPathToSize & blobs_to_read_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_, settings_) + : ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_) , config(config_) { const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 90cec1d5dc9..b6171a41dfb 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -237,7 +237,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co } auto s3_impl = std::make_unique( - settings->client, bucket, metadata, + settings->client, bucket, metadata.remote_fs_root_path, metadata.remote_fs_objects, settings->s3_max_single_read_retries, disk_read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) From 4a03402da59463140e0a193f0745ab81646dbc81 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Apr 2022 16:41:37 +0200 Subject: [PATCH 329/372] Update ReadBufferFromRemoteFSGather.h --- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index f8a6209eb59..d12513cba1f 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -16,8 +16,6 @@ namespace Poco { class Logger; } namespace DB { - - /** * Remote disk might need to split one clickhouse file into multiple files in remote fs. * This class works like a proxy to allow transition from one file into multiple. From 87c2b3e2cf5ccff068bef788690e17fc49c93264 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 8 Apr 2022 10:52:10 -0400 Subject: [PATCH 330/372] fixed Nullable, tests added --- src/Interpreters/FillingRow.cpp | 2 +- .../0_stateless/02233_interpolate_1.reference | 48 +++++++++++++++++++ .../0_stateless/02233_interpolate_1.sql | 12 ++++- 3 files changed, 60 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 9679923f10b..bb8661d0ef9 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -122,7 +122,7 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & int { Columns columns = interpolate_block.getColumns(); for (size_t i = 0; i < size; ++i) - interpolate_columns[i]->insertFrom(*columns[i], 0); + interpolate_columns[i]->insertFrom(*columns[i]->convertToFullColumnIfConst(), 0); } else for (const auto & interpolate_column : interpolate_columns) diff --git a/tests/queries/0_stateless/02233_interpolate_1.reference b/tests/queries/0_stateless/02233_interpolate_1.reference index bb1a93131e5..64f5a8308bf 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.reference +++ b/tests/queries/0_stateless/02233_interpolate_1.reference @@ -190,3 +190,51 @@ original 7 10.5 14 11 15 11.5 16 +0 \N +0.5 \N +1 original \N +1.5 \N +2 \N +2.5 \N +3 \N +3.5 \N +4 original \N +4.5 \N +5 \N +5.5 \N +6 \N +6.5 \N +7 original \N +7.5 \N +8 \N +8.5 \N +9 \N +9.5 \N +10 \N +10.5 \N +11 \N +11.5 \N +0 \N +0.5 \N +1 original \N +1.5 \N +2 \N +2.5 \N +3 \N +3.5 \N +4 original \N +4.5 \N +5 \N +5.5 \N +6 \N +6.5 \N +7 original \N +7.5 \N +8 \N +8.5 \N +9 \N +9.5 \N +10 \N +10.5 \N +11 \N +11.5 \N diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index 52b597e06e0..b11385e17b6 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -59,4 +59,14 @@ SELECT n, source, inter FROM ( SELECT n, any(source), sum(inter) AS inter_s FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 ) GROUP BY n -ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); \ No newline at end of file +ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_s AS inter_s + 1 ); + +# Test INTERPOLATE with Nullable in result +SELECT n, source, inter + NULL AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); + +# Test INTERPOLATE with Nullable in source +SELECT n, source, inter AS inter_p FROM ( + SELECT toFloat32(number % 10) AS n, 'original' AS source, number + NULL AS inter FROM numbers(10) WHERE (number % 3) = 1 +) ORDER BY n ASC WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE ( inter_p AS inter_p + 1 ); From 2c99ef0eccbc914e0004498458109feac5bef49c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 23:04:24 +0800 Subject: [PATCH 331/372] refactor HiveTableMetadata --- src/Storages/Hive/HiveCommon.cpp | 131 +++++++++++++++---------------- src/Storages/Hive/HiveCommon.h | 53 +++++++------ src/Storages/Hive/HiveFile.h | 2 + 3 files changed, 94 insertions(+), 92 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 510ac2a98d6..12b5a7ba7ce 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -29,31 +29,6 @@ ThriftHiveMetastoreClientPool::ThriftHiveMetastoreClientPool(ThriftHiveMetastore { } -bool HiveMetastoreClient::shouldUpdateTableMetadata( - const String & db_name, const String & table_name, const std::vector & partitions) -{ - String cache_key = getCacheKey(db_name, table_name); - HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); - if (!metadata) - return true; - - auto old_partiton_infos = metadata->getPartitionInfos(); - if (old_partiton_infos.size() != partitions.size()) - return true; - - for (const auto & partition : partitions) - { - auto it = old_partiton_infos.find(partition.sd.location); - if (it == old_partiton_infos.end()) - return true; - - const auto & old_partition_info = it->second; - if (!old_partition_info.haveSameParameters(partition)) - return true; - } - return false; -} - void HiveMetastoreClient::tryCallHiveClient(std::function func) { int i = 0; @@ -91,44 +66,17 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( }; tryCallHiveClient(client_call); - bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); + // bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); String cache_key = getCacheKey(db_name, table_name); HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); - - if (update_cache) + if (metadata) { - LOG_INFO(log, "Reload hive partition metadata info for {}.{}", db_name, table_name); - - /// Generate partition infos from partitions and old partition infos(if exists). - std::map new_partition_infos; - if (metadata) - { - auto & old_partiton_infos = metadata->getPartitionInfos(); - for (const auto & partition : partitions) - { - auto it = old_partiton_infos.find(partition.sd.location); - if (it == old_partiton_infos.end() || !it->second.haveSameParameters(partition) || !it->second.initialized) - { - new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); - continue; - } - else - { - PartitionInfo new_partition_info(partition); - new_partition_info.files = std::move(it->second.files); - new_partition_info.initialized = true; - } - } - } - else - { - for (const auto & partition : partitions) - new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); - } - - metadata = std::make_shared( - db_name, table_name, table, std::move(new_partition_infos)); + metadata->updateIfNeeded(partitions); + } + else + { + metadata = std::make_shared(db_name, table_name, table, partitions); table_metadata_cache.set(cache_key, metadata); } return metadata; @@ -157,14 +105,14 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin bool HiveMetastoreClient::PartitionInfo::haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const { /// Parameters include keys:numRows,numFiles,rawDataSize,totalSize,transient_lastDdlTime - auto it1 = partition.parameters.cbegin(); - auto it2 = other.parameters.cbegin(); - for (; it1 != partition.parameters.cend() && it2 != other.parameters.cend(); ++it1, ++it2) + auto it = partition.parameters.cbegin(); + auto oit = other.parameters.cbegin(); + for (; it != partition.parameters.cend() && oit != other.parameters.cend(); ++it, ++oit) { - if (it1->first != it2->first || it1->second != it2->second) + if (it->first != oit->first || it->second != oit->second) return false; } - return (it1 == partition.parameters.cend() && it2 == other.parameters.cend()); + return (it == partition.parameters.cend() && oit == other.parameters.cend()); } std::vector HiveMetastoreClient::HiveTableMetadata::getPartitions() const @@ -221,6 +169,57 @@ std::vector HiveMetastoreClient::HiveTableMetadat return result; } +HiveFilesCachePtr HiveMetastoreClient::HiveTableMetadata::getHiveFilesCache() const +{ + return hive_files_cache; +} + +void HiveMetastoreClient::HiveTableMetadata::updateIfNeeded(const std::vector & partitions) +{ + std::lock_guard lock{mutex}; + + if (!shouldUpdate(partitions)) + return; + + std::map new_partition_infos; + auto & old_partiton_infos = partition_infos; + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end() || !it->second.haveSameParameters(partition) || !it->second.initialized) + { + new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); + continue; + } + else + { + new_partition_infos.emplace(partition.sd.location, std::move(it->second)); + } + } + + partition_infos.swap(new_partition_infos); +} + +bool HiveMetastoreClient::HiveTableMetadata::shouldUpdate(const std::vector & partitions) +{ + const auto & old_partiton_infos = partition_infos; + if (old_partiton_infos.size() != partitions.size()) + return false; + + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end()) + return true; + + const auto & old_partition_info = it->second; + if (!old_partition_info.haveSameParameters(partition)) + return true; + } + return false; +} + + HiveMetastoreClientFactory & HiveMetastoreClientFactory::instance() { static HiveMetastoreClientFactory factory; @@ -234,7 +233,6 @@ using namespace Apache::Hadoop::Hive; HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name) { - std::lock_guard lock(mutex); auto it = clients.find(name); if (it == clients.end()) @@ -244,11 +242,12 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na return createThriftHiveMetastoreClient(name); }; auto client = std::make_shared(builder); - clients[name] = client; + clients.emplace(name, client); return client; } return it->second; } + std::shared_ptr HiveMetastoreClientFactory::createThriftHiveMetastoreClient(const String &name) { Poco::URI hive_metastore_url(name); diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index bce7d291351..985d0a0a716 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -40,7 +41,6 @@ private: class HiveMetastoreClient { public: - struct FileInfo { String path; @@ -63,57 +63,61 @@ public: bool initialized = false; /// If true, files are initialized. explicit PartitionInfo(const Apache::Hadoop::Hive::Partition & partition_): partition(partition_) {} + PartitionInfo(PartitionInfo &&) = default; + bool haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const; }; + class HiveTableMetadata; + using HiveTableMetadataPtr = std::shared_ptr; /// Used for speeding up metadata query process. - struct HiveTableMetadata + class HiveTableMetadata : boost::noncopyable { public: HiveTableMetadata( const String & db_name_, const String & table_name_, std::shared_ptr table_, - const std::map & partition_infos_) + const std::vector & partitions_) : db_name(db_name_) , table_name(table_name_) - , table(table_) - , partition_infos(partition_infos_) + , table(std::move(table_)) , empty_partition_keys(table->partitionKeys.empty()) + , hive_files_cache(std::make_shared(10000)) { + std::lock_guard lock(mutex); + for (const auto & partition : partitions_) + partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); } - - std::map & getPartitionInfos() - { - std::lock_guard lock{mutex}; - return partition_infos; - } - - std::shared_ptr getTable() const - { - std::lock_guard lock{mutex}; - return table; - } + std::shared_ptr getTable() const { return table; } std::vector getPartitions() const; std::vector getFilesByLocation(const HDFSFSPtr & fs, const String & location); - private: - String db_name; - String table_name; + HiveFilesCachePtr getHiveFilesCache() const; + void updateIfNeeded(const std::vector & partitions); + + private: + bool shouldUpdate(const std::vector & partitions); + + const String db_name; + const String table_name; + const std::shared_ptr table; + + /// Mutex to protect partition_infos. mutable std::mutex mutex; - std::shared_ptr table; std::map partition_infos; + const bool empty_partition_keys; + const HiveFilesCachePtr hive_files_cache; Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); }; - using HiveTableMetadataPtr = std::shared_ptr; explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_) : table_metadata_cache(1000) @@ -129,9 +133,6 @@ public: private: static String getCacheKey(const String & db_name, const String & table_name) { return db_name + "." + table_name; } - bool shouldUpdateTableMetadata( - const String & db_name, const String & table_name, const std::vector & partitions); - void tryCallHiveClient(std::function func); LRUCache table_metadata_cache; @@ -148,9 +149,9 @@ public: HiveMetastoreClientPtr getOrCreate(const String & name); +private: static std::shared_ptr createThriftHiveMetastoreClient(const String & name); -private: std::mutex mutex; std::map clients; }; diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 0faf2a190b7..6752da15a5d 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -166,6 +166,8 @@ protected: using HiveFilePtr = std::shared_ptr; using HiveFiles = std::vector; +using HiveFilesCache = LRUCache; +using HiveFilesCachePtr = std::shared_ptr; class HiveTextFile : public IHiveFile { From 081de5b0860d3c3041bdc6af01bcd14f6b0d4abe Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 8 Apr 2022 15:21:36 +0000 Subject: [PATCH 332/372] Add tonickkozlov to trusted users for auto ci run Cloudflare employee contributing to ClickHouse --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 6f00232be77..cdefb34d493 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -79,6 +79,7 @@ TRUSTED_CONTRIBUTORS = { "ilejn", # Arenadata, responsible for Kerberized Kafka "thomoco", # ClickHouse "BoloniniD", # Seasoned contributor, HSE + "tonickkozlov", # Cloudflare ] } From e319df17993d75d12992b28888dbafe3b641a7fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 8 Apr 2022 23:58:56 +0800 Subject: [PATCH 333/372] finish dev --- src/Storages/Hive/HiveFile.h | 10 +++--- src/Storages/Hive/StorageHive.cpp | 57 +++++++++++++++++-------------- src/Storages/Hive/StorageHive.h | 15 ++++---- 3 files changed, 44 insertions(+), 38 deletions(-) diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 6752da15a5d..cbdf17bd5b5 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -83,7 +83,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & storage_settings_, - ContextPtr context_) + const ContextPtr & context_) : WithContext(context_) , partition_values(partition_values_) , namenode_url(namenode_url_) @@ -166,7 +166,7 @@ protected: using HiveFilePtr = std::shared_ptr; using HiveFiles = std::vector; -using HiveFilesCache = LRUCache; +using HiveFilesCache = LRUCache; using HiveFilesCachePtr = std::shared_ptr; class HiveTextFile : public IHiveFile @@ -180,7 +180,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -202,7 +202,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -238,7 +238,7 @@ public: size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, - ContextPtr context_) + const ContextPtr & context_) : IHiveFile(partition_values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a91ecf6e280..9bc0a3bd2ec 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -424,7 +424,7 @@ static HiveFilePtr createHiveFile( size_t size, const NamesAndTypesList & index_names_and_types, const std::shared_ptr & hive_settings, - ContextPtr context) + const ContextPtr & context) { HiveFilePtr hive_file; if (format_name == "HiveText") @@ -449,9 +449,9 @@ static HiveFilePtr createHiveFile( HiveFiles StorageHive::collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level) const { LOG_DEBUG( @@ -517,7 +517,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, fields, query_info, hive_table_metadata, context_, prune_level); if (hive_file) { LOG_TRACE( @@ -533,42 +533,47 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( } std::vector -StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs) +StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs) { return hive_table_metadata->getFilesByLocation(fs, path); } -HiveFilePtr StorageHive::createHiveFileIfNeeded( +HiveFilePtr StorageHive::getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, const SelectQueryInfo & query_info, - ContextPtr context_, + const HiveTableMetadataPtr & hive_table_metadata, + const ContextPtr & context_, PruneLevel prune_level) const { - LOG_TRACE(log, "Create hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); + LOG_TRACE(log, "Get hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' if (startsWith(filename, ".")) return {}; - auto hive_file = createHiveFile( - format_name, - fields, - hdfs_namenode_url, - file_info.path, - file_info.last_modify_time, - file_info.size, - hivefile_name_types, - storage_settings, - context_); - /* + auto cache = hive_table_metadata->getHiveFilesCache(); + auto hive_file = cache->get(file_info.path); + if (!hive_file || hive_file->getLastModTs() < file_info.last_modify_time) { - std::lock_guard lock{init_mutex}; - hive_files_by_path[file_info.path] = hive_file; - std::cout << "size:" << hive_files_by_path.size() << std::endl; + LOG_TRACE(log, "Create hive file {}", file_info.path); + hive_file = createHiveFile( + format_name, + fields, + hdfs_namenode_url, + file_info.path, + file_info.last_modify_time, + file_info.size, + hivefile_name_types, + storage_settings, + context_); + cache->set(file_info.path, hive_file); + } + else + { + LOG_TRACE(log, "Get hive file {} from cache", file_info.path); } - */ if (prune_level >= PruneLevel::File) { @@ -711,9 +716,9 @@ Pipe StorageHive::read( HiveFiles StorageHive::collectHiveFiles( unsigned max_threads, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level) const { std::vector partitions = hive_table_metadata->getPartitions(); @@ -751,7 +756,7 @@ HiveFiles StorageHive::collectHiveFiles( pool.scheduleOrThrowOnError( [&]() { - auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_, prune_level); + auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level); if (hive_file) { std::lock_guard lock(hive_files_mutex); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b9841d28252..f7cd32c2162 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -91,31 +91,32 @@ private: static ASTPtr extractKeyExpressionList(const ASTPtr & node); - static std::vector listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs); + static std::vector listDirectory(const String & path, const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs); void initMinMaxIndexExpression(); HiveFiles collectHiveFiles( unsigned max_threads, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; HiveFiles collectHiveFilesFromPartition( const Apache::Hadoop::Hive::Partition & partition, const SelectQueryInfo & query_info, - HiveTableMetadataPtr hive_table_metadata, + const HiveTableMetadataPtr & hive_table_metadata, const HDFSFSPtr & fs, - ContextPtr context_, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; - HiveFilePtr createHiveFileIfNeeded( + HiveFilePtr getHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, const SelectQueryInfo & query_info, - ContextPtr context_, + const HiveTableMetadataPtr & hive_table_metadata, + const ContextPtr & context_, PruneLevel prune_level = PruneLevel::Max) const; void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const; From cd807da83890608b5d30b5cec1ee0b8f4f359e30 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 00:15:33 +0800 Subject: [PATCH 334/372] finish test --- src/Storages/Hive/StorageHive.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 9bc0a3bd2ec..3302df7da3d 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -546,8 +546,6 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( const ContextPtr & context_, PruneLevel prune_level) const { - LOG_TRACE(log, "Get hive file {} if needed, prune_level:{}", file_info.path, pruneLevelToString(prune_level)); - String filename = getBaseName(file_info.path); /// Skip temporary files starts with '.' if (startsWith(filename, ".")) @@ -557,7 +555,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( auto hive_file = cache->get(file_info.path); if (!hive_file || hive_file->getLastModTs() < file_info.last_modify_time) { - LOG_TRACE(log, "Create hive file {}", file_info.path); + LOG_TRACE(log, "Create hive file {}, prune_level {}", file_info.path, pruneLevelToString(prune_level)); hive_file = createHiveFile( format_name, fields, @@ -572,7 +570,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( } else { - LOG_TRACE(log, "Get hive file {} from cache", file_info.path); + LOG_TRACE(log, "Get hive file {} from cache, prune_level {}", file_info.path, pruneLevelToString(prune_level)); } if (prune_level >= PruneLevel::File) From 70f4503ba5d2e5bb73348773bc451bb026838437 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 00:28:07 +0800 Subject: [PATCH 335/372] use global context for cache --- src/Storages/Hive/StorageHive.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3302df7da3d..633816bb195 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -565,7 +565,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( file_info.size, hivefile_name_types, storage_settings, - context_); + context_->getGlobalContext()); cache->set(file_info.path, hive_file); } else From f506b6c1a369f9cf031336e40746b5f55cb97350 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Apr 2022 19:43:19 +0200 Subject: [PATCH 336/372] Fix linter issues and missed formatting --- tests/ci/build_check.py | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index e9d660156e1..2e2f09950fb 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -64,22 +64,22 @@ def get_packager_cmd( ) if build_config["build_type"]: - cmd += " --build-type={}".format(build_config["build_type"]) + cmd += f" --build-type={build_config['build_type']}" if build_config["sanitizer"]: - cmd += " --sanitizer={}".format(build_config["sanitizer"]) + cmd += f" --sanitizer={build_config['sanitizer']}" if build_config["splitted"] == "splitted": cmd += " --split-binary" if build_config["tidy"] == "enable": cmd += " --clang-tidy" cmd += " --cache=ccache" - cmd += " --ccache_dir={}".format(ccache_path) + cmd += f" --ccache_dir={ccache_path}" if "additional_pkgs" in build_config and build_config["additional_pkgs"]: cmd += " --additional-pkgs" - cmd += " --docker-image-version={}".format(image_version) - cmd += " --version={}".format(build_version) + cmd += f" --docker-image-version={image_version}" + cmd += f" --version={build_version}" if _can_export_binaries(build_config): cmd += " --with-binaries=tests" @@ -149,16 +149,9 @@ def create_json_artifact( json_name = "build_urls_" + build_name + ".json" - print( - "Dump json report", - result, - "to", - json_name, - "with env", - "build_urls_{build_name}", - ) + print(f"Dump json report {result} to {json_name} with env build_urls_{build_name}") - with open(os.path.join(temp_path, json_name), "w") as build_links: + with open(os.path.join(temp_path, json_name), "w", encoding="utf-8") as build_links: json.dump(result, build_links) @@ -337,7 +330,7 @@ def main(): print("::notice ::Build URLs: {}".format("\n".join(build_urls))) - print("::notice ::Log URL: {}".format(log_url)) + print(f"::notice ::Log URL: {log_url}") create_json_artifact( TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success From 71d54db2159a957335d4a3acdf8f7b2ffb4db1a4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 8 Apr 2022 14:21:18 +0200 Subject: [PATCH 337/372] Better handling arguments of backup engines. --- src/Backups/BackupInfo.cpp | 24 ++- src/Backups/BackupInfo.h | 1 + src/Backups/DirectoryBackup.cpp | 39 ++--- src/Backups/DirectoryBackup.h | 4 +- .../registerBackupEnginesFileAndDisk.cpp | 138 +++++++++--------- 5 files changed, 106 insertions(+), 100 deletions(-) diff --git a/src/Backups/BackupInfo.cpp b/src/Backups/BackupInfo.cpp index ba953b818c2..cab08e306d6 100644 --- a/src/Backups/BackupInfo.cpp +++ b/src/Backups/BackupInfo.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -23,7 +24,11 @@ String BackupInfo::toString() const auto list = std::make_shared(); func->arguments = list; func->children.push_back(list); - list->children.reserve(args.size()); + list->children.reserve(args.size() + !id_arg.empty()); + + if (!id_arg.empty()) + list->children.push_back(std::make_shared(id_arg)); + for (const auto & arg : args) list->children.push_back(std::make_shared(arg)); @@ -53,9 +58,22 @@ BackupInfo BackupInfo::fromAST(const IAST & ast) const auto * list = func->arguments->as(); if (!list) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected list, got {}", serializeAST(*func->arguments)); - res.args.reserve(list->children.size()); - for (const auto & elem : list->children) + + size_t index = 0; + if (!list->children.empty()) { + const auto * id = list->children[0]->as(); + if (id) + { + res.id_arg = id->name(); + ++index; + } + } + + res.args.reserve(list->children.size() - index); + for (; index < list->children.size(); ++index) + { + const auto & elem = list->children[index]; const auto * lit = elem->as(); if (!lit) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected literal, got {}", serializeAST(*elem)); diff --git a/src/Backups/BackupInfo.h b/src/Backups/BackupInfo.h index 9b7d03c6d6a..5b5c676ecf1 100644 --- a/src/Backups/BackupInfo.h +++ b/src/Backups/BackupInfo.h @@ -11,6 +11,7 @@ class IAST; struct BackupInfo { String backup_engine_name; + String id_arg; std::vector args; String toString() const; diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp index b5420c6c894..0deb41c200d 100644 --- a/src/Backups/DirectoryBackup.cpp +++ b/src/Backups/DirectoryBackup.cpp @@ -1,16 +1,9 @@ #include -#include -#include #include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - DirectoryBackup::DirectoryBackup( const String & backup_name_, @@ -19,23 +12,16 @@ DirectoryBackup::DirectoryBackup( const ContextPtr & context_, const std::optional & base_backup_info_) : BackupImpl(backup_name_, context_, base_backup_info_) - , disk(disk_), path(path_) + , disk(disk_) { - /// Path to backup must end with '/' - if (!path.ends_with("/")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to backup must end with '/', but {} doesn't.", getName(), quoteString(path)); - dir_path = fs::path(path).parent_path(); /// get path without terminating slash + /// Remove terminating slash. + path = (std::filesystem::path(path_) / "").parent_path(); /// If `disk` is not specified, we create an internal instance of `DiskLocal` here. if (!disk) { - auto fspath = fs::path{dir_path}; - if (!fspath.has_filename()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to a backup must be a directory path.", getName(), quoteString(path)); - path = fspath.filename() / ""; - dir_path = fs::path(path).parent_path(); /// get path without terminating slash - String disk_path = fspath.remove_filename(); - disk = std::make_shared(disk_path, disk_path, 0); + disk = std::make_shared(path, path, 0); + path = "."; } } @@ -47,13 +33,13 @@ DirectoryBackup::~DirectoryBackup() bool DirectoryBackup::backupExists() const { - return disk->isDirectory(dir_path); + return disk->isDirectory(path); } void DirectoryBackup::openImpl(OpenMode open_mode_) { if (open_mode_ == OpenMode::WRITE) - disk->createDirectories(dir_path); + disk->createDirectories(path); } void DirectoryBackup::closeImpl(const Strings & written_files_, bool writing_finalized_) @@ -62,20 +48,23 @@ void DirectoryBackup::closeImpl(const Strings & written_files_, bool writing_fin { /// Creating of the backup wasn't finished correctly, /// so the backup cannot be used and it's better to remove its files. - for (const String & file_name : written_files_) - disk->removeFileIfExists(file_name); + const auto & files_to_delete = written_files_; + for (const String & file_name : files_to_delete) + disk->removeFileIfExists(path / file_name); + if (disk->isDirectory(path) && disk->isDirectoryEmpty(path)) + disk->removeDirectory(path); } } std::unique_ptr DirectoryBackup::readFileImpl(const String & file_name) const { - String file_path = path + file_name; + auto file_path = path / file_name; return disk->readFile(file_path); } std::unique_ptr DirectoryBackup::writeFileImpl(const String & file_name) { - String file_path = path + file_name; + auto file_path = path / file_name; disk->createDirectories(fs::path(file_path).parent_path()); return disk->writeFile(file_path); } diff --git a/src/Backups/DirectoryBackup.h b/src/Backups/DirectoryBackup.h index c8cd5dfc009..499a1893dca 100644 --- a/src/Backups/DirectoryBackup.h +++ b/src/Backups/DirectoryBackup.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -30,8 +31,7 @@ private: std::unique_ptr writeFileImpl(const String & file_name) override; DiskPtr disk; - String path; - String dir_path; /// `path` without terminating slash + std::filesystem::path path; }; } diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 6a34d67115d..e3b06a21d96 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -13,8 +14,9 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INVALID_CONFIG_PARAMETER; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -22,83 +24,70 @@ namespace { namespace fs = std::filesystem; - [[noreturn]] void throwDiskIsAllowed(const String & disk_name) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name); - } - - [[noreturn]] void throwPathNotAllowed(const fs::path & path) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} is not allowed for backups", quoteString(String{path})); - } - - void checkAllowedPathInConfigIsValid(const String & key, const fs::path & value) - { - if (value.empty() || value.is_relative()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Configuration parameter {} has a wrong value {}", key, String{value}); - } - - /// Checks that a disk name and a path specified as parameters of Disk() are valid. - void checkDiskNameAndPath(const String & disk_name, fs::path & path, const Poco::Util::AbstractConfiguration & config) + /// Checks that a disk name specified as parameters of Disk() is valid. + void checkDiskName(const String & disk_name, const Poco::Util::AbstractConfiguration & config) { String key = "backups.allowed_disk"; - bool disk_name_found = false; + if (!config.has(key)) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "The \"backups.allowed_disk\" configuration parameter is not set, cannot use Disk() backup engine"); + size_t counter = 0; - while (config.has(key)) + while (config.getString(key) != disk_name) { - if (config.getString(key) == disk_name) - { - disk_name_found = true; - break; - } key = "backups.allowed_disk[" + std::to_string(++counter) + "]"; + if (!config.has(key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups, see the \"backups.allowed_disk\" configuration parameter", disk_name); } - - if (!disk_name_found) - throwDiskIsAllowed(disk_name); - - path = path.lexically_normal(); - if (!path.is_relative() || path.empty() || (*path.begin() == "..")) - throwPathNotAllowed(path); } - /// Checks that a path specified as a parameter of File() is valid. - void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config) + /// Checks that a path specified as parameters of Disk() is valid. + void checkPath(const String & disk_name, const DiskPtr & disk, fs::path & path) { - String key = "backups.allowed_path"; + path = path.lexically_normal(); + if (!path.is_relative() && (disk->getType() == DiskType::Local)) + path = path.lexically_proximate(disk->getPath()); + bool path_ok = path.empty() || (path.is_relative() && (*path.begin() != "..")); + if (!path_ok) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} to backup must be inside the specified disk {}", quoteString(path.c_str()), disk_name); + } + + /// Checks that a path specified as parameters of File() is valid. + void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config, const fs::path & data_dir) { path = path.lexically_normal(); if (path.empty()) - throwPathNotAllowed(path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to backup must not be empty"); + + String key = "backups.allowed_path"; + if (!config.has(key)) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, + "The \"backups.allowed_path\" configuration parameter is not set, cannot use File() backup engine"); if (path.is_relative()) { - if (*path.begin() == "..") - throwPathNotAllowed(path); + auto first_allowed_path = fs::path(config.getString(key)); + if (first_allowed_path.is_relative()) + first_allowed_path = data_dir / first_allowed_path; - auto base = fs::path(config.getString(key, "")); - checkAllowedPathInConfigIsValid(key, base); - path = base / path; - return; + path = first_allowed_path / path; } - bool path_found_in_config = false; size_t counter = 0; - while (config.has(key)) + while (true) { - auto base = fs::path(config.getString(key)); - checkAllowedPathInConfigIsValid(key, base); - auto rel = path.lexically_relative(base); - if (!rel.empty() && (*rel.begin() != "..")) - { - path_found_in_config = true; + auto allowed_path = fs::path(config.getString(key)); + if (allowed_path.is_relative()) + allowed_path = data_dir / allowed_path; + auto rel = path.lexically_proximate(allowed_path); + bool path_ok = rel.empty() || (rel.is_relative() && (*rel.begin() != "..")); + if (path_ok) break; - } key = "backups.allowed_path[" + std::to_string(++counter) + "]"; + if (!config.has(key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Path {} is not allowed for backups, see the \"backups.allowed_path\" configuration parameter", + quoteString(path.c_str())); } - - if (!path_found_in_config) - throwPathNotAllowed(path); } } @@ -109,6 +98,15 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) { String backup_name = params.backup_info.toString(); const String & engine_name = params.backup_info.backup_engine_name; + + if (!params.backup_info.id_arg.empty()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Backup engine '{}' requires first argument to be a string", + engine_name); + } + const auto & args = params.backup_info.args; DiskPtr disk; @@ -123,7 +121,9 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) } path = args[0].safeGet(); - checkPath(path, params.context->getConfigRef()); + const auto & config = params.context->getConfigRef(); + const auto & data_dir = params.context->getPath(); + checkPath(path, config, data_dir); } else if (engine_name == "Disk") { @@ -135,30 +135,28 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) } String disk_name = args[0].safeGet(); + const auto & config = params.context->getConfigRef(); + checkDiskName(disk_name, config); path = args[1].safeGet(); - checkDiskNameAndPath(disk_name, path, params.context->getConfigRef()); disk = params.context->getDisk(disk_name); + checkPath(disk_name, disk, path); } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected backup engine '{}'", engine_name); - std::unique_ptr backup; - - if (!path.has_filename() && !path.empty()) - { - if (!params.password.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); - backup = std::make_unique(backup_name, disk, path, params.context, params.base_backup_info); - } - else if (hasRegisteredArchiveFileExtension(path)) + if (hasRegisteredArchiveFileExtension(path)) { auto archive_backup = std::make_unique(backup_name, disk, path, params.context, params.base_backup_info); archive_backup->setCompression(params.compression_method, params.compression_level); archive_backup->setPassword(params.password); - backup = std::move(archive_backup); + return archive_backup; } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to backup must be either a directory or a path to an archive"); - - return backup; + { + if (!params.password.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); + return std::make_unique(backup_name, disk, path, params.context, params.base_backup_info); + } }; factory.registerBackupEngine("File", creator_fn); From 2894672ecc099c3bcde43f9d211d0f0765a6b651 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 8 Apr 2022 22:12:57 +0300 Subject: [PATCH 338/372] Update DatabaseReplicatedWorker.cpp --- src/Databases/DatabaseReplicatedWorker.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 84c3f857a81..5765f1c6598 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -319,7 +319,6 @@ bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name, void DatabaseReplicatedDDLWorker::initializeLogPointer(const String & processed_entry_name) { updateMaxDDLEntryID(processed_entry_name); - assert(max_id.load() == parse(getAndSetZooKeeper()->get(fs::path(database->replica_path) / "log_ptr"))); } UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const From cb962b8ad5e14145bb78e238e82941609089df4b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Apr 2022 22:34:58 +0200 Subject: [PATCH 339/372] Stop rewriting BUILD_NAME env in build_check.py --- .github/workflows/backport_branches.yml | 20 ++++---- .github/workflows/master.yml | 68 ++++++++++++------------- .github/workflows/pull_request.yml | 68 ++++++++++++------------- .github/workflows/release_branches.yml | 28 +++++----- tests/ci/build_check.py | 2 +- 5 files changed, 93 insertions(+), 93 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c5ee2acfea0..44fe082b04d 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -155,8 +155,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -196,8 +196,8 @@ jobs: - name: Upload build URLs to artifacts uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -241,8 +241,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -286,8 +286,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -331,8 +331,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index eab7ce36eb7..efaf1c64c05 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -219,8 +219,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -260,8 +260,8 @@ jobs: - name: Upload build URLs to artifacts uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -305,8 +305,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -350,8 +350,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -395,8 +395,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -440,8 +440,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -485,8 +485,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -530,8 +530,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -575,8 +575,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -620,8 +620,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -668,8 +668,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -713,8 +713,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -758,8 +758,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -803,8 +803,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -848,8 +848,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -893,8 +893,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -938,8 +938,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8942cca391e..8072f816cb8 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -272,8 +272,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -317,8 +317,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -362,8 +362,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -404,8 +404,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -446,8 +446,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -491,8 +491,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -536,8 +536,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -581,8 +581,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -626,8 +626,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -671,8 +671,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -719,8 +719,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -764,8 +764,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -809,8 +809,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -854,8 +854,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -899,8 +899,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -944,8 +944,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -989,8 +989,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index b2af465142b..91e1a224204 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -146,8 +146,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -187,8 +187,8 @@ jobs: - name: Upload build URLs to artifacts uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -232,8 +232,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -277,8 +277,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -322,8 +322,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -367,8 +367,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | @@ -412,8 +412,8 @@ jobs: if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: - name: ${{ env.BUILD_NAME }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - name: Cleanup if: always() run: | diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 2e2f09950fb..840f4ff0d76 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -136,7 +136,7 @@ def create_json_artifact( success: bool, ): subprocess.check_call( - f"echo 'BUILD_NAME=build_urls_{build_name}' >> $GITHUB_ENV", shell=True + f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True ) result = { From a09fcdac1fdf65aea094486a93a6d4bf0ad0f349 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sat, 9 Apr 2022 00:34:23 +0200 Subject: [PATCH 340/372] Do not fail on a poisoned ccache --- tests/ci/build_check.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 840f4ff0d76..74b2d3f89c3 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -6,6 +6,7 @@ import json import os import sys import time +from shutil import rmtree from typing import List, Optional, Tuple from env_helper import REPO_COPY, TEMP_PATH, CACHES_PATH, IMAGES_PATH @@ -270,7 +271,12 @@ def main(): ccache_path = os.path.join(CACHES_PATH, build_name + "_ccache") logging.info("Will try to fetch cache for our build") - get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, TEMP_PATH) + try: + get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, TEMP_PATH) + except Exception as e: + # In case there are issues with ccache, remove the path and do not fail a build + logging.info("Failed to get ccache, building without it. Error: %s", e) + rmtree(ccache_path, ignore_errors=True) if not os.path.exists(ccache_path): logging.info("cache was not fetched, will create empty dir") From fcd28fc3f43500acf48d900af0fc21d10562a506 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Apr 2022 01:07:14 +0200 Subject: [PATCH 341/372] Make backup metadata clearer with new setting 'use_base'. --- src/Backups/BackupImpl.cpp | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index e9dcb2c5b61..21300f2dbae 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -203,9 +203,12 @@ void BackupImpl::writeBackupMetadata() config->setString(prefix + "checksum", getHexUIntLowercase(info.checksum)); if (info.base_size) { - config->setUInt(prefix + "base_size", info.base_size); - if (info.base_checksum != info.checksum) + config->setBool(prefix + "use_base", true); + if (info.base_size != info.size) + { + config->setUInt(prefix + "base_size", info.base_size); config->setString(prefix + "base_checksum", getHexUIntLowercase(info.base_checksum)); + } } } ++index; @@ -255,13 +258,14 @@ void BackupImpl::readBackupMetadata() if (info.size) { info.checksum = unhexChecksum(config->getString(prefix + "checksum")); - info.base_size = config->getUInt(prefix + "base_size", 0); + bool use_base = config->getBool(prefix + "use_base", false); + info.base_size = config->getUInt(prefix + "base_size", use_base ? info.size : 0); if (info.base_size) { - if (config->has(prefix + "base_checksum")) - info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum")); - else + if (info.base_size == info.size) info.base_checksum = info.checksum; + else + info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum")); } } file_infos.emplace(name, info); @@ -347,11 +351,6 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const return std::make_unique(nullptr, 0, UInt128{0, 0}); } - auto read_callback = [backup = std::static_pointer_cast(shared_from_this()), file_name]() - { - return backup->readFileImpl(file_name); - }; - if (!info.base_size) { /// Data goes completely from this backup, the base backup isn't used. From bcd727d5355ee7deeea1a737a57d9c0efad9131c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Apr 2022 01:32:17 +0200 Subject: [PATCH 342/372] Allow creating directories containing a new archive. --- src/Backups/ArchiveBackup.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp index f7a13418fa5..0c4b0c3cd40 100644 --- a/src/Backups/ArchiveBackup.cpp +++ b/src/Backups/ArchiveBackup.cpp @@ -42,6 +42,14 @@ void ArchiveBackup::openImpl(OpenMode open_mode_) /// mutex is already locked if (open_mode_ == OpenMode::WRITE) { + /// Create a directory to contain the archive. + auto dir_path = fs::path(path).parent_path(); + if (disk) + disk->createDirectories(dir_path); + else + std::filesystem::create_directories(dir_path); + + /// Start writing the archive. if (disk) writer = createArchiveWriter(path, disk->writeFile(path)); else From 758f8235c2f886276e16e55ba16597db5da077d9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Apr 2022 01:44:46 +0200 Subject: [PATCH 343/372] Collect backup entries before opening an archive for writing. --- src/Interpreters/InterpreterBackupQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 01970bc5cc2..b630bac9515 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -41,8 +41,8 @@ namespace void executeBackup(const ContextPtr & context, const ASTBackupQuery & query) { auto backup_settings = BackupSettings::fromBackupQuery(query); - BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), backup_settings, context); auto backup_entries = makeBackupEntries(context, query.elements, backup_settings); + BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), backup_settings, context); writeBackupEntries(backup, std::move(backup_entries), context->getSettingsRef().max_backup_threads); } From 7e89f760f36d70756b2076cbf14ffd304fd5f703 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Apr 2022 10:43:58 +0800 Subject: [PATCH 344/372] remove useless code --- src/Storages/Hive/StorageHive.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index f7cd32c2162..63b42d68a82 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -156,8 +156,6 @@ private: std::shared_ptr storage_settings; - // mutable std::map hive_files_by_path; - Poco::Logger * log = &Poco::Logger::get("StorageHive"); }; From 5efd8a4ec020d4006c9f80ceb395a8e3c18f6e0d Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sat, 9 Apr 2022 15:00:41 +0800 Subject: [PATCH 345/372] If data type has DEFAULT NULL in table definition, make data type Nullable --- src/Interpreters/InterpreterCreateQuery.cpp | 16 ++++++++++++++++ .../02266_auto_add_nullable.reference | 6 ++++++ .../0_stateless/02266_auto_add_nullable.sql | 17 +++++++++++++++++ 3 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/02266_auto_add_nullable.reference create mode 100644 tests/queries/0_stateless/02266_auto_add_nullable.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 14bf9df2347..f3953eb67fe 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -54,6 +54,7 @@ #include #include #include +#include #include #include @@ -480,6 +481,21 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { column_type = makeNullable(column_type); } + else if (!hasNullable(column_type) && + col_decl.default_specifier == "DEFAULT" && + col_decl.default_expression && + col_decl.default_expression->as() && + col_decl.default_expression->as()->value.isNull()) + { + if (column_type->lowCardinality()) + { + const auto * low_cardinality_type = typeid_cast(column_type.get()); + assert(low_cardinality_type); + column_type = std::make_shared(makeNullable(low_cardinality_type->getDictionaryType())); + } + else + column_type = makeNullable(column_type); + } column_names_and_types.emplace_back(col_decl.name, column_type); } diff --git a/tests/queries/0_stateless/02266_auto_add_nullable.reference b/tests/queries/0_stateless/02266_auto_add_nullable.reference new file mode 100644 index 00000000000..582bd1a1d6d --- /dev/null +++ b/tests/queries/0_stateless/02266_auto_add_nullable.reference @@ -0,0 +1,6 @@ +val0 Nullable(Int8) DEFAULT NULL +val1 Nullable(Int8) DEFAULT NULL +val2 Nullable(UInt8) DEFAULT NULL +val3 Nullable(String) DEFAULT NULL +val4 LowCardinality(Nullable(Int8)) DEFAULT NULL +val5 LowCardinality(Nullable(Int8)) DEFAULT NULL diff --git a/tests/queries/0_stateless/02266_auto_add_nullable.sql b/tests/queries/0_stateless/02266_auto_add_nullable.sql new file mode 100644 index 00000000000..7a9c6fbb19f --- /dev/null +++ b/tests/queries/0_stateless/02266_auto_add_nullable.sql @@ -0,0 +1,17 @@ +SET allow_suspicious_low_cardinality_types = 1; +DROP TABLE IF EXISTS 02266_auto_add_nullable; + +CREATE TABLE 02266_auto_add_nullable +( + val0 Int8 DEFAULT NULL, + val1 Nullable(Int8) DEFAULT NULL, + val2 UInt8 DEFAULT NUll, + val3 String DEFAULT null, + val4 LowCardinality(Int8) DEFAULT NULL, + val5 LowCardinality(Nullable(Int8)) DEFAULT NULL +) +ENGINE = MergeTree order by tuple(); + +DESCRIBE TABLE 02266_auto_add_nullable; + +DROP TABLE IF EXISTS 02266_auto_add_nullable; \ No newline at end of file From f61cad144cf82048fb4a71ff183942ecff77f6ad Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 10 Apr 2022 15:52:12 +0200 Subject: [PATCH 346/372] Get rid of strange abstraction --- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 49 +++------- .../AzureBlobStorage/DiskAzureBlobStorage.h | 4 +- src/Disks/HDFS/DiskHDFS.cpp | 60 ++---------- src/Disks/HDFS/DiskHDFS.h | 4 +- src/Disks/IDiskRemote.cpp | 39 ++++---- src/Disks/IDiskRemote.h | 43 +-------- src/Disks/S3/DiskS3.cpp | 93 ++++++------------- src/Disks/S3/DiskS3.h | 4 +- 8 files changed, 70 insertions(+), 226 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index 78b9b9e3446..556c28bd3f4 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -32,21 +32,6 @@ DiskAzureBlobStorageSettings::DiskAzureBlobStorageSettings( thread_pool_size(thread_pool_size_) {} -class AzureBlobStoragePathKeeper : public RemoteFSPathKeeper -{ -public: - /// RemoteFSPathKeeper constructed with a placeholder argument for chunk_limit, it is unused in this class - AzureBlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {} - - void addPath(const String & path) override - { - paths.push_back(path); - } - - std::vector paths; -}; - - DiskAzureBlobStorage::DiskAzureBlobStorage( const String & name_, DiskPtr metadata_disk_, @@ -150,36 +135,24 @@ bool DiskAzureBlobStorage::checkUniqueId(const String & id) const } -void DiskAzureBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) +void DiskAzureBlobStorage::removeFromRemoteFS(const std::vector & paths) { - auto * paths_keeper = dynamic_cast(fs_paths_keeper.get()); - - if (paths_keeper) + for (const auto & path : paths) { - for (const auto & path : paths_keeper->paths) + try { - try - { - auto delete_info = blob_container_client->DeleteBlob(path); - if (!delete_info.Value.Deleted) - throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path); - } - catch (const Azure::Storage::StorageException & e) - { - LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message); - throw; - } + auto delete_info = blob_container_client->DeleteBlob(path); + if (!delete_info.Value.Deleted) + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path); + } + catch (const Azure::Storage::StorageException & e) + { + LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message); + throw; } } } - -RemoteFSPathKeeperPtr DiskAzureBlobStorage::createFSPathKeeper() const -{ - return std::make_shared(); -} - - void DiskAzureBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) { auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context); diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h index efc245e7eb3..ff99e246d31 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h @@ -67,9 +67,7 @@ public: bool checkUniqueId(const String & id) const override; - void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override; - - RemoteFSPathKeeperPtr createFSPathKeeper() const override; + void removeFromRemoteFS(const std::vector & paths) override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override; diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index a3817a85a36..a0b96aba728 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -30,35 +30,6 @@ namespace ErrorCodes } -class HDFSPathKeeper : public RemoteFSPathKeeper -{ -public: - using Chunk = std::vector; - using Chunks = std::list; - - explicit HDFSPathKeeper(size_t chunk_limit_) : RemoteFSPathKeeper(chunk_limit_) {} - - void addPath(const String & path) override - { - if (chunks.empty() || chunks.back().size() >= chunk_limit) - { - chunks.push_back(Chunks::value_type()); - chunks.back().reserve(chunk_limit); - } - chunks.back().push_back(path.data()); - } - - void removePaths(Fn auto && remove_chunk_func) - { - for (auto & chunk : chunks) - remove_chunk_func(std::move(chunk)); - } - -private: - Chunks chunks; -}; - - DiskHDFS::DiskHDFS( const String & disk_name_, const String & hdfs_root_path_, @@ -109,30 +80,17 @@ std::unique_ptr DiskHDFS::writeFile(const String & path return std::make_unique(std::move(hdfs_buffer), std::move(create_metadata_callback), hdfs_path); } - -RemoteFSPathKeeperPtr DiskHDFS::createFSPathKeeper() const +void DiskHDFS::removeFromRemoteFS(const std::vector & paths) { - return std::make_shared(settings->objects_chunk_size_to_delete); -} + for (const auto & hdfs_path : paths) + { + const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2); - -void DiskHDFS::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) -{ - auto * hdfs_paths_keeper = dynamic_cast(fs_paths_keeper.get()); - if (hdfs_paths_keeper) - hdfs_paths_keeper->removePaths([&](std::vector && chunk) - { - for (const auto & hdfs_object_path : chunk) - { - const String & hdfs_path = hdfs_object_path; - const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2); - - /// Add path from root to file name - int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0); - if (res == -1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path); - } - }); + /// Add path from root to file name + int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0); + if (res == -1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path); + } } bool DiskHDFS::checkUniqueId(const String & hdfs_uri) const diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index eba58101bc4..5c6e011dc96 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -62,9 +62,7 @@ public: std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) override; - void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override; - - RemoteFSPathKeeperPtr createFSPathKeeper() const override; + void removeFromRemoteFS(const std::vector & paths) override; /// Check file exists and ClickHouse has an access to it /// Overrode in remote disk diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index b475ae1ee94..ead951084ad 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -270,7 +270,7 @@ std::unordered_map IDiskRemote::getSerializedMetadata(const std: return metadatas; } -void IDiskRemote::removeMetadata(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper) +void IDiskRemote::removeMetadata(const String & path, std::vector & paths_to_remove) { LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path)); @@ -282,13 +282,13 @@ void IDiskRemote::removeMetadata(const String & path, RemoteFSPathKeeperPtr fs_p try { - auto metadata_updater = [fs_paths_keeper, this] (Metadata & metadata) + auto metadata_updater = [&paths_to_remove, this] (Metadata & metadata) { if (metadata.ref_count == 0) { for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects) { - fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path); + paths_to_remove.push_back(remote_fs_root_path + remote_fs_object_path); if (cache) { @@ -327,18 +327,18 @@ void IDiskRemote::removeMetadata(const String & path, RemoteFSPathKeeperPtr fs_p } -void IDiskRemote::removeMetadataRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper) +void IDiskRemote::removeMetadataRecursive(const String & path, std::vector & paths_to_remove) { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. if (metadata_disk->isFile(path)) { - removeMetadata(path, fs_paths_keeper); + removeMetadata(path, paths_to_remove); } else { for (auto it = iterateDirectory(path); it->isValid(); it->next()) - removeMetadataRecursive(it->path(), fs_paths_keeper); + removeMetadataRecursive(it->path(), paths_to_remove); metadata_disk->removeDirectory(path); } @@ -480,50 +480,47 @@ void IDiskRemote::replaceFile(const String & from_path, const String & to_path) moveFile(from_path, to_path); } - void IDiskRemote::removeSharedFile(const String & path, bool delete_metadata_only) { - RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); - removeMetadata(path, fs_paths_keeper); + std::vector paths_to_remove; + removeMetadata(path, paths_to_remove); if (!delete_metadata_only) - removeFromRemoteFS(fs_paths_keeper); + removeFromRemoteFS(paths_to_remove); } - void IDiskRemote::removeSharedFileIfExists(const String & path, bool delete_metadata_only) { - RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); - + std::vector paths_to_remove; if (metadata_disk->exists(path)) { - removeMetadata(path, fs_paths_keeper); + removeMetadata(path, paths_to_remove); if (!delete_metadata_only) - removeFromRemoteFS(fs_paths_keeper); + removeFromRemoteFS(paths_to_remove); } } void IDiskRemote::removeSharedFiles(const RemoveBatchRequest & files, bool delete_metadata_only) { - RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); + std::vector paths_to_remove; for (const auto & file : files) { bool skip = file.if_exists && !metadata_disk->exists(file.path); if (!skip) - removeMetadata(file.path, fs_paths_keeper); + removeMetadata(file.path, paths_to_remove); } if (!delete_metadata_only) - removeFromRemoteFS(fs_paths_keeper); + removeFromRemoteFS(paths_to_remove); } void IDiskRemote::removeSharedRecursive(const String & path, bool delete_metadata_only) { - RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); - removeMetadataRecursive(path, fs_paths_keeper); + std::vector paths_to_remove; + removeMetadataRecursive(path, paths_to_remove); if (!delete_metadata_only) - removeFromRemoteFS(fs_paths_keeper); + removeFromRemoteFS(paths_to_remove); } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index aa78468c7bb..990855d4bc3 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -21,41 +21,6 @@ namespace CurrentMetrics namespace DB { -/// Path to blob with it's size -struct BlobPathWithSize -{ - std::string relative_path; - uint64_t bytes_size; - - BlobPathWithSize() = default; - BlobPathWithSize(const BlobPathWithSize & other) = default; - - BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_) - : relative_path(relative_path_) - , bytes_size(bytes_size_) - {} -}; - -/// List of blobs with their sizes -using BlobsPathToSize = std::vector; - -/// Helper class to collect paths into chunks of maximum size. -/// For s3 it is Aws::vector, for hdfs it is std::vector. -class RemoteFSPathKeeper -{ -public: - explicit RemoteFSPathKeeper(size_t chunk_limit_) : chunk_limit(chunk_limit_) {} - - virtual ~RemoteFSPathKeeper() = default; - - virtual void addPath(const String & path) = 0; - -protected: - size_t chunk_limit; -}; - -using RemoteFSPathKeeperPtr = std::shared_ptr; - class IAsynchronousReader; using AsynchronousReaderPtr = std::shared_ptr; @@ -165,9 +130,7 @@ public: bool checkUniqueId(const String & id) const override = 0; - virtual void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) = 0; - - virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0; + virtual void removeFromRemoteFS(const std::vector & paths) = 0; static AsynchronousReaderPtr getThreadPoolReader(); static ThreadPool & getThreadPoolWriter(); @@ -190,9 +153,9 @@ protected: FileCachePtr cache; private: - void removeMetadata(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper); + void removeMetadata(const String & path, std::vector & paths_to_remove); - void removeMetadataRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper); + void removeMetadataRecursive(const String & path, std::vector & paths_to_remove); bool tryReserve(UInt64 bytes); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index b6171a41dfb..6ed53bcb9c2 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -60,52 +60,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -/// Helper class to collect keys into chunks of maximum size (to prepare batch requests to AWS API) -/// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html -class S3PathKeeper : public RemoteFSPathKeeper -{ -public: - using Chunk = Aws::Vector; - using Chunks = std::list; - - explicit S3PathKeeper(size_t chunk_limit_) : RemoteFSPathKeeper(chunk_limit_) {} - - void addPath(const String & path) override - { - if (chunks.empty() || chunks.back().size() >= chunk_limit) - { - /// add one more chunk - chunks.push_back(Chunks::value_type()); - chunks.back().reserve(chunk_limit); - } - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(path); - chunks.back().push_back(obj); - } - - void removePaths(Fn auto && remove_chunk_func) - { - for (auto & chunk : chunks) - remove_chunk_func(std::move(chunk)); - } - - static String getChunkKeys(const Chunk & chunk) - { - String res; - for (const auto & obj : chunk) - { - const auto & key = obj.GetKey(); - if (!res.empty()) - res.append(", "); - res.append(key.c_str(), key.size()); - } - return res; - } - -private: - Chunks chunks; -}; - template void throwIfError(Aws::Utils::Outcome & response) { @@ -168,31 +122,36 @@ DiskS3::DiskS3( { } -RemoteFSPathKeeperPtr DiskS3::createFSPathKeeper() const +void DiskS3::removeFromRemoteFS(const std::vector & paths) { auto settings = current_settings.get(); - return std::make_shared(settings->objects_chunk_size_to_delete); -} -void DiskS3::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) -{ - auto settings = current_settings.get(); - auto * s3_paths_keeper = dynamic_cast(fs_paths_keeper.get()); - - if (s3_paths_keeper) - s3_paths_keeper->removePaths([&](S3PathKeeper::Chunk && chunk) + size_t chunk_size_limit = settings->objects_chunk_size_to_delete; + size_t current_position = 0; + while (current_position < paths.size()) + { + std::vector current_chunk; + String keys; + for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position) { - String keys = S3PathKeeper::getChunkKeys(chunk); - LOG_TRACE(log, "Remove AWS keys {}", keys); - Aws::S3::Model::Delete delkeys; - delkeys.SetObjects(chunk); - Aws::S3::Model::DeleteObjectsRequest request; - request.SetBucket(bucket); - request.SetDelete(delkeys); - auto outcome = settings->client->DeleteObjects(request); - // Do not throw here, continue deleting other chunks - logIfError(outcome, [&](){return "Can't remove AWS keys: " + keys;}); - }); + Aws::S3::Model::ObjectIdentifier obj; + obj.SetKey(paths[current_position]); + current_chunk.push_back(obj); + + if (!keys.empty()) + keys += ", "; + keys += paths[current_position]; + } + + LOG_TRACE(log, "Remove AWS keys {}", keys); + Aws::S3::Model::Delete delkeys; + delkeys.SetObjects(current_chunk); + Aws::S3::Model::DeleteObjectsRequest request; + request.SetBucket(bucket); + request.SetDelete(delkeys); + auto outcome = settings->client->DeleteObjects(request); + logIfError(outcome, [&](){return "Can't remove AWS keys: " + keys;}); + } } void DiskS3::moveFile(const String & from_path, const String & to_path) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 32eb9ee7aef..80784362769 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -91,9 +91,7 @@ public: WriteMode mode, const WriteSettings & settings) override; - void removeFromRemoteFS(RemoteFSPathKeeperPtr keeper) override; - - RemoteFSPathKeeperPtr createFSPathKeeper() const override; + void removeFromRemoteFS(const std::vector & paths) override; void moveFile(const String & from_path, const String & to_path, bool send_metadata); void moveFile(const String & from_path, const String & to_path) override; From 324020a46f9bf481d60b86820110fbc46cfc3690 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 10 Apr 2022 15:59:43 +0200 Subject: [PATCH 347/372] Restore class --- src/Disks/IDiskRemote.h | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 990855d4bc3..ac2b1634d05 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -21,6 +21,23 @@ namespace CurrentMetrics namespace DB { +/// Path to blob with it's size +struct BlobPathWithSize +{ + std::string relative_path; + uint64_t bytes_size; + + BlobPathWithSize() = default; + BlobPathWithSize(const BlobPathWithSize & other) = default; + + BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_) + : relative_path(relative_path_) + , bytes_size(bytes_size_) + {} +}; + +/// List of blobs with their sizes +using BlobsPathToSize = std::vector; class IAsynchronousReader; using AsynchronousReaderPtr = std::shared_ptr; From 24e86f81cba62a44b51eb3c95b17826b81346376 Mon Sep 17 00:00:00 2001 From: fenglv Date: Sun, 10 Apr 2022 17:20:51 +0000 Subject: [PATCH 348/372] Fix window view when is proc time and window kind larger than day fix --- src/Storages/WindowView/StorageWindowView.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3fca34ed511..8b14976b040 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -915,21 +915,31 @@ void StorageWindowView::threadFuncCleanup() void StorageWindowView::threadFuncFireProc() { + static bool window_kind_larger_than_day = window_kind == IntervalKind::Week || window_kind == IntervalKind::Month + || window_kind == IntervalKind::Quarter || window_kind == IntervalKind::Year; + std::unique_lock lock(fire_signal_mutex); UInt32 timestamp_now = std::time(nullptr); - while (next_fire_signal <= timestamp_now) + /// When window kind is larger than day, getWindowUpperBound() will get a day num instead of timestamp, + /// and addTime will also add day num to the next_fire_signal, so we need to convert it into timestamp. + /// Otherwise, we will get wrong result and after create window view with window kind larger than day, + /// since day num is too smaller than current timestamp, it will fire a lot. + auto exact_fire_signal = window_kind_larger_than_day ? next_fire_signal * 86400 : next_fire_signal; + + while (exact_fire_signal <= timestamp_now) { try { - fire(next_fire_signal); + fire(exact_fire_signal); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - max_fired_watermark = next_fire_signal; + max_fired_watermark = exact_fire_signal; next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); + exact_fire_signal = window_kind_larger_than_day ? next_fire_signal * 86400 : next_fire_signal; } UInt64 timestamp_ms = static_cast(Poco::Timestamp().epochMicroseconds()) / 1000; From cac2df914b6e4d2dae28c6de9ebd62974abd8b6c Mon Sep 17 00:00:00 2001 From: fenglv Date: Sun, 10 Apr 2022 17:28:17 +0000 Subject: [PATCH 349/372] fix --- src/Storages/WindowView/StorageWindowView.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8b14976b040..d70b5c6e989 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -944,7 +944,9 @@ void StorageWindowView::threadFuncFireProc() UInt64 timestamp_ms = static_cast(Poco::Timestamp().epochMicroseconds()) / 1000; if (!shutdown_called) - fire_task->scheduleAfter(std::max(UInt64(0), static_cast(next_fire_signal) * 1000 - timestamp_ms)); + fire_task->scheduleAfter(std::max( + UInt64(0), + static_cast(window_kind_larger_than_day ? next_fire_signal * 86400 : next_fire_signal) * 1000 - timestamp_ms)); } void StorageWindowView::threadFuncFireEvent() From 383076ca0e30b13c3c8abc9d87d75946d49d81a0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 10 Apr 2022 16:42:43 -0300 Subject: [PATCH 350/372] fix typo --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index bf8accc0f47..0d984dc4dee 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -79,7 +79,7 @@ struct Settings; M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ M(Seconds, execute_merges_on_single_replica_time_threshold, 0, "When greater than zero only a single replica starts the merge immediately, others wait up to that amount of time to download the result instead of doing merges locally. If the chosen replica doesn't finish the merge during that amount of time, fallback to standard behavior happens.", 0) \ - M(Seconds, remote_fs_execute_merges_on_single_replica_time_threshold, 3 * 60 * 60, "When greater than zero only a single replica starts the merge immediatelys when merged part on shared storage and 'allow_remote_fs_zero_copy_replication' is enabled.", 0) \ + M(Seconds, remote_fs_execute_merges_on_single_replica_time_threshold, 3 * 60 * 60, "When greater than zero only a single replica starts the merge immediately if merged part on shared storage and 'allow_remote_fs_zero_copy_replication' is enabled.", 0) \ M(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \ M(Bool, always_fetch_merged_part, false, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ From 5c8569fc660cd1042435cd3839b2a92bd57345fc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sun, 10 Apr 2022 22:01:37 +0200 Subject: [PATCH 351/372] Fix scan report filename --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 836421f34dd..bd54fd975c0 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -112,7 +112,7 @@ jobs: run: | curl --form token="${COVERITY_TOKEN}" \ --form email='security+coverity@clickhouse.com' \ - --form file="@$TEMP_PATH/$BUILD_NAME/clickhouse-scan.tgz" \ + --form file="@$TEMP_PATH/$BUILD_NAME/coverity-scan.tgz" \ --form version="${GITHUB_REF#refs/heads/}-${GITHUB_SHA::6}" \ --form description="Nighly Scan: $(date +'%Y-%m-%dT%H:%M:%S')" \ https://scan.coverity.com/builds?project=ClickHouse%2FClickHouse From 090fd728842b9cc20800df0ace80d9f59ba673cc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 11 Apr 2022 11:19:31 +0800 Subject: [PATCH 352/372] fix bug --- src/Storages/Hive/HiveCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 12b5a7ba7ce..7b2f04f7073 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -204,7 +204,7 @@ bool HiveMetastoreClient::HiveTableMetadata::shouldUpdate(const std::vector Date: Mon, 11 Apr 2022 15:50:10 +0800 Subject: [PATCH 353/372] fix typo --- docs/zh/operations/configuration-files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/configuration-files.md b/docs/zh/operations/configuration-files.md index 7998baafb6c..c99b8fcfca3 100644 --- a/docs/zh/operations/configuration-files.md +++ b/docs/zh/operations/configuration-files.md @@ -3,7 +3,7 @@ ClickHouse支持多配置文件管理。主配置文件是`/etc/clickhouse-server/config.xml`。其余文件须在目录`/etc/clickhouse-server/config.d`。 !!! 注意: - 所有配置文件必须是XML格式。此外,配置文件须有相同的跟元素,通常是``。 + 所有配置文件必须是XML格式。此外,配置文件须有相同的根元素,通常是``。 主配置文件中的一些配置可以通过`replace`或`remove`属性被配置文件覆盖。 From 0ae63dfdbd724856dbd95d60f367ad51181229b1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Apr 2022 12:45:16 +0200 Subject: [PATCH 354/372] Fix code style in registerBackupEnginesFileAndDisk.cpp --- src/Backups/registerBackupEnginesFileAndDisk.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index e3b06a21d96..fa1786c6350 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -53,7 +53,8 @@ namespace } /// Checks that a path specified as parameters of File() is valid. - void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config, const fs::path & data_dir) { + void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config, const fs::path & data_dir) + { path = path.lexically_normal(); if (path.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to backup must not be empty"); From 5c0d15e0170cf15abc53174220d320ab2453742c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Apr 2022 12:52:26 +0200 Subject: [PATCH 355/372] fix flaky tests --- tests/queries/0_stateless/00971_query_id_in_logs.sh | 3 ++- .../queries/0_stateless/01003_kill_query_race_condition.sh | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00971_query_id_in_logs.sh b/tests/queries/0_stateless/00971_query_id_in_logs.sh index 9e927f36a9c..4de6e02d10d 100755 --- a/tests/queries/0_stateless/00971_query_id_in_logs.sh +++ b/tests/queries/0_stateless/00971_query_id_in_logs.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace @@ -9,4 +10,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e # No log lines without query id -$CLICKHOUSE_CLIENT --query_id=hello --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello} ' | grep -P '<\w+>' ||: +$CLICKHOUSE_CLIENT --query_id=hello_00971 --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello_00971} ' | grep -P '<\w+>' ||: diff --git a/tests/queries/0_stateless/01003_kill_query_race_condition.sh b/tests/queries/0_stateless/01003_kill_query_race_condition.sh index 64caf0f88d1..f98897b1544 100755 --- a/tests/queries/0_stateless/01003_kill_query_race_condition.sh +++ b/tests/queries/0_stateless/01003_kill_query_race_condition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race +# Tags: race, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,12 +9,12 @@ set -e function thread1() { - $CLICKHOUSE_CLIENT --query_id=hello --query "SELECT count() FROM numbers(1000000000)" --format Null; + $CLICKHOUSE_CLIENT --query_id=hello_01003 --query "SELECT count() FROM numbers(1000000000)" --format Null; } function thread2() { - $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello'" --format Null + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello_01003'" --format Null sleep 0.$RANDOM } From 6e346a7fc38d683a198a506b205f562b4085c7d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Apr 2022 14:32:55 +0200 Subject: [PATCH 356/372] Support atomic replace under OSX --- .../{renameat2.cpp => atomicRename.cpp} | 65 +++++++++++++++++-- src/Common/{renameat2.h => atomicRename.h} | 2 +- src/Databases/DatabaseAtomic.cpp | 4 +- src/Disks/DiskLocal.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 6 files changed, 65 insertions(+), 12 deletions(-) rename src/Common/{renameat2.cpp => atomicRename.cpp} (69%) rename src/Common/{renameat2.h => atomicRename.h} (95%) diff --git a/src/Common/renameat2.cpp b/src/Common/atomicRename.cpp similarity index 69% rename from src/Common/renameat2.cpp rename to src/Common/atomicRename.cpp index 8ee9081af56..fb6b0ada40b 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/atomicRename.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -55,7 +55,7 @@ namespace ErrorCodes namespace DB { -static bool supportsRenameat2Impl() +static bool supportsAtomicRenameImpl() { VersionNumber renameat2_minimal_version(3, 15, 0); VersionNumber linux_version(Poco::Environment::osVersion()); @@ -64,7 +64,7 @@ static bool supportsRenameat2Impl() static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { - if (!supportsRenameat2()) + if (!supportsAtomicRename()) return false; if (old_path.empty() || new_path.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); @@ -93,14 +93,67 @@ static bool renameat2(const std::string & old_path, const std::string & new_path throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}", old_path, new_path), new_path, ErrorCodes::SYSTEM_ERROR); } -bool supportsRenameat2() +bool supportsAtomicRename() { - static bool supports = supportsRenameat2Impl(); + static bool supports = supportsAtomicRenameImpl(); return supports; } } +#elif defined(__APPLE__) + +// Includes +#include // For dlsym +#include // For renamex_np +#include // For stderror + +#ifndef RENAME_SWAP + #define RENAME_SWAP 0x00000002 +#endif +#ifndef RENAME_EXCL + #define RENAME_EXCL 0x00000004 +#endif + + +#define RENAME_NOREPLACE RENAME_EXCL +#define RENAME_EXCHANGE RENAME_SWAP + + +static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) +{ + auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); + if (fun == NULL) + return false; + + if (old_path.empty() || new_path.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); + + /// int olddirfd (ignored for absolute oldpath), const char *oldpath, + /// int newdirfd (ignored for absolute newpath), const char *newpath, + /// unsigned int flags + if (0 == (*fun)(old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) + return true; + int errnum = errno; + + if ((errnum == ENOTSUP) || (errnum == EINVAL) + return false; + if (errnum == EEXIST) + throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); + if (errnum == ENOENT) + throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); + throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}: {}", old_path, new_path, stderror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); +} + + + +static bool supportsAtomicRenameImpl() +{ + auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); + return fun != NULL; +} + + #else #define RENAME_NOREPLACE -1 @@ -114,7 +167,7 @@ static bool renameat2(const std::string &, const std::string &, int) return false; } -bool supportsRenameat2() +bool supportsAtomicRename() { return false; } diff --git a/src/Common/renameat2.h b/src/Common/atomicRename.h similarity index 95% rename from src/Common/renameat2.h rename to src/Common/atomicRename.h index 141c5d385c5..6da8a8f623b 100644 --- a/src/Common/renameat2.h +++ b/src/Common/atomicRename.h @@ -6,7 +6,7 @@ namespace DB { /// Returns true, if the following functions supported by the system -bool supportsRenameat2(); +bool supportsAtomicRename(); /// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception void renameNoReplace(const std::string & old_path, const std::string & new_path); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index adfcd83f5a7..622d38e01bd 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -158,7 +158,7 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ return; } - if (exchange && !supportsRenameat2()) + if (exchange && !supportsAtomicRename()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported"); auto & other_db = dynamic_cast(to_database); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 8aad42ab475..d81782a8af1 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2f51d942403..7513c3bf849 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f3953eb67fe..24c58c819a4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include From 991d6d18dfcb65c905957d890c7b2b4ce36c83bb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Apr 2022 15:08:51 +0200 Subject: [PATCH 357/372] Update 01171_mv_select_insert_isolation_long.sh --- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 3de63615bc4..5086515e9eb 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -11,6 +11,7 @@ set -e $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tmp"; $CLICKHOUSE_CLIENT --query "CREATE TABLE src (n Int8, m Int8, CONSTRAINT c CHECK xxHash32(n+m) % 8 != 0) ENGINE=MergeTree ORDER BY n PARTITION BY 0 < n SETTINGS old_parts_lifetime=0"; $CLICKHOUSE_CLIENT --query "CREATE TABLE dst (nm Int16, CONSTRAINT c CHECK xxHash32(nm) % 8 != 0) ENGINE=MergeTree ORDER BY nm SETTINGS old_parts_lifetime=0"; $CLICKHOUSE_CLIENT --query "CREATE MATERIALIZED VIEW mv TO dst (nm Int16) AS SELECT n*m AS nm FROM src"; @@ -154,3 +155,4 @@ $CLICKHOUSE_CLIENT --query "SELECT count(), sum(nm) FROM mv" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; $CLICKHOUSE_CLIENT --query "DROP TABLE dst"; $CLICKHOUSE_CLIENT --query "DROP TABLE mv"; +$CLICKHOUSE_CLIENT --query "DROP TABLE tmp"; From 83c814f65839c046bd16fad7bac472c6b493605d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Apr 2022 15:25:20 +0200 Subject: [PATCH 358/372] OSX fixes --- src/Common/atomicRename.cpp | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/src/Common/atomicRename.cpp b/src/Common/atomicRename.cpp index fb6b0ada40b..c63c0e05899 100644 --- a/src/Common/atomicRename.cpp +++ b/src/Common/atomicRename.cpp @@ -119,40 +119,47 @@ bool supportsAtomicRename() #define RENAME_NOREPLACE RENAME_EXCL #define RENAME_EXCHANGE RENAME_SWAP +namespace DB +{ static bool renameat2(const std::string & old_path, const std::string & new_path, int flags) { - auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); - if (fun == NULL) + using function_type = int (*)(const char * from, const char * to, unsigned int flags); + static function_type fun = reinterpret_cast(dlsym(RTLD_DEFAULT, "renamex_np")); + if (fun == nullptr) return false; if (old_path.empty() || new_path.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path); - /// int olddirfd (ignored for absolute oldpath), const char *oldpath, - /// int newdirfd (ignored for absolute newpath), const char *newpath, - /// unsigned int flags - if (0 == (*fun)(old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) + if (0 == (*fun)(old_path.c_str(), new_path.c_str(), flags)) return true; int errnum = errno; - if ((errnum == ENOTSUP) || (errnum == EINVAL) + if (errnum == ENOTSUP || errnum == EINVAL) return false; if (errnum == EEXIST) throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); if (errnum == ENOENT) throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL); - throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}: {}", old_path, new_path, stderror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); + throwFromErrnoWithPath( + fmt::format("Cannot rename {} to {}: {}", old_path, new_path, strerror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR); } - static bool supportsAtomicRenameImpl() { auto fun = dlsym(RTLD_DEFAULT, "renamex_np"); - return fun != NULL; + return fun != nullptr; } +bool supportsAtomicRename() +{ + static bool supports = supportsAtomicRenameImpl(); + return supports; +} + +} #else From df81d719068e5a094ef22d8a5fe8c0114ee211dc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:28:46 +0200 Subject: [PATCH 359/372] Remove duplicate --- .github/workflows/pull_request.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 8072f816cb8..4881bed3f55 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -1052,7 +1052,6 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' CHECK_NAME=ClickHouse build check (actions) REPORTS_PATH=${{runner.temp}}/reports_dir - REPORTS_PATH=${{runner.temp}}/reports_dir TEMP_PATH=${{runner.temp}}/report_check EOF - name: Download json reports From 9a946c0213380bb3cff407b2016f982950cc5b23 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:35:13 +0200 Subject: [PATCH 360/372] Add typing --- tests/ci/build_report_check.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 7ca958bd745..e47ca3a7811 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -4,6 +4,8 @@ import json import logging import os import sys +from typing import Dict, List, Tuple + from github import Github from env_helper import ( @@ -44,7 +46,7 @@ class BuildResult: self.with_coverage = with_coverage -def group_by_artifacts(build_urls): +def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]: groups = { "apk": [], "deb": [], @@ -52,7 +54,7 @@ def group_by_artifacts(build_urls): "tgz": [], "rpm": [], "performance": [], - } + } # type: Dict[str, List[str]] for url in build_urls: if url.endswith("performance.tgz"): groups["performance"].append(url) @@ -74,7 +76,9 @@ def group_by_artifacts(build_urls): return groups -def process_report(build_report): +def process_report( + build_report, +) -> Tuple[List[BuildResult], List[List[str]], List[str]]: build_config = build_report["build_config"] build_result = BuildResult( compiler=build_config["compiler"], @@ -98,6 +102,7 @@ def process_report(build_report): build_logs_urls.append(build_report["log_url"]) found_group = True + # No one group of urls is found, a failed report if not found_group: build_results.append(build_result) build_urls.append([""]) From 7293e0164729d41f530cfcae11452005dd0055fa Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Apr 2022 09:47:27 -0400 Subject: [PATCH 361/372] some comments added --- src/Interpreters/ExpressionAnalyzer.cpp | 3 +++ src/Interpreters/QueryNormalizer.cpp | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index dd3162aa8a4..47e90e9c745 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1341,12 +1341,15 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai for (const auto & child : select_query->select()->children) select.insert(child->getAliasOrColumnName()); + /// collect columns required for interpolate expressions - + /// interpolate expression can use any available column auto find_columns = [&step, &select](IAST * function) { auto f_impl = [&step, &select](IAST * fn, auto fi) { if (auto * ident = fn->as()) { + /// exclude columns already in select expression - they are already in required list if (select.count(ident->getColumnName()) == 0) step.addRequiredOutput(ident->getColumnName()); return; diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 3c0a965dfa4..28b2195c9c3 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -135,6 +135,7 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & static bool needVisitChild(const ASTPtr & child) { + /// exclude interpolate elements - they are not subject for normalization and will be processed in filling transform return !(child->as() || child->as() || child->as()); } From 11897d56b911d1a98ecbc5625f24e0a5879210cb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 17:03:27 +0200 Subject: [PATCH 362/372] Upload ccache for the first run in PR --- tests/ci/ccache_utils.py | 7 ++++++- tests/ci/fast_test_check.py | 8 +++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/ci/ccache_utils.py b/tests/ci/ccache_utils.py index 7b0b0f01aa3..bd155b02cb4 100644 --- a/tests/ci/ccache_utils.py +++ b/tests/ci/ccache_utils.py @@ -57,10 +57,12 @@ def dowload_file_with_progress(url, path): def get_ccache_if_not_exists( path_to_ccache_dir, s3_helper, current_pr_number, temp_path -): +) -> int: + """returns: number of PR for downloaded PR. -1 if ccache not found""" ccache_name = os.path.basename(path_to_ccache_dir) cache_found = False prs_to_check = [current_pr_number] + ccache_pr = -1 if current_pr_number != 0: prs_to_check.append(0) for pr_number in prs_to_check: @@ -87,6 +89,7 @@ def get_ccache_if_not_exists( decompress_fast(compressed_cache, path_to_decompress) logging.info("Files on path %s", os.listdir(path_to_decompress)) cache_found = True + ccache_pr = pr_number break if cache_found: break @@ -98,6 +101,8 @@ def get_ccache_if_not_exists( else: logging.info("ccache downloaded") + return ccache_pr + def upload_ccache(path_to_ccache_dir, s3_helper, current_pr_number, temp_path): logging.info("Uploading cache %s for pr %s", path_to_ccache_dir, current_pr_number) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 41f1f9a88e8..f4a695eac1a 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -113,7 +113,10 @@ if __name__ == "__main__": cache_path = os.path.join(caches_path, "fasttest") logging.info("Will try to fetch cache for our build") - get_ccache_if_not_exists(cache_path, s3_helper, pr_info.number, temp_path) + ccache_for_pr = get_ccache_if_not_exists( + cache_path, s3_helper, pr_info.number, temp_path + ) + upload_master_ccache = ccache_for_pr in (-1, 0) if not os.path.exists(cache_path): logging.info("cache was not fetched, will create empty dir") @@ -179,6 +182,9 @@ if __name__ == "__main__": logging.info("Will upload cache") upload_ccache(cache_path, s3_helper, pr_info.number, temp_path) + if upload_master_ccache: + logging.info("Will upload a fallback cache for master") + upload_ccache(cache_path, s3_helper, 0, temp_path) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, NAME, test_results) From 2588f8011f84e28ce76999f5392ce6c694dba737 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Apr 2022 11:13:29 -0400 Subject: [PATCH 363/372] comment fix --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 47e90e9c745..5e5931713e0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1349,7 +1349,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { if (auto * ident = fn->as()) { - /// exclude columns already in select expression - they are already in required list + /// exclude columns from select expression - they are already available if (select.count(ident->getColumnName()) == 0) step.addRequiredOutput(ident->getColumnName()); return; From faa26b4377a1f5f1d27b3c7178918ab51037dcae Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Apr 2022 15:36:18 +0200 Subject: [PATCH 364/372] Improve report and variables naming --- tests/ci/build_report_check.py | 91 ++++++++++++++++++---------------- 1 file changed, 48 insertions(+), 43 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index e47ca3a7811..44cc45390a5 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -115,7 +115,7 @@ def get_build_name_from_file_name(file_name): return file_name.replace("build_urls_", "").replace(".json", "") -if __name__ == "__main__": +def main(): logging.basicConfig(level=logging.INFO) reports_path = REPORTS_PATH temp_path = TEMP_PATH @@ -125,7 +125,7 @@ if __name__ == "__main__": os.makedirs(temp_path) build_check_name = sys.argv[1] - reports_length = int(sys.argv[2]) if len(sys.argv) > 2 else 0 + required_builds = int(sys.argv[2]) if len(sys.argv) > 2 else 0 gh = Github(get_best_robot_token()) pr_info = PRInfo() @@ -134,19 +134,20 @@ if __name__ == "__main__": logging.info("Check is already finished according to github status, exiting") sys.exit(0) - reports_order = CI_CONFIG["builds_report_config"][build_check_name] - logging.info("My reports list %s", reports_order) + builds_for_check = CI_CONFIG["builds_report_config"][build_check_name] + logging.info("My reports list %s", builds_for_check) + required_builds = required_builds or len(builds_for_check) - build_reports_map = {} - for root, dirs, files in os.walk(reports_path): + # Collect reports from json artifacts + builds_report_map = {} + for root, _, files in os.walk(reports_path): for f in files: if f.startswith("build_urls_") and f.endswith(".json"): logging.info("Found build report json %s", f) build_name = get_build_name_from_file_name(f) - if build_name in reports_order: + if build_name in builds_for_check: with open(os.path.join(root, f), "rb") as file_handler: - build_report = json.load(file_handler) - build_reports_map[build_name] = build_report + builds_report_map[build_name] = json.load(file_handler) else: logging.info( "Skipping report %s for build %s, it's not in our reports list", @@ -154,44 +155,45 @@ if __name__ == "__main__": build_name, ) - reports_length = reports_length or len(reports_order) - some_builds_are_missing = len(build_reports_map) < reports_length - - if some_builds_are_missing: - logging.info( - "Expected to get %s build results, got %s", - len(reports_order), - len(build_reports_map), - ) - else: - logging.info("Got exactly %s builds", len(build_reports_map)) - + # Sort reports by config order build_reports = [ - build_reports_map[build_name] - for build_name in reports_order - if build_name in build_reports_map + builds_report_map[build_name] + for build_name in builds_for_check + if build_name in builds_report_map ] - build_results = [] - build_artifacts = [] + some_builds_are_missing = len(build_reports) < required_builds + if some_builds_are_missing: + logging.warning( + "Expected to get %s build results, got only %s", + required_builds, + len(build_reports), + ) + else: + logging.info("Got exactly %s builds", len(builds_report_map)) + + # Group build artifacts by groups + build_results = [] # type: List[BuildResult] + build_artifacts = [] # build_logs = [] for build_report in build_reports: build_result, build_artifacts_url, build_logs_url = process_report(build_report) - logging.info("Got %s result for report", len(build_result)) - build_results += build_result - build_artifacts += build_artifacts_url - build_logs += build_logs_url + logging.info( + "Got %s artifact groups for build report report", len(build_result) + ) + build_results.extend(build_result) + build_artifacts.extend(build_artifacts_url) + build_logs.extend(build_logs_url) - logging.info("Totally got %s results", len(build_results)) - if len(build_results) == 0: - logging.info("No builds, failing check") + total_groups = len(build_results) + logging.info("Totally got %s artifact groups", total_groups) + if total_groups == 0: + logging.error("No success builds, failing check") sys.exit(1) s3_helper = S3Helper("https://s3.amazonaws.com") - pr_info = PRInfo() - branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" if pr_info.number != 0: @@ -224,9 +226,10 @@ if __name__ == "__main__": report_path, s3_path_prefix + "/report.html" ) logging.info("Report url %s", url) + print(f"::notice ::Report url: {url}") - total_builds = len(build_results) - ok_builds = 0 + # Prepare a commit status + ok_groups = 0 summary_status = "success" for build_result in build_results: if build_result.status == "failure" and summary_status != "error": @@ -235,18 +238,16 @@ if __name__ == "__main__": summary_status = "error" if build_result.status == "success": - ok_builds += 1 + ok_groups += 1 - if ok_builds == 0 or some_builds_are_missing: + if ok_groups == 0 or some_builds_are_missing: summary_status = "error" addition = "" if some_builds_are_missing: - addition = f"({len(build_reports_map)} < {reports_length})" + addition = f"({len(build_reports)} of {required_builds} builds are OK)" - description = f"{ok_builds}/{total_builds} builds are OK {addition}" - - print(f"::notice ::Report url: {url}") + description = f"{ok_groups}/{total_groups} artifact groups are OK {addition}" commit = get_commit(gh, pr_info.sha) commit.create_status( @@ -258,3 +259,7 @@ if __name__ == "__main__": if summary_status == "error": sys.exit(1) + + +if __name__ == "__main__": + main() From fc62e95d76de675099c4bed11c975526c0a8db19 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Apr 2022 22:24:45 +0200 Subject: [PATCH 365/372] Fix --- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 410d1ae6dd4..bf71f4f1f86 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -53,6 +53,8 @@ Pipe StorageSystemRemoteDataPaths::read( { std::vector remote_paths_by_local_path; disk->getRemotePathsRecursive("store", remote_paths_by_local_path); + disk->getRemotePathsRecursive("data", remote_paths_by_local_path); + disk->getRemotePathsRecursive("metadata", remote_paths_by_local_path); FileCachePtr cache; auto cache_base_path = disk->getCacheBasePath(); From 51dd39b42fee59079c2b09e235ba601cd000222c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Apr 2022 22:46:51 +0200 Subject: [PATCH 366/372] Update StorageSystemRemoteDataPaths.cpp --- src/Storages/System/StorageSystemRemoteDataPaths.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index bf71f4f1f86..dbce4c25773 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -54,7 +54,6 @@ Pipe StorageSystemRemoteDataPaths::read( std::vector remote_paths_by_local_path; disk->getRemotePathsRecursive("store", remote_paths_by_local_path); disk->getRemotePathsRecursive("data", remote_paths_by_local_path); - disk->getRemotePathsRecursive("metadata", remote_paths_by_local_path); FileCachePtr cache; auto cache_base_path = disk->getCacheBasePath(); From 9d31c44d39b16b56864bd7db935d43fcb82ac8dc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 12 Apr 2022 12:01:51 +0800 Subject: [PATCH 367/372] fix bug of read buffer from hdfs --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 1bafa49e55b..af4b324cd77 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -30,7 +30,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory {})", offset, read_until_position - 1); + if (read_until_position < file_offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", file_offset, read_until_position - 1); - num_bytes_to_read = read_until_position - offset; + num_bytes_to_read = read_until_position - file_offset; } else { @@ -94,28 +94,28 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemorynext(); if (result) - BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset); /// use the buffer returned by `impl` + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); /// use the buffer returned by `impl` return result; } From 6b07105de65f376d723025a85b178e30bf92ac76 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 11 Apr 2022 22:38:23 -0700 Subject: [PATCH 368/372] fix typo in cmake message --- cmake/strip_binary.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/strip_binary.cmake b/cmake/strip_binary.cmake index 1f24790a159..2d6a3888503 100644 --- a/cmake/strip_binary.cmake +++ b/cmake/strip_binary.cmake @@ -46,7 +46,7 @@ macro(clickhouse_make_empty_debug_info_for_nfpm) add_custom_command(TARGET ${EMPTY_DEBUG_TARGET} POST_BUILD COMMAND mkdir -p "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug" COMMAND touch "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug/${EMPTY_DEBUG_TARGET}.debug" - COMMENT "Addiding empty debug info for NFPM" VERBATIM + COMMENT "Adding empty debug info for NFPM" VERBATIM ) install(FILES "${EMPTY_DEBUG_DESTINATION_DIR}/lib/debug/${EMPTY_DEBUG_TARGET}.debug" DESTINATION "${CMAKE_INSTALL_LIBDIR}/debug/${CMAKE_INSTALL_FULL_BINDIR}" COMPONENT clickhouse) From 0ae820550d0355b00ed8b1fc8bf1447b2022fddc Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Tue, 12 Apr 2022 14:30:53 +0800 Subject: [PATCH 369/372] fix ut error sometimes --- src/Coordination/tests/gtest_coordination.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 7dfd451e111..a64fde5ad05 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -407,7 +407,7 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); changelog.compact(6); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -1469,7 +1469,7 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) } changelog_2.compact(105); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); @@ -1489,7 +1489,7 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) } changelog_3.compact(125); - std::this_thread::sleep_for(std::chrono::microseconds(200)); + std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_111_117.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_118_124.bin" + params.extension)); From abf8c3c5ad8dc302864e9e9cb5cf57fc709cc5a3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Apr 2022 09:51:26 +0200 Subject: [PATCH 370/372] Fix and improve cancel-rerun lambda - Fix rerun for DocsCheck - Additional check for workflow head sha1 - Fix workflow debug info - Fix event_data parsing - Always print debug info --- .../cancel_and_rerun_workflow_lambda/app.py | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 8deed464012..4cce1e5024a 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 from collections import namedtuple -from typing import Any, Dict +from typing import Any, Dict, List import json import time @@ -11,7 +11,7 @@ import boto3 # type: ignore NEED_RERUN_OR_CANCELL_WORKFLOWS = { "PullRequestCI", - "Docs", + "DocsCheck", "DocsRelease", "BackportPR", } @@ -93,7 +93,9 @@ WorkflowDescription = namedtuple( def get_workflows_description_for_pull_request(pull_request_event): + head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] + head_sha = pull_request_event["head"]["sha"] print("PR", pull_request_event["number"], "has head ref", head_branch) workflows_data = [] workflows = _exec_get_with_retry( @@ -111,17 +113,23 @@ def get_workflows_description_for_pull_request(pull_request_event): print("Too many workflows found") break + DEBUG_INFO["workflows"] = [] # type: List[Dict[str, str]] workflow_descriptions = [] for workflow in workflows_data: - DEBUG_INFO["workflow"] = workflow # Some time workflow["head_repository"]["full_name"] is None if workflow["head_repository"] is None: continue + DEBUG_INFO["workflows"].append( + { + "full_name": workflow["head_repository"]["full_name"], + "name": workflow["name"], + } + ) # unfortunately we cannot filter workflows from forks in request to API # so doing it manually if ( - workflow["head_repository"]["full_name"] - == pull_request_event["head"]["repo"]["full_name"] + workflow["head_sha"] == head_sha + and workflow["head_repository"]["full_name"] == head_repo and workflow["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS ): workflow_descriptions.append( @@ -170,7 +178,7 @@ def exec_workflow_url(urls_to_cancel, token): def main(event): token = get_token_from_aws() DEBUG_INFO["event_body"] = event["body"] - event_data = event["body"] + event_data = json.loads(event["body"]) print("Got event for PR", event_data["number"]) action = event_data["action"] @@ -220,7 +228,6 @@ def main(event): def handler(event, _): try: main(event) - except Exception: + finally: for name, value in DEBUG_INFO.items(): print(f"Value of {name}: ", value) - raise From 30a9fce67ac46d11fcf2f5a0611bded1688abbd6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Apr 2022 10:59:44 +0200 Subject: [PATCH 371/372] Rebuild the least docker layer --- tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile b/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile index f53be71a893..0d50224c51d 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile +++ b/tests/ci/cancel_and_rerun_workflow_lambda/Dockerfile @@ -1,13 +1,13 @@ FROM public.ecr.aws/lambda/python:3.9 -# Copy function code -COPY app.py ${LAMBDA_TASK_ROOT} - # Install the function's dependencies using file requirements.txt # from your project folder. COPY requirements.txt . RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + # Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) CMD [ "app.handler" ] From 8f822af3580c4d170a10e7d62687fb6b1c8d8a33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Apr 2022 11:47:22 +0200 Subject: [PATCH 372/372] Fix mongodb test with new cert --- tests/integration/helpers/mongo_cert.pem | 89 +++++++++++++----------- 1 file changed, 47 insertions(+), 42 deletions(-) diff --git a/tests/integration/helpers/mongo_cert.pem b/tests/integration/helpers/mongo_cert.pem index 9e18b1d4469..b9c3b83e473 100644 --- a/tests/integration/helpers/mongo_cert.pem +++ b/tests/integration/helpers/mongo_cert.pem @@ -1,44 +1,49 @@ ------BEGIN RSA PRIVATE KEY----- -MIIEpAIBAAKCAQEAtz2fpa8hyUff8u8jYlh20HbkOO8hQi64Ke2Prack2Br0lhOr -1MI6I8nVk5iDrt+7ix2Cnt+2aZKb6HJv0CG1V25yWg+jgsXeIT1KHTJf8rTmYxhb -t+ye+S1Z0h/Rt+xqSd9XXfzOLPGHYfyx6ZQ4AumO/HoEFD4IH/qiREjwtOfRXuhz -CohqtUTyYR7pJmZqBSuGac461WVRisnjfKRxeVa3itc84/RgktgYej2x4PQBFk13 -xAXKrWmHkwdgWklTuuK8Gtoqz65Y4/J9CSl+Bd08QDdRnaVvq1u1eNTZg1BVyeRv -jFYBMSathKASrng5nK66Fdilw6tO/9khaP0SDQIDAQABAoIBAAm/5qGrKtIJ1/mW -Dbzq1g+Lc+MvngZmc/gPIsjrjsNM09y0WT0txGgpEgsTX1ZLoy/otw16+7qsSU1Z -4WcilAJ95umx0VJg8suz9iCNkJtaUrPNFPw5Q9AgQJo0hTUTCCi8EGr4y4OKqlhl -WJYEA+LryGbYmyT0k/wXmtClTOFjKS09mK4deQ1DqbBxayR9MUZgRJzEODA8eGXs -Rc6fJUenMVNMzIVLgpossRtKImoZNcf5UtCKL3HECunndQeMu4zuqLMU+EzL1F/o -iHKF7v3CVmsK0OxNJfOfT0abN3XaJttFwTJyghQjgP8OX1IKjlj3vo9xwEDfVUEf -GVIER0UCgYEA2j+kjaT3Dw2lNlBEsA8uvVlLJHBc9JBtMGduGYsIEgsL/iStqXv4 -xoA9N3CwkN/rrQpDfi/16DMXAAYdjGulPwiMNFBY22TCzhtPC2mAnfaSForxwZCs -lwc3KkIloo3N5XvN78AuZf8ewiS+bOEj+HHHqqSb1+u/csuaXO9neesCgYEA1u/I -Mlt/pxJkH+c3yOskwCh/CNhq9szi8G9LXROIQ58BT2ydJSEPpt7AhUTtQGimQQTW -KLiffJSkjuVaFckR1GjCoAmFGYw9wUb+TmFNScz5pJ2dXse8aBysAMIQfEIcRAEa -gKnkLBH6nw3+/Hm3xwoBc35t8Pa2ek7LsWDfbecCgYBhilQW4gVw+t49uf4Y2ZBA -G+pTbMx+mRXTrkYssFB5D+raOLZMqxVyUdoKLxkahpkkCxRDD1hN4JeE8Ta/jVSb -KUzQDKDJ3OybhOT86rgK4SpFXO/TXL9l+FmVT17WmZ3N1Fkjr7aM60pp5lYc/zo+ -TUu5XjwwcjJsMcbZhj2u5QKBgQCDNuUn4PYAP9kCJPzIWs0XxmEvPDeorZIJqFgA -3XC9n2+EVlFlHlbYz3oGofqY7Io6fUJkn7k1q+T+G4QwcozA+KeAXe90lkoJGVcc -8IfnewwYc+RjvVoG0SIsYE0CHrX0yhus2oqiYON4gGnfJkuMZk5WfKOPjH4AEuSF -SBd+lwKBgQCHG/DA6u2mYmezPF9hebWFoyAVSr2PDXDhu8cNNHCpx9GewJXhuK/P -tW8mazHzUuJKBvmaUXDIXFh4K6FNhjH16p5jR1w3hsPE7NEZhjfVRaUYPmBqaOYR -jp8H+Sh5g4Rwbtfp6Qhu6UAKi/y6Vozs5GkJtSiNrjNDVrD+sGGrXA== ------END RSA PRIVATE KEY----- +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQC3uaPiZMfjPBBE +yDEYJsJIoriu0SaC80uTmPM7bFpnOOXOBvbT4wD2q+uVaLQifKtPTgZAkP5Y3rX8 +S5TOzaLsNp68S1Ja/EzxQUolOSgb4A948TTiUTrTjfMxsPRhmxXTjozWV8CFtL9P +Lg6H+55oyQOJedWYe1kSWRJQayXSweBK5qjOPi2qDF/xdFRQuMivpBUar/b/E9GQ +RKpIaoqMYsl/WF/tReb4N658UxkVlFdR8s48UoA9LfJLMPr4N+QDTfvtcT2bYlpT +4a9b6IXa9BQKCw3AKfTqEPO1XunH//iLNkt1bLtqgZNyT/tY0tLY3EKMXIDuRBVn +KCbfVJ1RAgMBAAECggEAJFCjXiqBgB7tMEtJuPZgTK8tRhC9RgEFHUWMPmCqdeC/ +O7wQqc0i8Z8Fz+CESpTN370Sa0y9mZ9b5WSjI0VuQLaDJcDVpHpeUwmOuFDV5ryh +EkzLITjhIdPbECVkCK7433o7yFpMCaGydtopsSNBKoEhG9ljKOKotoG4pwCm10N5 +K9Qepj82OjRhLkpmuiMFb4/vvOm5dglYmkq5+n/fdUYFtrYr3NvMSCTlietPHDgV +Wb3strvk1g9ARWfa2j7Q6moF2sbyob9zVLoRiD9VgmNB60v7QAJxDctVkbOoDgKp +uN2fkxTHwlOPAO6Zhgnie11jnZr1711TFxmEfMkSKQKBgQDqpB8m0hSJsWLKWxQK +yx+5Xgs+Cr8gb0AYHJQ87obj2XqwXLpBSMrkzTn6vIGRv+NMSfiM/755RUm5aJPY +om+7F68JEIL26ZA7bIfjHhV5o9fvpo+6N6cJyR08Q/KkF8Tej9K4qQec0W/jtKeZ +KAJ1k7/BBuN82iTtEJ3GWBaaRwKBgQDIcwQrGlyyXqnBK25gl/E1Ng+V3p/2sy98 +1BpEshxen4KorHEXCJArydELtvK/ll6agil6QebrJN5dtYOOgvcDTu1mQjdUPN3C +VXpSQ0L8XxfyTNYQTWON9wJGL1pzlTiyHvlSrQFsFWMUoxrqndWIIRtrXjap1npp +HDrcqy2/pwKBgB5fHhUlTjlAd7wfq+l1v2Z8ENJ4C6NEIzS7xkhYy6cEiIf5iLZY +mMKi+eVFrzPRdbdzP7Poipwh5tgT/EcnR3UdLK/srjcNpni6pKA2TatQFOxVT/dX +qsxudtVNKkQpO3dfgHQclPqsdWIxCRye/CqB9Gkk3h9UEUGKTBHXZx2TAoGAF0tG +cLvfidr2Xzxs10zQ+x4NMZ1teX3ZRuhfJRyNr3FZ/cAMZGDaYDxTzsiz7Q/Mbqgx +qcN+0lS2gq1VXHpbukaxz/Bh/agVHUBRtr2aSznBzqafOcXEi/roiL94A3aT4B85 +WiJAyA60NPG/bwRojClMxm1sbNA/6XceYAaEioECgYEA3m88G3UwizfJAsfT5H5K +3HXNYzQ1XGrA8shI0kxeqfNP5qmTfH5q/K2VMWeShT3F/9Ytgc+H8c9XP1wKq7Zl +6AtmdDOeLzHkgwVK0p20/Wh2Qjw4ikJLdM+y8wnfMiwCXWQxoh1X905EwNtyBc2Z +9S3G5CXldFHC4NGdx0vetiE= +-----END PRIVATE KEY----- -----BEGIN CERTIFICATE----- -MIICqDCCAZACFBdaMnuT0pWhmrh05UT3HXJ+kI0yMA0GCSqGSIb3DQEBCwUAMA0x -CzAJBgNVBAMMAmNhMB4XDTIxMDQwNjE3MDQxNVoXDTIyMDQwNjE3MDQxNVowFDES -MBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKC -AQEAtz2fpa8hyUff8u8jYlh20HbkOO8hQi64Ke2Prack2Br0lhOr1MI6I8nVk5iD -rt+7ix2Cnt+2aZKb6HJv0CG1V25yWg+jgsXeIT1KHTJf8rTmYxhbt+ye+S1Z0h/R -t+xqSd9XXfzOLPGHYfyx6ZQ4AumO/HoEFD4IH/qiREjwtOfRXuhzCohqtUTyYR7p -JmZqBSuGac461WVRisnjfKRxeVa3itc84/RgktgYej2x4PQBFk13xAXKrWmHkwdg -WklTuuK8Gtoqz65Y4/J9CSl+Bd08QDdRnaVvq1u1eNTZg1BVyeRvjFYBMSathKAS -rng5nK66Fdilw6tO/9khaP0SDQIDAQABMA0GCSqGSIb3DQEBCwUAA4IBAQAct2If -isMLHIqyL9GjY4b0xcxF4svFU/DUwNanStmoFMW1ifPf1cCqeMzyQOxBCDdMs0RT -hBbDYHW0BMXDqYIr3Ktbu38/3iVyr3pb56YOCKy8yHXpmKEaUBhCknSLcQyvNfeS -tM+DWsKFTZfyR5px+WwXbGKVMYwLaTON+/wcv1MeKMig3CxluaCpEJVYYwAiUc4K -sgvQNAunwGmPLPoXtUnpR2ZWiQA5R6yjS1oIe+8vpryFP6kjhWs0HR0jZEtLulV5 -WXUuxkqTXiBIvYpsmusoR44e9rptwLbV1wL/LUScRt9ttqFM3N5/Pof+2UwkSjGB -GAyPmw0Pkqtt+lva +MIIDazCCAlOgAwIBAgIUO9pfiBMsADdk9nBMHs10n8kaIr8wDQYJKoZIhvcNAQEL +BQAwRTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDAeFw0yMjA0MTIwOTQxNDVaFw0yNTAx +MDUwOTQxNDVaMEUxCzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEw +HwYDVQQKDBhJbnRlcm5ldCBXaWRnaXRzIFB0eSBMdGQwggEiMA0GCSqGSIb3DQEB +AQUAA4IBDwAwggEKAoIBAQC3uaPiZMfjPBBEyDEYJsJIoriu0SaC80uTmPM7bFpn +OOXOBvbT4wD2q+uVaLQifKtPTgZAkP5Y3rX8S5TOzaLsNp68S1Ja/EzxQUolOSgb +4A948TTiUTrTjfMxsPRhmxXTjozWV8CFtL9PLg6H+55oyQOJedWYe1kSWRJQayXS +weBK5qjOPi2qDF/xdFRQuMivpBUar/b/E9GQRKpIaoqMYsl/WF/tReb4N658UxkV +lFdR8s48UoA9LfJLMPr4N+QDTfvtcT2bYlpT4a9b6IXa9BQKCw3AKfTqEPO1XunH +//iLNkt1bLtqgZNyT/tY0tLY3EKMXIDuRBVnKCbfVJ1RAgMBAAGjUzBRMB0GA1Ud +DgQWBBSx7Tx8W4c6wjW0qkeG7CAMLY7YkjAfBgNVHSMEGDAWgBSx7Tx8W4c6wjW0 +qkeG7CAMLY7YkjAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQAb +/Up/LEIdwhiN/S3HolxY2D2BrTpKHLQuggBN4+gZlK5OksCkM46LYlP/ruHXCxbR +mQoRhmooj4TvkKyBwzvKq76O+OuRtBhXzRipnBbNTqFPLf9enJUrut8lsFrI+pdl +Nn4PSGGbFPpQ5vFRCktczwwYh0zLuZ/1DbFsbRWlDnZdvoWZdfV0qsvcBRK2DXDI +29xSfw897OpITIkaryZigQVsKv8TXhfsaq9PUuH0/z84S82QG5fR6FzULofgkylb +wXvwaSdcu3k4Lo8j77BEAEvlH8Ynja0eojx5Avl9h4iw/IOQKE4GAg56CzcequLv +clPlaBBWoD6yn+q4NhLF -----END CERTIFICATE-----