mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +00:00
Merge pull request #4 from nikvas0/nikvas0/fix-double-index
Nikvas0/fix double index
This commit is contained in:
commit
15ee9e170b
@ -106,7 +106,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
const ASTFunction & engine = *storage.engine;
|
||||
/// Currently, there are no database engines, that support any arguments.
|
||||
if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key
|
||||
|| storage.order_by || storage.sample_by || !storage.indexes.empty() || storage.settings)
|
||||
|| storage.order_by || storage.sample_by || (storage.indexes && !storage.indexes->children.empty()) || storage.settings)
|
||||
{
|
||||
std::stringstream ostr;
|
||||
formatAST(storage, ostr, false, false);
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
IAST * primary_key = nullptr;
|
||||
IAST * order_by = nullptr;
|
||||
IAST * sample_by = nullptr;
|
||||
std::vector<IAST *> indexes;
|
||||
ASTExpressionList * indexes = nullptr;
|
||||
ASTSetQuery * settings = nullptr;
|
||||
|
||||
String getID(char) const override { return "Storage definition"; }
|
||||
@ -39,11 +39,8 @@ public:
|
||||
res->set(res->order_by, order_by->clone());
|
||||
if (sample_by)
|
||||
res->set(res->sample_by, sample_by->clone());
|
||||
|
||||
for (const auto& index : indexes) {
|
||||
res->indexes.emplace_back(nullptr);
|
||||
res->set(res->indexes.back(), index->clone());
|
||||
}
|
||||
if (indexes)
|
||||
res->set(res->indexes, indexes->clone());
|
||||
|
||||
if (settings)
|
||||
res->set(res->settings, settings->clone());
|
||||
@ -78,9 +75,10 @@ public:
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
|
||||
sample_by->formatImpl(s, state, frame);
|
||||
}
|
||||
for (const auto& index : indexes) {
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "INDEX " << (s.hilite ? hilite_none : "");
|
||||
index->formatImpl(s, state, frame);
|
||||
if (indexes)
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "INDEXES " << (s.hilite ? hilite_none : "");
|
||||
indexes->formatImpl(s, state, frame);
|
||||
}
|
||||
if (settings)
|
||||
{
|
||||
|
@ -13,7 +13,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Index name(expr) TYPE typename(args) in create query
|
||||
/** name BY expr TYPE typename(args) GRANULARITY int in create query
|
||||
*/
|
||||
class ASTIndexDeclaration : public IAST
|
||||
{
|
||||
@ -27,8 +27,10 @@ public:
|
||||
String getID(char) const override { return "Index"; }
|
||||
|
||||
ASTPtr clone() const override {
|
||||
auto res = std::make_shared<ASTIndexDeclaration>(*this);
|
||||
res->children.clear();
|
||||
auto res = std::make_shared<ASTIndexDeclaration>();
|
||||
|
||||
res->name = name;
|
||||
res->granularity = granularity;
|
||||
|
||||
if (expr)
|
||||
res->set(res->expr, expr->clone());
|
||||
|
@ -138,6 +138,12 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserIndexDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
return ParserList(std::make_unique<ParserIndexDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||
.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
|
||||
bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
@ -147,20 +153,20 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword s_primary_key("PRIMARY KEY");
|
||||
ParserKeyword s_order_by("ORDER BY");
|
||||
ParserKeyword s_sample_by("SAMPLE BY");
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_indexes("INDEXES");
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
|
||||
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
|
||||
ParserExpression expression_p;
|
||||
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
|
||||
ParserIndexDeclaration index_p;
|
||||
ParserIndexDeclarationList indexes_p;
|
||||
|
||||
ASTPtr engine;
|
||||
ASTPtr partition_by;
|
||||
ASTPtr primary_key;
|
||||
ASTPtr order_by;
|
||||
ASTPtr sample_by;
|
||||
ASTs indexes;
|
||||
ASTPtr indexes;
|
||||
ASTPtr settings;
|
||||
|
||||
if (!s_engine.ignore(pos, expected))
|
||||
@ -205,9 +211,8 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_index.ignore(pos, expected)) {
|
||||
indexes.emplace_back(nullptr);
|
||||
if (index_p.parse(pos, indexes.back(), expected))
|
||||
if (s_indexes.ignore(pos, expected)) {
|
||||
if (indexes_p.parse(pos, indexes, expected))
|
||||
continue;
|
||||
else
|
||||
return false;
|
||||
@ -228,11 +233,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
storage->set(storage->primary_key, primary_key);
|
||||
storage->set(storage->order_by, order_by);
|
||||
storage->set(storage->sample_by, sample_by);
|
||||
|
||||
for (const auto& index : indexes) {
|
||||
storage->indexes.emplace_back(nullptr);
|
||||
storage->set(storage->indexes.back(), index);
|
||||
}
|
||||
storage->set(storage->indexes, indexes);
|
||||
|
||||
storage->set(storage->settings, settings);
|
||||
|
||||
|
@ -220,7 +220,7 @@ protected:
|
||||
|
||||
|
||||
/**
|
||||
* INDEX name BY expr TYPE typename(arg1, arg2, ...) GRANULARITY value
|
||||
* name BY expr TYPE typename(arg1, arg2, ...) GRANULARITY value
|
||||
*/
|
||||
class ParserIndexDeclaration : public IParserBase
|
||||
{
|
||||
@ -228,14 +228,21 @@ public:
|
||||
ParserIndexDeclaration() {}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "INDEX"; }
|
||||
const char * getName() const override { return "index declaration"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserIndexDeclarationList : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "index declaration list"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* ENGINE = name [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr]
|
||||
* [INDEX name BY expr TYPE type(args) ... GRANULARITY value] [SETTINGS name = value, ...]
|
||||
* [INDEXES name BY expr TYPE type(args) GRANULARITY value, ...] [SETTINGS name = value, ...]
|
||||
*/
|
||||
class ParserStorage : public IParserBase
|
||||
{
|
||||
|
@ -47,6 +47,7 @@
|
||||
|
||||
#include <algorithm>
|
||||
#include <iomanip>
|
||||
#include <set>
|
||||
#include <thread>
|
||||
#include <typeinfo>
|
||||
#include <typeindex>
|
||||
@ -94,7 +95,7 @@ MergeTreeData::MergeTreeData(
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const ASTs & indexes_ast_,
|
||||
const ASTPtr & indexes_ast_,
|
||||
const MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool require_part_metadata_,
|
||||
@ -349,15 +350,32 @@ void MergeTreeData::setPrimaryKeyAndColumns(
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::setSkipIndexes(const ASTs & indexes_asts, bool only_check)
|
||||
void MergeTreeData::setSkipIndexes(const ASTPtr & indexes_asts, bool only_check)
|
||||
{
|
||||
if (!only_check) {
|
||||
for (const auto &index_ast : indexes_asts) {
|
||||
if (!indexes_asts)
|
||||
{
|
||||
return;
|
||||
}
|
||||
if (!only_check)
|
||||
{
|
||||
indexes.clear();
|
||||
std::set<String> names;
|
||||
auto index_list = std::dynamic_pointer_cast<ASTExpressionList>(indexes_asts);
|
||||
|
||||
for (const auto &index_ast : index_list->children)
|
||||
{
|
||||
indexes.push_back(
|
||||
std::move(MergeTreeIndexFactory::instance().get(
|
||||
*this,
|
||||
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast),
|
||||
global_context)));
|
||||
if (names.find(indexes.back()->name) != names.end())
|
||||
{
|
||||
throw Exception(
|
||||
"Index with name `" + indexes.back()->name + "` already exsists",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
names.insert(indexes.back()->name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -309,7 +309,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 ASTPtr & indexes_ast_,
|
||||
const MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool require_part_metadata_,
|
||||
@ -727,7 +727,7 @@ private:
|
||||
|
||||
void setPrimaryKeyAndColumns(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast, const ColumnsDescription & new_columns, bool only_check = false);
|
||||
|
||||
void setSkipIndexes(const ASTs & indexes_asts, bool only_check = false);
|
||||
void setSkipIndexes(const ASTPtr & indexes_asts, bool only_check = false);
|
||||
|
||||
void initPartitionKey();
|
||||
|
||||
|
@ -553,23 +553,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
Names all_column_names = data.getColumns().getNamesOfPhysical();
|
||||
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
|
||||
|
||||
LOG_DEBUG(log, "Before extract");
|
||||
NamesAndTypesList gathering_columns, merging_columns;
|
||||
Names gathering_column_names, merging_column_names;
|
||||
extractMergingAndGatheringColumns(
|
||||
all_columns, data.sorting_key_expr, data.indexes,
|
||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
|
||||
LOG_DEBUG(log, "After extract");
|
||||
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
|
||||
data, future_part.name, future_part.part_info);
|
||||
new_data_part->partition.assign(future_part.getPartition());
|
||||
new_data_part->relative_path = TMP_PREFIX + future_part.name;
|
||||
new_data_part->is_temp = true;
|
||||
|
||||
LOG_DEBUG(log, "New Part");
|
||||
|
||||
size_t sum_input_rows_upper_bound = merge_entry->total_size_marks * data.index_granularity;
|
||||
|
||||
MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate);
|
||||
|
@ -12,6 +12,12 @@
|
||||
|
||||
namespace DB {
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
}
|
||||
|
||||
|
||||
class MergeTreeTestIndex;
|
||||
|
||||
struct MergeTreeTestGranule : public MergeTreeIndexGranule {
|
||||
@ -24,6 +30,9 @@ struct MergeTreeTestGranule : public MergeTreeIndexGranule {
|
||||
|
||||
void deserializeBinary(ReadBuffer &istr) override {
|
||||
readIntBinary(emp, istr);
|
||||
if (emp != 10) {
|
||||
throw Exception("kek bad read", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
}
|
||||
//std::cerr << "TESTINDEX: read " << emp << "\n";
|
||||
}
|
||||
|
||||
|
@ -340,7 +340,7 @@ 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->indexes.empty() || args.storage_def->settings;
|
||||
|| args.storage_def->sample_by || (args.storage_def->indexes && !args.storage_def->indexes->children.empty()) || args.storage_def->settings;
|
||||
|
||||
String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree"));
|
||||
|
||||
@ -563,7 +563,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
ASTPtr order_by_ast;
|
||||
ASTPtr primary_key_ast;
|
||||
ASTPtr sample_by_ast;
|
||||
ASTs indexes_ast;
|
||||
ASTPtr indexes_ast;
|
||||
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
|
||||
|
||||
if (is_extended_storage_def)
|
||||
@ -584,8 +584,8 @@ 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());
|
||||
if (args.storage_def->indexes) {
|
||||
indexes_ast = args.storage_def->indexes->ptr();
|
||||
}
|
||||
|
||||
storage_settings.loadFromQuery(*args.storage_def);
|
||||
|
@ -95,7 +95,7 @@ StoragePtr StorageFactory::get(
|
||||
}
|
||||
|
||||
if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by
|
||||
|| storage_def->sample_by || !storage_def->indexes.empty())
|
||||
|| storage_def->sample_by || (storage_def->indexes && !storage_def->indexes->children.empty()))
|
||||
&& !endsWith(name, "MergeTree"))
|
||||
{
|
||||
throw Exception(
|
||||
|
@ -58,7 +58,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 ASTPtr & indexes_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag)
|
||||
|
@ -174,7 +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 ASTPtr & indexes_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag);
|
||||
|
@ -206,7 +206,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const ASTs & indexes_ast_,
|
||||
const ASTPtr & indexes_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag)
|
||||
@ -228,9 +228,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
{
|
||||
if (path_.empty())
|
||||
throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
if (!indexes_ast_.empty()) {
|
||||
throw Exception("check indexes support for ReplicatedMergeTree", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
|
||||
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
|
||||
zookeeper_path.resize(zookeeper_path.size() - 1);
|
||||
|
@ -552,7 +552,7 @@ protected:
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const ASTs & indexes_ast_,
|
||||
const ASTPtr & indexes_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag);
|
||||
|
Loading…
Reference in New Issue
Block a user