This commit is contained in:
Nikita Vasilev 2019-02-06 10:49:18 +03:00
parent 9eaabcbf3f
commit ef154a4bb2
8 changed files with 52 additions and 39 deletions

View File

@ -26,7 +26,7 @@ void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creat
ErrorCodes::LOGICAL_ERROR);
}
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(
std::unique_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const

View File

@ -18,46 +18,51 @@ namespace DB
{
class MergeTreeData;
class MergeTreeIndex;
class IMergeTreeIndex;
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<MergeTreeIndex>;
using MergeTreeIndexPtr = std::shared_ptr<const IMergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<IMergeTreeIndex>;
struct MergeTreeIndexGranule
/// Stores some info about a single block of data.
struct IMergeTreeIndexGranule
{
virtual ~MergeTreeIndexGranule() = default;
virtual ~IMergeTreeIndexGranule() = default;
virtual void serializeBinary(WriteBuffer & ostr) const = 0;
virtual void deserializeBinary(ReadBuffer & istr) = 0;
virtual bool empty() const = 0;
/// Updates the stored info using rows of the specified block.
/// Reads no more than `limit` rows.
/// After finishing updating `pos` will store the position of the first row which was not read.
virtual void update(const Block & block, size_t * pos, size_t limit) = 0;
};
using MergeTreeIndexGranulePtr = std::shared_ptr<MergeTreeIndexGranule>;
using MergeTreeIndexGranulePtr = std::shared_ptr<IMergeTreeIndexGranule>;
using MergeTreeIndexGranules = std::vector<MergeTreeIndexGranulePtr>;
/// Condition on the index.
class IndexCondition
class IIndexCondition
{
public:
virtual ~IndexCondition() = default;
virtual ~IIndexCondition() = default;
/// Checks if this index is useful for query.
virtual bool alwaysUnknownOrTrue() const = 0;
virtual bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const = 0;
};
using IndexConditionPtr = std::shared_ptr<IndexCondition>;
using IndexConditionPtr = std::shared_ptr<IIndexCondition>;
/// Structure for storing basic index info like columns, expression, arguments, ...
class MergeTreeIndex
class IMergeTreeIndex
{
public:
MergeTreeIndex(
IMergeTreeIndex(
String name,
ExpressionActionsPtr expr,
const Names & columns,
@ -71,7 +76,7 @@ public:
, header(header)
, granularity(granularity) {}
virtual ~MergeTreeIndex() = default;
virtual ~IMergeTreeIndex() = default;
/// gets filename without extension
String getFileName() const { return INDEX_FILE_PREFIX + name; }
@ -89,7 +94,6 @@ public:
size_t granularity;
};
using MergeTreeIndices = std::vector<MutableMergeTreeIndexPtr>;
@ -99,12 +103,12 @@ class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
public:
using Creator = std::function<
std::unique_ptr<MergeTreeIndex>(
std::unique_ptr<IMergeTreeIndex>(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)>;
std::unique_ptr<MergeTreeIndex> get(
std::unique_ptr<IMergeTreeIndex> get(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const;

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
MergeTreeMinMaxGranule::MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index)
: MergeTreeIndexGranule(), index(index), parallelogram()
: IMergeTreeIndexGranule(), index(index), parallelogram()
{
}
@ -54,6 +54,11 @@ void MergeTreeMinMaxGranule::deserializeBinary(ReadBuffer & istr)
void MergeTreeMinMaxGranule::update(const Block & block, size_t * pos, size_t limit)
{
if (*pos >= block.rows())
throw Exception(
"The provided position is not less than the number of block rows. Position: "
+ toString(*pos) + ", Block rows: " + toString(block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
size_t rows_read = std::min(limit, block.rows() - *pos);
for (size_t i = 0; i < index.columns.size(); ++i)
@ -82,7 +87,7 @@ MinMaxCondition::MinMaxCondition(
const SelectQueryInfo &query,
const Context &context,
const MergeTreeMinMaxIndex &index)
: IndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}
: IIndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}
bool MinMaxCondition::alwaysUnknownOrTrue() const
{
@ -95,7 +100,7 @@ bool MinMaxCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) c
= std::dynamic_pointer_cast<MergeTreeMinMaxGranule>(idx_granule);
if (!granule)
throw Exception(
"Minmax index condition got wrong granule", ErrorCodes::LOGICAL_ERROR);
"Minmax index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
return condition.mayBeTrueInParallelogram(granule->parallelogram, index.data_types);
}
@ -109,11 +114,11 @@ MergeTreeIndexGranulePtr MergeTreeMinMaxIndex::createIndexGranule() const
IndexConditionPtr MergeTreeMinMaxIndex::createIndexCondition(
const SelectQueryInfo & query, const Context & context) const
{
return std::make_shared<MinMaxCondition>(query, context, *this);
return std::make_shared<MinMaxCondition>(query, context, *this);
};
std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
std::unique_ptr<IMergeTreeIndex> MergeTreeMinMaxIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)

View File

@ -13,7 +13,7 @@ namespace DB
class MergeTreeMinMaxIndex;
struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
struct MergeTreeMinMaxGranule : public IMergeTreeIndexGranule
{
explicit MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index);
@ -30,7 +30,7 @@ struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
};
class MinMaxCondition : public IndexCondition
class MinMaxCondition : public IIndexCondition
{
public:
MinMaxCondition(
@ -49,7 +49,7 @@ private:
};
class MergeTreeMinMaxIndex : public MergeTreeIndex
class MergeTreeMinMaxIndex : public IMergeTreeIndex
{
public:
MergeTreeMinMaxIndex(
@ -59,7 +59,7 @@ public:
const DataTypes & data_types_,
const Block & header_,
size_t granularity_)
: MergeTreeIndex(name_, expr_, columns_, data_types_, header_, granularity_) {}
: IMergeTreeIndex(name_, expr_, columns_, data_types_, header_, granularity_) {}
~MergeTreeMinMaxIndex() override = default;
@ -70,7 +70,7 @@ public:
};
std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
std::unique_ptr<IMergeTreeIndex> MergeTreeMinMaxIndexCreator(
const NamesAndTypesList & columns, std::shared_ptr<ASTIndexDeclaration> node, const Context & context);
}

View File

@ -21,7 +21,7 @@ const Field UNKNOWN_FIELD(3);
MergeTreeUniqueGranule::MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index)
: MergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
: IMergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
{
set->setHeader(index.header);
}
@ -79,6 +79,11 @@ void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
void MergeTreeUniqueGranule::update(const Block & new_block, size_t * pos, size_t limit)
{
if (*pos >= new_block.rows())
throw Exception(
"The provided position is not less than the number of block rows. Position: "
+ toString(*pos) + ", Block rows: " + toString(new_block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
size_t rows_read = std::min(limit, new_block.rows() - *pos);
if (index.max_rows && size() > index.max_rows)
@ -116,7 +121,7 @@ UniqueCondition::UniqueCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex &index)
: IndexCondition(), index(index)
: IIndexCondition(), index(index)
{
for (size_t i = 0, size = index.columns.size(); i < size; ++i)
{
@ -163,7 +168,7 @@ bool UniqueCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) c
auto granule = std::dynamic_pointer_cast<MergeTreeUniqueGranule>(idx_granule);
if (!granule)
throw Exception(
"Unique index condition got wrong granule", ErrorCodes::LOGICAL_ERROR);
"Unique index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
if (useless)
return true;
@ -348,7 +353,7 @@ IndexConditionPtr MergeTreeUniqueIndex::createIndexCondition(
};
std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
std::unique_ptr<IMergeTreeIndex> MergeTreeUniqueIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)

View File

@ -14,7 +14,7 @@ namespace DB
class MergeTreeUniqueIndex;
struct MergeTreeUniqueGranule : public MergeTreeIndexGranule
struct MergeTreeUniqueGranule : public IMergeTreeIndexGranule
{
explicit MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index);
@ -34,7 +34,7 @@ struct MergeTreeUniqueGranule : public MergeTreeIndexGranule
};
class UniqueCondition : public IndexCondition
class UniqueCondition : public IIndexCondition
{
public:
UniqueCondition(
@ -63,7 +63,7 @@ private:
};
class MergeTreeUniqueIndex : public MergeTreeIndex
class MergeTreeUniqueIndex : public IMergeTreeIndex
{
public:
MergeTreeUniqueIndex(
@ -74,7 +74,7 @@ public:
const Block & header_,
size_t granularity_,
size_t max_rows_)
: MergeTreeIndex(std::move(name_), std::move(expr_), columns_, data_types_, header_, granularity_), max_rows(max_rows_) {}
: IMergeTreeIndex(std::move(name_), std::move(expr_), columns_, data_types_, header_, granularity_), max_rows(max_rows_) {}
~MergeTreeUniqueIndex() override = default;
@ -86,7 +86,7 @@ public:
size_t max_rows = 0;
};
std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
std::unique_ptr<IMergeTreeIndex> MergeTreeUniqueIndexCreator(
const NamesAndTypesList & columns, std::shared_ptr<ASTIndexDeclaration> node, const Context & context);
}

View File

@ -257,7 +257,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
{
stream.assertMark();
}
catch (Exception &e)
catch (Exception & e)
{
e.addMessage("Cannot read mark " + toString(mark_num)
+ " in file " + stream.mrk_file_path
@ -268,7 +268,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
{
index->createIndexGranule()->deserializeBinary(stream.uncompressed_hashing_buf);
}
catch (Exception &e)
catch (Exception & e)
{
e.addMessage("Cannot read granule " + toString(mark_num)
+ " in file " + stream.bin_file_path

View File

@ -113,7 +113,6 @@ namespace ErrorCodes
extern const int KEEPER_EXCEPTION;
extern const int ALL_REPLICAS_LOST;
extern const int REPLICA_STATUS_CHANGED;
extern const int INCORRECT_QUERY;
}
namespace ActionLocks