added index interface and factory

This commit is contained in:
Nikita Vasilev 2018-12-26 15:19:24 +03:00
parent 36083e1036
commit 04a8ea85af
7 changed files with 105 additions and 4 deletions

View File

@ -94,6 +94,7 @@ MergeTreeData::MergeTreeData(
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_,
const ASTs & indexes_ast_,
const MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool require_part_metadata_,

View File

@ -306,6 +306,7 @@ public:
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTs & indexes_ast_,
const MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool require_part_metadata_,

View File

@ -0,0 +1,31 @@
#include <Storages/MergeTree/MergeTreeIndexes.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INCORRECT_QUERY;
}
void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creator) {
if (!indexes.emplace(name, std::move(creator)).second)
throw Exception("MergeTreeIndexFactory: the Index creator name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
IMergeTreeIndex MergeTreeIndexFactory::get(const ASTIndexDeclaration & node) const {
if (!node.type)
throw Exception(
"for INDEX TYPE is required",
ErrorCodes::INCORRECT_QUERY);
auto it = indexes.find(node.type->name);
if (it == indexes.end())
throw Exception(
"Unknown Index type '" + node.type->name + "'",
ErrorCodes::INCORRECT_QUERY);
return it->second(node);
}
}

View File

@ -0,0 +1,60 @@
#pragma once
#include <string>
#include <unordered_map>
#include <vector>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Parsers/ASTIndexDeclaration.h>
namespace DB
{
class IMergeTreeIndex;
using MergeTreeIndexes = std::vector<IMergeTreeIndex>;
/// Interface for secondary MergeTree indexes
class IMergeTreeIndex
{
public:
virtual void load(const MergeTreeData & storage, const String & part_path) = 0;
virtual void store(const MergeTreeData & storage, const String & part_path,
MergeTreeDataPartChecksums & checksums) const = 0;
virtual void update(const Block & block, const Names & column_names) = 0;
virtual void merge(const IMergeTreeIndex & other) = 0;
virtual bool alwaysUnknownOrTrue() const = 0;
virtual bool maybeTrue() const = 0;
String name;
ExpressionActionsPtr expr;
Block header;
};
class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
{
friend class ext::singleton<MergeTreeIndexFactory>;
public:
using Creator = std::function<IMergeTreeIndex(const ASTIndexDeclaration & node)>;
protected:
MergeTreeIndexFactory() {};
IMergeTreeIndex get(const ASTIndexDeclaration & node) const;
void registerIndex(const std::string & name, Creator creator);
const auto & getAllIndexes() const {
return indexes;
}
private:
using Indexes = std::unordered_map<std::string, Creator>;
Indexes indexes;
};
}

View File

@ -336,7 +336,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
*/
bool is_extended_storage_def =
args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by || args.storage_def->sample_by || args.storage_def->settings;
args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by
|| args.storage_def->sample_by || !args.storage_def->indexes.empty() || args.storage_def->settings;
String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree"));
@ -559,6 +560,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
ASTPtr order_by_ast;
ASTPtr primary_key_ast;
ASTPtr sample_by_ast;
ASTs indexes_ast;
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
if (is_extended_storage_def)
@ -579,6 +581,10 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.storage_def->sample_by)
sample_by_ast = args.storage_def->sample_by->ptr();
for (auto& index : args.storage_def->indexes) {
indexes_ast.push_back(index->ptr());
}
storage_settings.loadFromQuery(*args.storage_def);
}
else
@ -615,13 +621,13 @@ static StoragePtr create(const StorageFactory::Arguments & args)
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
args.columns,
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
sample_by_ast, merging_params, storage_settings,
sample_by_ast, indexes_ast, merging_params, storage_settings,
args.has_force_restore_data_flag);
else
return StorageMergeTree::create(
args.data_path, args.database_name, args.table_name, args.columns, args.attach,
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
sample_by_ast, merging_params, storage_settings,
sample_by_ast, indexes_ast, merging_params, storage_settings,
args.has_force_restore_data_flag);
}

View File

@ -51,6 +51,7 @@ StorageMergeTree::StorageMergeTree(
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTs & indexes_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag)
@ -59,7 +60,7 @@ StorageMergeTree::StorageMergeTree(
data(database_name, table_name,
full_path, columns_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, merging_params_,
sample_by_ast_, indexes_ast_, merging_params_,
settings_, false, attach),
reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()),
log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)"))

View File

@ -174,6 +174,7 @@ protected:
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTs & indexes_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag);