提交 5bc6bd55 编写于 作者: A Alexey Zatelepin

allow PreCommitted parts to contain each other, clean up code

上级 958a6f09
......@@ -145,20 +145,24 @@ public:
using DataParts = std::set<DataPartPtr, LessDataPart>;
using DataPartsVector = std::vector<DataPartPtr>;
/// Some operations on the set of parts return a Transaction object.
/// Auxiliary object to add a set of parts into the working set in two steps:
/// * First, as PreCommitted parts (the parts are ready, but not yet in the active set).
/// * Next, if commit() is called, the parts are added to the active set and the parts that are
/// covered by them are marked Outdated.
/// If neither commit() nor rollback() was called, the destructor rollbacks the operation.
class Transaction : private boost::noncopyable
{
public:
Transaction() {}
void commit();
/// Return parts marked Obsolete as a result of the transaction commit.
DataPartsVector commit();
void rollback();
bool isEmpty() const
{
return parts_to_add_on_rollback.empty() && parts_to_remove_on_rollback.empty();
return precommitted_parts.empty();
}
~Transaction()
......@@ -172,23 +176,18 @@ public:
tryLogCurrentException("~MergeTreeData::Transaction");
}
}
private:
friend class MergeTreeData;
MergeTreeData * data = nullptr;
/// What to do on rollback.
DataPartsVector parts_to_remove_on_rollback;
DataPartsVector parts_to_add_on_rollback;
DataParts precommitted_parts;
void clear()
{
data = nullptr;
parts_to_remove_on_rollback.clear();
parts_to_add_on_rollback.clear();
precommitted_parts.clear();
}
void replaceParts(DataPartState move_precommitted_to, DataPartState move_committed_to, bool remove_without_delay);
};
/// An object that stores the names of temporary files created in the part directory during ALTER of its
......@@ -368,14 +367,17 @@ public:
/// If until is non-null, wake up from the sleep earlier if the event happened.
void delayInsertIfNeeded(Poco::Event * until = nullptr);
/// Renames temporary part to a permanent part and adds it to the working set.
/// If increment != nullptr, part index is determing using increment. Otherwise part index remains unchanged.
/// Renames temporary part to a permanent part and adds it to the parts set.
/// It is assumed that the part does not intersect with existing parts.
/// If out_transaction != nullptr, sets it to an object allowing to rollback part addition (but not the renaming).
/// If increment != nullptr, part index is determing using increment. Otherwise part index remains unchanged.
/// If out_transaction != nullptr, adds the part in the PreCommitted state (the part will be added to the
/// active set later with out_transaction->commit()).
/// Else, commits the part immediately.
void renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
/// The same as renameTempPartAndAdd but the part can intersect existing parts.
/// Deletes and returns all parts covered by the added part (in ascending order).
/// 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);
......@@ -573,8 +575,8 @@ private:
/// Work with data parts
struct TagByName{};
struct TagByStateAndName{};
struct TagByInfo{};
struct TagByStateAndInfo{};
static const MergeTreePartInfo & dataPartPtrToInfo(const DataPartPtr & part)
{
......@@ -588,14 +590,14 @@ private:
using DataPartsIndexes = boost::multi_index_container<DataPartPtr,
boost::multi_index::indexed_by<
/// Index by Name
/// Index by Info
boost::multi_index::ordered_unique<
boost::multi_index::tag<TagByName>,
boost::multi_index::tag<TagByInfo>,
boost::multi_index::global_fun<const DataPartPtr &, const MergeTreePartInfo &, dataPartPtrToInfo>
>,
/// Index by (State, Name), is used to obtain ordered slices of parts with the same state
/// Index by (State, Info), is used to obtain ordered slices of parts with the same state
boost::multi_index::ordered_unique<
boost::multi_index::tag<TagByStateAndName>,
boost::multi_index::tag<TagByStateAndInfo>,
boost::multi_index::global_fun<const DataPartPtr &, DataPartStateAndInfo, dataPartPtrToStateAndInfo>,
LessStateDataPart
>
......@@ -605,16 +607,16 @@ private:
/// Current set of data parts.
mutable std::mutex data_parts_mutex;
DataPartsIndexes data_parts_indexes;
DataPartsIndexes::index<TagByName>::type & data_parts_by_name;
DataPartsIndexes::index<TagByStateAndName>::type & data_parts_by_state_and_name;
DataPartsIndexes::index<TagByInfo>::type & data_parts_by_info;
DataPartsIndexes::index<TagByStateAndInfo>::type & data_parts_by_state_and_info;
using DataPartIteratorByAndName = DataPartsIndexes::index<TagByName>::type::iterator;
using DataPartIteratorByStateAndName = DataPartsIndexes::index<TagByStateAndName>::type::iterator;
using DataPartIteratorByInfo = DataPartsIndexes::index<TagByInfo>::type::iterator;
using DataPartIteratorByStateAndInfo = DataPartsIndexes::index<TagByStateAndInfo>::type::iterator;
boost::iterator_range<DataPartIteratorByStateAndName> getDataPartsStateRange(DataPartState state) const
boost::iterator_range<DataPartIteratorByStateAndInfo> getDataPartsStateRange(DataPartState state) const
{
auto begin = data_parts_by_state_and_name.lower_bound(state, LessStateDataPart());
auto end = data_parts_by_state_and_name.upper_bound(state, LessStateDataPart());
auto begin = data_parts_by_state_and_info.lower_bound(state, LessStateDataPart());
auto end = data_parts_by_state_and_info.upper_bound(state, LessStateDataPart());
return {begin, end};
}
......@@ -623,25 +625,25 @@ private:
return [state] (const DataPartPtr & part) { part->state = state; };
}
void modifyPartState(DataPartIteratorByStateAndName it, DataPartState state)
void modifyPartState(DataPartIteratorByStateAndInfo it, DataPartState state)
{
if (!data_parts_by_state_and_name.modify(it, getStateModifier(state)))
if (!data_parts_by_state_and_info.modify(it, getStateModifier(state)))
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
void modifyPartState(DataPartIteratorByAndName it, DataPartState state)
void modifyPartState(DataPartIteratorByInfo it, DataPartState state)
{
if (!data_parts_by_state_and_name.modify(data_parts_indexes.project<TagByStateAndName>(it), getStateModifier(state)))
if (!data_parts_by_state_and_info.modify(data_parts_indexes.project<TagByStateAndInfo>(it), getStateModifier(state)))
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
void modifyPartState(const DataPartPtr & part, DataPartState state)
{
auto it = data_parts_by_name.find(part->info);
if (it == data_parts_by_name.end() || (*it).get() != part.get())
throw Exception("Part " + part->name + " is not exists", ErrorCodes::LOGICAL_ERROR);
auto it = data_parts_by_info.find(part->info);
if (it == data_parts_by_info.end() || (*it).get() != part.get())
throw Exception("Part " + part->name + " doesn't exist", ErrorCodes::LOGICAL_ERROR);
if (!data_parts_by_state_and_name.modify(data_parts_indexes.project<TagByStateAndName>(it), getStateModifier(state)))
if (!data_parts_by_state_and_info.modify(data_parts_indexes.project<TagByStateAndInfo>(it), getStateModifier(state)))
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
......@@ -672,7 +674,14 @@ private:
void removePartContributionToColumnSizes(const DataPartPtr & part);
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
DataPartPtr getAnyPartInPartition(const String & partition_id, std::unique_lock<std::mutex> & data_parts_lock);
DataPartPtr getAnyPartInPartition(const String & partition_id, std::lock_guard<std::mutex> & data_parts_lock);
/// Return parts in the Committed set that are covered by the new_part_info or the part that covers it.
/// Will check that the new part doesn't already exist and that it doesn't intersect existing part.
DataPartsVector getActivePartsToReplace(
const MergeTreePartInfo & new_part_info,
DataPartPtr & out_covering_part,
std::lock_guard<std::mutex> & data_parts_lock) const;
/// Checks whether the column is in the primary key.
bool isPrimaryKeyColumn(const ASTPtr &node) const;
......
......@@ -29,6 +29,11 @@ struct MergeTreePartInfo
< std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level);
}
bool operator==(const MergeTreePartInfo & rhs) const
{
return !(*this < rhs || rhs < *this);
}
/// Contains another part (obtained after merging another part with some other)
bool contains(const MergeTreePartInfo & rhs) const
{
......
......@@ -2259,13 +2259,13 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
checkPartAndAddToZooKeeper(part, ops, part_name);
MergeTreeData::Transaction transaction;
replaced_parts = data.renameTempPartAndReplace(part, nullptr, &transaction);
data.renameTempPartAndReplace(part, nullptr, &transaction);
/// Do not commit if the part is obsolete
if (!transaction.isEmpty())
{
getZooKeeper()->multi(ops);
transaction.commit();
replaced_parts = transaction.commit();
}
/** If a quorum is tracked for this part, you must update it.
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册