mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-05 05:52:05 +00:00
added index interface and factory
This commit is contained in:
parent
36083e1036
commit
04a8ea85af
@ -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_,
|
||||
|
@ -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_,
|
||||
|
31
dbms/src/Storages/MergeTree/MergeTreeIndexes.cpp
Normal file
31
dbms/src/Storages/MergeTree/MergeTreeIndexes.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
60
dbms/src/Storages/MergeTree/MergeTreeIndexes.h
Normal file
60
dbms/src/Storages/MergeTree/MergeTreeIndexes.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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)"))
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user