ClickHouse/dbms/Storages/MergeTree/MergeTreeIndices.h

180 lines
5.1 KiB
C++
Raw Normal View History

2018-12-26 12:19:24 +00:00
#pragma once
#include <string>
#include <unordered_map>
#include <vector>
2018-12-26 17:34:44 +00:00
#include <memory>
#include <Core/Block.h>
2019-01-02 14:24:26 +00:00
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
2018-12-29 11:12:41 +00:00
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MarkRange.h>
2018-12-26 17:34:44 +00:00
#include <Interpreters/ExpressionActions.h>
2018-12-26 12:19:24 +00:00
#include <Parsers/ASTIndexDeclaration.h>
#include <DataTypes/DataTypeLowCardinality.h>
2018-12-26 12:19:24 +00:00
2019-01-05 09:26:02 +00:00
constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
2018-12-26 12:19:24 +00:00
namespace DB
{
2019-01-07 18:53:51 +00:00
class MergeTreeData;
2019-02-06 07:49:18 +00:00
class IMergeTreeIndex;
2018-12-28 17:53:41 +00:00
2019-02-06 07:49:18 +00:00
using MergeTreeIndexPtr = std::shared_ptr<const IMergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<IMergeTreeIndex>;
2018-12-29 11:12:41 +00:00
2018-12-29 12:04:00 +00:00
2019-02-06 07:49:18 +00:00
/// Stores some info about a single block of data.
struct IMergeTreeIndexGranule
2019-01-03 16:47:42 +00:00
{
2019-02-06 07:49:18 +00:00
virtual ~IMergeTreeIndexGranule() = default;
2019-01-03 16:47:42 +00:00
2019-01-04 14:33:38 +00:00
virtual void serializeBinary(WriteBuffer & ostr) const = 0;
2019-01-07 16:49:34 +00:00
virtual void deserializeBinary(ReadBuffer & istr) = 0;
2019-01-03 16:47:42 +00:00
2019-01-04 14:33:38 +00:00
virtual bool empty() const = 0;
2019-03-08 15:36:39 +00:00
};
using MergeTreeIndexGranulePtr = std::shared_ptr<IMergeTreeIndexGranule>;
using MergeTreeIndexGranules = std::vector<MergeTreeIndexGranulePtr>;
/// Aggregates info about a single block of data.
struct IMergeTreeIndexAggregator
{
virtual ~IMergeTreeIndexAggregator() = default;
virtual bool empty() const = 0;
2019-03-08 19:52:21 +00:00
virtual MergeTreeIndexGranulePtr getGranuleAndReset() = 0;
2019-02-06 07:49:18 +00:00
/// 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;
2019-01-03 16:47:42 +00:00
};
2019-03-08 15:36:39 +00:00
using MergeTreeIndexAggregatorPtr = std::shared_ptr<IMergeTreeIndexAggregator>;
using MergeTreeIndexAggregators = std::vector<MergeTreeIndexAggregatorPtr>;
2019-01-03 16:47:42 +00:00
2019-02-06 07:49:18 +00:00
2019-01-07 12:51:14 +00:00
/// Condition on the index.
2019-06-19 15:30:48 +00:00
class IMergeTreeIndexCondition
2019-01-26 06:26:49 +00:00
{
2019-01-07 12:51:14 +00:00
public:
2019-06-19 15:30:48 +00:00
virtual ~IMergeTreeIndexCondition() = default;
2019-01-07 12:51:14 +00:00
/// Checks if this index is useful for query.
virtual bool alwaysUnknownOrTrue() const = 0;
2019-01-08 17:27:44 +00:00
virtual bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const = 0;
2019-01-06 15:22:04 +00:00
};
2019-06-19 15:30:48 +00:00
using MergeTreeIndexConditionPtr = std::shared_ptr<IMergeTreeIndexCondition>;
2019-01-07 12:51:14 +00:00
2019-01-06 15:22:04 +00:00
2018-12-29 12:04:00 +00:00
/// Structure for storing basic index info like columns, expression, arguments, ...
2019-02-06 07:49:18 +00:00
class IMergeTreeIndex
2018-12-26 12:19:24 +00:00
{
public:
2019-02-06 07:49:18 +00:00
IMergeTreeIndex(
2019-08-03 11:02:40 +00:00
String name_,
ExpressionActionsPtr expr_,
const Names & columns_,
const DataTypes & data_types_,
const Block & header_,
size_t granularity_)
: name(name_)
, expr(expr_)
, columns(columns_)
, data_types(data_types_)
, header(header_)
, granularity(granularity_) {}
2018-12-26 12:19:24 +00:00
2019-02-06 07:49:18 +00:00
virtual ~IMergeTreeIndex() = default;
2018-12-26 12:19:24 +00:00
2019-01-05 09:26:02 +00:00
/// gets filename without extension
2019-01-26 06:26:49 +00:00
String getFileName() const { return INDEX_FILE_PREFIX + name; }
2019-01-05 09:26:02 +00:00
2019-02-25 08:43:19 +00:00
/// Checks whether the column is in data skipping index.
virtual bool mayBenefitFromIndexForIn(const ASTPtr & node) const = 0;
2019-01-04 14:33:38 +00:00
virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0;
2019-03-08 19:52:21 +00:00
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0;
2019-01-04 14:33:38 +00:00
2019-06-19 15:30:48 +00:00
virtual MergeTreeIndexConditionPtr createIndexCondition(
2019-01-04 15:54:34 +00:00
const SelectQueryInfo & query_info, const Context & context) const = 0;
2019-08-28 18:23:20 +00:00
Names getColumnsRequiredForIndexCalc() const { return expr->getRequiredColumns(); }
/// Index name
2018-12-26 12:19:24 +00:00
String name;
2019-08-28 18:23:20 +00:00
2019-08-29 10:56:43 +00:00
/// Index expression (x * y)
/// with columns arguments
2018-12-26 12:19:24 +00:00
ExpressionActionsPtr expr;
2019-08-28 18:23:20 +00:00
2019-08-29 10:56:43 +00:00
/// Names of columns for index
2019-01-04 14:33:38 +00:00
Names columns;
2019-08-28 18:23:20 +00:00
/// Data types of columns
2019-01-04 14:33:38 +00:00
DataTypes data_types;
2019-08-28 18:23:20 +00:00
/// Block with columns and data_types
2019-01-27 18:02:15 +00:00
Block header;
2019-08-28 18:23:20 +00:00
/// Skip index granularity
2019-01-10 12:57:12 +00:00
size_t granularity;
2018-12-26 12:19:24 +00:00
};
2019-01-17 12:11:36 +00:00
using MergeTreeIndices = std::vector<MutableMergeTreeIndexPtr>;
2019-01-05 18:33:30 +00:00
class MergeTreeIndexFactory : private boost::noncopyable
2018-12-26 12:19:24 +00:00
{
public:
static MergeTreeIndexFactory & instance();
2019-01-07 18:53:51 +00:00
using Creator = std::function<
2019-02-06 07:49:18 +00:00
std::unique_ptr<IMergeTreeIndex>(
2019-01-30 10:18:59 +00:00
const NamesAndTypesList & columns,
2019-01-07 18:53:51 +00:00
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)>;
2019-02-06 07:49:18 +00:00
std::unique_ptr<IMergeTreeIndex> get(
2019-01-30 10:18:59 +00:00
const NamesAndTypesList & columns,
2019-01-09 17:05:52 +00:00
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const;
2018-12-26 12:19:24 +00:00
void registerIndex(const std::string & name, Creator creator);
2019-01-26 06:26:49 +00:00
const auto & getAllIndexes() const { return indexes; }
2018-12-26 12:19:24 +00:00
2018-12-26 17:34:44 +00:00
protected:
2019-02-06 09:05:05 +00:00
MergeTreeIndexFactory();
2018-12-26 17:34:44 +00:00
2018-12-26 12:19:24 +00:00
private:
using Indexes = std::unordered_map<std::string, Creator>;
Indexes indexes;
};
2019-12-15 06:34:43 +00:00
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
2019-01-26 06:26:49 +00:00
}