提交 b738d1ba 编写于 作者: V Vitaliy Lyudvichenko

Add multi index for data_parts storage. And fixed bugs. [#CLICKHOUSE-3452]

Fixed handling of obsolete parts.
Fixed conflict resolution between simultaneous PreCommitted covering parts.
Fixed memory leak caused by ordinary MergeTree parts stucked in Deleting state.
Added hidden _state column into system.parts.
上级 34eef961
......@@ -361,6 +361,7 @@ namespace ErrorCodes
extern const int HTTP_LENGTH_REQUIRED = 381;
extern const int CANNOT_LOAD_CATBOOST_MODEL = 382;
extern const int CANNOT_APPLY_CATBOOST_MODEL = 383;
extern const int PART_IS_TEMPORARILY_LOCKED = 384;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -15,7 +15,10 @@
#include <DataStreams/GraphiteRollupSortedBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <common/RangeFiltered.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/ordered_index.hpp>
#include <boost/multi_index/global_fun.hpp>
#include <boost/range/iterator_range_core.hpp>
namespace DB
{
......@@ -102,7 +105,16 @@ public:
using DataPartStates = std::initializer_list<DataPartState>;
using DataPartStateVector = std::vector<DataPartState>;
struct DataPartPtrLess
/// Auxiliary structure for index comparison. Keep in mind lifetime of MergeTreePartInfo.
struct DataPartStateAndInfo
{
DataPartState state;
const MergeTreePartInfo & info;
DataPartStateAndInfo(DataPartState state, const MergeTreePartInfo & info) : state(state), info(info) {}
};
struct LessDataPart
{
using is_transparent = void;
......@@ -111,7 +123,28 @@ public:
bool operator()(const DataPartPtr & lhs, const DataPartPtr & rhs) const { return lhs->info < rhs->info; }
};
using DataParts = std::set<DataPartPtr, DataPartPtrLess>;
struct LessStateDataPart
{
using is_transparent = void;
bool operator() (const DataPartStateAndInfo & lhs, const DataPartStateAndInfo & rhs) const
{
return std::forward_as_tuple(static_cast<UInt8>(lhs.state), lhs.info)
< std::forward_as_tuple(static_cast<UInt8>(rhs.state), rhs.info);
}
bool operator() (DataPartStateAndInfo info, const DataPartState & state) const
{
return static_cast<size_t>(info.state) < static_cast<size_t>(state);
}
bool operator() (const DataPartState & state, DataPartStateAndInfo info) const
{
return static_cast<size_t>(state) < static_cast<size_t>(info.state);
}
};
using DataParts = std::set<DataPartPtr, LessDataPart>;
using DataPartsVector = std::vector<DataPartPtr>;
/// Some operations on the set of parts return a Transaction object.
......@@ -125,6 +158,11 @@ public:
void rollback();
bool isEmpty() const
{
return parts_to_add_on_rollback.empty() && parts_to_remove_on_rollback.empty();
}
~Transaction()
{
try
......@@ -304,22 +342,17 @@ public:
/// Returns a copy of the list so that the caller shouldn't worry about locks.
DataParts getDataParts(const DataPartStates & affordable_states) const;
DataPartsVector getDataPartsVector(const DataPartStates & affordable_states) const;
DataPartsVector getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector & out_states_snapshot) const;
/// Returns sorted list of the parts with specified states
/// out_states will contain snapshot of each part state
DataPartsVector getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const;
/// Returns a virtual container iteration only through parts with specified states
decltype(auto) getDataPartsRange(const DataPartStates & affordable_states) const
{
return createRangeFiltered(DataPart::getStatesFilter(affordable_states), data_parts);
}
/// Returns absolutely all parts (and snapshot of their states)
DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const;
/// Returns Committed parts
DataParts getDataParts() const;
DataPartsVector getDataPartsVector() const;
/// Returns all parts except Temporary and Deleting ones
DataParts getAllDataParts() const;
/// Returns an comitted part with the given name or a part containing it. If there is no such part, returns nullptr.
DataPartPtr getActiveContainingPart(const String & part_name);
......@@ -367,8 +400,8 @@ public:
/// Removes parts from data_parts, they should be in Deleting state
void removePartsFinally(const DataPartsVector & parts);
/// Delete irrelevant parts.
void clearOldParts();
/// Delete irrelevant parts from memory and disk.
void clearOldPartsFromFilesystem();
/// Deleate all directories which names begin with "tmp"
/// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime
......@@ -530,10 +563,81 @@ private:
String log_name;
Logger * log;
/// Work with data parts
struct TagByName{};
struct TagByStateAndName{};
static const MergeTreePartInfo & dataPartPtrToInfo(const DataPartPtr & part)
{
return part->info;
}
static DataPartStateAndInfo dataPartPtrToStateAndInfo(const DataPartPtr & part)
{
return {part->state, part->info};
};
using DataPartsIndexes = boost::multi_index_container<DataPartPtr,
boost::multi_index::indexed_by<
/// Index by Name
boost::multi_index::ordered_unique<
boost::multi_index::tag<TagByName>,
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
boost::multi_index::ordered_unique<
boost::multi_index::tag<TagByStateAndName>,
boost::multi_index::global_fun<const DataPartPtr &, DataPartStateAndInfo, dataPartPtrToStateAndInfo>,
LessStateDataPart
>
>
>;
/// Current set of data parts.
DataParts data_parts;
mutable std::mutex data_parts_mutex;
// TODO: this mutex could be a bottleneck. If so, make it shared, and split parts onto partitions
DataPartsIndexes data_parts_indexes;
DataPartsIndexes::index<TagByName>::type & data_parts_by_name;
DataPartsIndexes::index<TagByStateAndName>::type & data_parts_by_state_and_name;
using DataPartIteratorByAndName = DataPartsIndexes::index<TagByName>::type::iterator;
using DataPartIteratorByStateAndName = DataPartsIndexes::index<TagByStateAndName>::type::iterator;
boost::iterator_range<DataPartIteratorByStateAndName> 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());
return {begin, end};
}
static decltype(auto) getStateModifier(DataPartState state)
{
return [state] (const DataPartPtr & part) { part->state = state; };
}
void modifyPartState(DataPartIteratorByStateAndName it, DataPartState state)
{
if (!data_parts_by_state_and_name.modify(it, getStateModifier(state)))
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
void modifyPartState(DataPartIteratorByAndName it, DataPartState state)
{
if (!data_parts_by_state_and_name.modify(data_parts_indexes.project<TagByStateAndName>(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);
if (!data_parts_by_state_and_name.modify(data_parts_indexes.project<TagByStateAndName>(it), getStateModifier(state)))
throw Exception("Can't modify " + (*it)->getNameWithState(), ErrorCodes::LOGICAL_ERROR);
}
/// Used to serialize calls to grabOldParts.
std::mutex grab_old_parts_mutex;
......@@ -566,7 +670,7 @@ 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::lock_guard<std::mutex> & data_parts_lock);
DataPartPtr getAnyPartInPartition(const String & partition_id, std::unique_lock<std::mutex> & data_parts_lock);
};
}
......@@ -185,7 +185,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(
if (prev_part && part->info.partition_id == (*prev_part)->info.partition_id
&& part->info.min_block < (*prev_part)->info.max_block)
{
LOG_ERROR(log, "Part " << part->name << " intersects previous part " << (*prev_part)->name);
LOG_ERROR(log, "Part " << part->getNameWithState() << " intersects previous part " << (*prev_part)->getNameWithState());
}
prev_part = &part;
......
......@@ -38,6 +38,7 @@ namespace ErrorCodes
extern const int FORMAT_VERSION_TOO_OLD;
extern const int UNKNOWN_FORMAT;
extern const int UNEXPECTED_FILE_IN_DATA_PART;
extern const int NOT_FOUND_EXPECTED_DATA_PART;
}
......@@ -934,4 +935,16 @@ String MergeTreeDataPart::stateString() const
return stateToString(state);
}
void MergeTreeDataPart::assertState(const std::initializer_list<MergeTreeDataPart::State> & affordable_states) const
{
if (!checkState(affordable_states))
{
String states_str;
for (auto state : affordable_states)
states_str += stateToString(state) + " ";
throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART);
}
}
}
......@@ -187,17 +187,7 @@ struct MergeTreeDataPart
}
/// Throws an exception if state of the part is not in affordable_states
void assertState(const std::initializer_list<State> & affordable_states) const
{
if (!checkState(affordable_states))
{
String states_str;
for (auto state : affordable_states)
states_str += stateToString(state) + " ";
throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str);
}
}
void assertState(const std::initializer_list<State> & affordable_states) const;
/// In comparison with lambdas, it is move assignable and could has several overloaded operator()
struct StatesFilter
......@@ -324,4 +314,7 @@ private:
void checkConsistency(bool require_part_metadata);
};
using MergeTreeDataPartState = MergeTreeDataPart::State;
}
......@@ -68,7 +68,7 @@ StorageMergeTree::StorageMergeTree(
}
else
{
data.clearOldParts();
data.clearOldPartsFromFilesystem();
}
/// Temporary directories contain incomplete results of merges (after forced restart)
......@@ -188,7 +188,7 @@ void StorageMergeTree::alter(
if (primary_key_is_modified && supportsSampling())
throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS);
MergeTreeData::DataParts parts = data.getAllDataParts();
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
for (const MergeTreeData::DataPartPtr & part : parts)
{
if (auto transaction = data.alterDataPart(part, columns_for_parts, new_primary_key_ast, false))
......@@ -291,7 +291,7 @@ bool StorageMergeTree::merge(
/// Clear old parts. It does not matter to do it more frequently than each second.
if (auto lock = time_after_previous_cleanup.lockTestAndRestartAfter(1))
{
data.clearOldParts();
data.clearOldPartsFromFilesystem();
data.clearOldTemporaryDirectories();
}
......
......@@ -93,6 +93,7 @@ namespace ErrorCodes
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
extern const int TOO_MUCH_FETCHES;
extern const int BAD_DATA_PART_NAME;
extern const int PART_IS_TEMPORARILY_LOCKED;
}
......@@ -766,7 +767,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
/// Parts in ZK.
NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end());
MergeTreeData::DataParts parts = data.getAllDataParts();
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
/// Local parts that are not in ZK.
MergeTreeData::DataParts unexpected_parts;
......@@ -1145,7 +1146,21 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
if (!do_fetch)
{
merger.renameMergedTemporaryPart(part, parts, &transaction);
getZooKeeper()->multi(ops); /// After long merge, get fresh ZK handle, because previous session may be expired.
/// Do not commit if the part is obsolete
if (!transaction.isEmpty())
{
getZooKeeper()->multi(ops); /// After long merge, get fresh ZK handle, because previous session may be expired.
transaction.commit();
}
/** Removing old chunks from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts.
*/
/** With `ZCONNECTIONLOSS` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
* This is not a problem, because in this case the merge will remain in the queue, and we will try again.
*/
merge_selecting_handle->schedule();
if (auto part_log = context.getPartLog(database_name, table_name))
{
......@@ -1178,15 +1193,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
}
}
/** Removing old chunks from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts.
*/
/** With `ZCONNECTIONLOSS` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
* This is not a problem, because in this case the merge will remain in the queue, and we will try again.
*/
transaction.commit();
merge_selecting_handle->schedule();
ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
}
}
......@@ -1409,8 +1415,9 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr
/// It's important that no old parts remain (after the merge), because otherwise,
/// after adding a new replica, this new replica downloads them, but does not delete them.
/// And, if you do not, the parts will come to life after the server is restarted.
/// Therefore, we use getAllDataParts.
auto parts = data.getAllDataParts();
/// Therefore, we use all data parts.
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
for (const auto & part : parts)
{
if (!entry_part_info.contains(part->info))
......@@ -1577,6 +1584,11 @@ bool StorageReplicatedMergeTree::queueTask()
/// Interrupted merge or downloading a part is not an error.
LOG_INFO(log, e.message());
}
else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
{
/// Part cannot be added temporarily
LOG_INFO(log, e.displayText());
}
else
tryLogCurrentException(__PRETTY_FUNCTION__);
......@@ -2161,6 +2173,13 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
MergeTreeData::Transaction transaction;
auto removed_parts = data.renameTempPartAndReplace(part, nullptr, &transaction);
/// Do not commit if the part is obsolete
if (!transaction.isEmpty())
{
getZooKeeper()->multi(ops);
transaction.commit();
}
if (auto part_log = context.getPartLog(database_name, table_name))
{
PartLogElement elem;
......@@ -2192,10 +2211,6 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
}
}
getZooKeeper()->multi(ops);
transaction.commit();
/** If a quorum is tracked for this part, you must update it.
* If you do not have time, in case of losing the session, when you restart the server - see the `ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart` method.
*/
......
......@@ -39,7 +39,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_)
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()}
}
{
}
......@@ -53,9 +53,12 @@ BlockInputStreams StorageSystemParts::read(
const size_t max_block_size,
const unsigned num_streams)
{
check(column_names);
//check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
auto it_state_column = std::find(column_names.begin(), column_names.end(), "_state");
bool has_state_column = it_state_column != column_names.end();
/// Will apply WHERE to subset of columns and then add more columns.
/// This is kind of complicated, but we use WHERE to do less work.
......@@ -142,6 +145,8 @@ BlockInputStreams StorageSystemParts::read(
/// Finally, create the result.
Block block = getSampleBlock();
if (has_state_column)
block.insert(ColumnWithTypeAndName(std::make_shared<DataTypeString>(), "_state"));
for (size_t i = 0; i < filtered_database_column->size();)
{
......@@ -198,10 +203,18 @@ BlockInputStreams StorageSystemParts::read(
using State = MergeTreeDataPart::State;
MergeTreeData::DataPartStateVector all_parts_state;
MergeTreeData::DataPartsVector all_parts;
if (need[0])
all_parts = data->getDataPartsVector({State::Committed, State::Outdated}, all_parts_state);
{
/// If has_state_column is requested, return all states
if (!has_state_column)
all_parts = data->getDataPartsVector({State::Committed, State::Outdated}, &all_parts_state);
else
all_parts = data->getAllDataPartsVector(&all_parts_state);
}
else
all_parts = data->getDataPartsVector({State::Committed}, all_parts_state);
all_parts = data->getDataPartsVector({State::Committed}, &all_parts_state);
/// Finally, we'll go through the list of parts.
for (size_t part_number = 0; part_number < all_parts.size(); ++part_number)
......@@ -248,11 +261,30 @@ BlockInputStreams StorageSystemParts::read(
block.getByPosition(i++).column->insert(database);
block.getByPosition(i++).column->insert(table);
block.getByPosition(i++).column->insert(engine);
if (has_state_column)
block.getByPosition(i++).column->insert(part->stateString());
}
}
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(block));
}
NameAndTypePair StorageSystemParts::getColumn(const String & column_name) const
{
if (column_name == "_state")
return NameAndTypePair("_state", std::make_shared<DataTypeString>());
return ITableDeclaration::getColumn(column_name);
}
bool StorageSystemParts::hasColumn(const String & column_name) const
{
if (column_name == "_state")
return true;
return ITableDeclaration::hasColumn(column_name);
}
}
......@@ -20,6 +20,10 @@ public:
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
NameAndTypePair getColumn(const String & column_name) const override;
bool hasColumn(const String & column_name) const override;
BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
......
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#include <gtest/gtest.h>
#pragma GCC diagnostic pop
#include <common/RangeFiltered.h>
#include <vector>
#include <set>
TEST(RangeFiltered, simple)
{
std::vector<int> v;
for (int i = 0; i < 10; ++i)
v.push_back(i);
auto v30 = createRangeFiltered([] (int i) { return i % 3 == 0; }, v);
auto v31 = createRangeFiltered([] (int i) { return i % 3 != 0; }, v);
for (const int & i : v30)
ASSERT_EQ(i % 3, 0);
for (const int & i : v31)
ASSERT_NE(i % 3, 0);
{
auto it = v30.begin();
ASSERT_EQ(*it, 0);
auto it2 = std::next(it);
ASSERT_EQ(*it2, 3);
it = std::next(it2);
ASSERT_EQ(*it, 6);
}
{
auto it = std::next(v30.begin());
ASSERT_EQ(*it, 3);
*it = 2; /// it becomes invalid
ASSERT_EQ(*(++it), 6); /// but iteration is sucessfull
*v30.begin() = 1;
ASSERT_EQ(*v30.begin(), 6);
}
}
......@@ -26,6 +26,7 @@ def started_cluster():
pass
cluster.shutdown()
def test_random_inserts(started_cluster):
# Duration of the test, reduce it if don't want to wait
DURATION_SECONDS = 10# * 60
......@@ -55,7 +56,9 @@ def test_random_inserts(started_cluster):
inserter.get_answer()
answer="{}\t{}\t{}\t{}\n".format(num_timestamps, num_timestamps, min_timestamp, max_timestamp)
for node in nodes:
assert TSV(node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple")) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1")
res = node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple")
assert TSV(res) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1")
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
......@@ -4,6 +4,7 @@
[[ -n "$1" ]] && host="$1" || host="localhost"
[[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 10 ))
[[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 10 ))
[[ -n "$4" ]] && iters_per_timestamp="$4" || iters_per_timestamp=1
timestamps=`seq $min_timestamp $max_timestamp`
......@@ -40,6 +41,6 @@ for i in $timestamps; do
cur_timestamp=$(date +%s)
done
#echo $i >> $host".txt"
reliable_insert "$i"
done
\ No newline at end of file
done
sleep 1
#pragma once
#include <type_traits>
/// Similar to boost::filtered_range but a little bit easier and allows to convert ordinary iterators to filtered
template <typename F, typename C>
struct RangeFiltered
{
/// Template parameter C may be const. Then const_iterator is used.
using RawIterator = decltype(std::declval<C>().begin());
class Iterator;
/// Will iterate over elements for which filter(*it) == true
template <typename F_, typename C_> /// Another template for universal references to work.
RangeFiltered(F_ && filter, C_ && container)
: filter(std::move(filter)), container(container) {}
Iterator begin() const
{
return Iterator{*this, std::begin(container)};
}
Iterator end() const
{
return Iterator{*this, std::end(container)};
}
/// Convert ordinary iterator to filtered one
/// Real position will be in range [ordinary_iterator; end()], so it is suitable to use with lower[upper]_bound()
inline Iterator convert(RawIterator ordinary_iterator) const
{
return Iterator{*this, ordinary_iterator};
}
/// It is similar to boost::filtered_iterator, but has additional features:
/// it doesn't store end() iterator
/// it doesn't store predicate, so it allows to implement operator=()
/// it guarantees that operator++() works properly in case of filter(*it) == false
class Iterator
{
public:
using Range = RangeFiltered<F, C>;
typedef Iterator self_type;
typedef typename std::iterator_traits<RawIterator>::value_type value_type;
typedef typename std::iterator_traits<RawIterator>::reference reference;
typedef const value_type & const_reference;
typedef typename std::iterator_traits<RawIterator>::pointer pointer;
typedef const value_type * const_pointer;
typedef typename std::iterator_traits<RawIterator>::difference_type difference_type;
typedef std::bidirectional_iterator_tag iterator_category;
Iterator(const Range & range_, RawIterator iter_)
: range(&range_), iter(iter_)
{
for (; iter != std::end(range->container) && !range->filter(*iter); ++iter);
}
Iterator(const Iterator & rhs) = default;
Iterator(Iterator && rhs) = default;
Iterator operator++()
{
++iter;
for (; iter != std::end(range->container) && !range->filter(*iter); ++iter);
return *this;
}
Iterator operator--()
{
--iter;
for (; !range->filter(*iter); --iter); /// Don't check std::begin() bound
return *this;
}
pointer operator->()
{
return iter.operator->();
}
const_pointer operator->() const
{
return iter.operator->();
}
reference operator*()
{
return *iter;
}
const_reference operator*() const
{
return *iter;
}
bool operator==(const self_type & rhs) const
{
return iter == rhs.iter;
}
bool operator!=(const self_type & rhs) const
{
return iter != rhs.iter;
}
self_type & operator=(const self_type & rhs) = default;
self_type & operator=(self_type && rhs) = default;
~Iterator() = default;
private:
const Range * range = nullptr;
RawIterator iter;
};
protected:
F filter;
C & container;
};
template <typename F, typename C>
inline RangeFiltered<std::decay_t<F>, std::remove_reference_t<C>> createRangeFiltered(F && filter, C && container)
{
return {std::forward<F>(filter), std::forward<C>(container)};
};
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册