Merge pull request #4 from nikvas0/nikvas0/fix-double-index

Nikvas0/fix double index
This commit is contained in:
Nikita Vasilev 2019-01-08 14:11:50 +03:00 committed by GitHub
commit 15ee9e170b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 77 additions and 50 deletions

View File

@ -106,7 +106,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
const ASTFunction & engine = *storage.engine; const ASTFunction & engine = *storage.engine;
/// Currently, there are no database engines, that support any arguments. /// Currently, there are no database engines, that support any arguments.
if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key 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; std::stringstream ostr;
formatAST(storage, ostr, false, false); formatAST(storage, ostr, false, false);

View File

@ -19,7 +19,7 @@ public:
IAST * primary_key = nullptr; IAST * primary_key = nullptr;
IAST * order_by = nullptr; IAST * order_by = nullptr;
IAST * sample_by = nullptr; IAST * sample_by = nullptr;
std::vector<IAST *> indexes; ASTExpressionList * indexes = nullptr;
ASTSetQuery * settings = nullptr; ASTSetQuery * settings = nullptr;
String getID(char) const override { return "Storage definition"; } String getID(char) const override { return "Storage definition"; }
@ -39,11 +39,8 @@ public:
res->set(res->order_by, order_by->clone()); res->set(res->order_by, order_by->clone());
if (sample_by) if (sample_by)
res->set(res->sample_by, sample_by->clone()); res->set(res->sample_by, sample_by->clone());
if (indexes)
for (const auto& index : indexes) { res->set(res->indexes, indexes->clone());
res->indexes.emplace_back(nullptr);
res->set(res->indexes.back(), index->clone());
}
if (settings) if (settings)
res->set(res->settings, settings->clone()); 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 : ""); s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
sample_by->formatImpl(s, state, frame); sample_by->formatImpl(s, state, frame);
} }
for (const auto& index : indexes) { if (indexes)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "INDEX " << (s.hilite ? hilite_none : ""); {
index->formatImpl(s, state, frame); s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "INDEXES " << (s.hilite ? hilite_none : "");
indexes->formatImpl(s, state, frame);
} }
if (settings) if (settings)
{ {

View File

@ -13,7 +13,7 @@
namespace DB 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 class ASTIndexDeclaration : public IAST
{ {
@ -27,8 +27,10 @@ public:
String getID(char) const override { return "Index"; } String getID(char) const override { return "Index"; }
ASTPtr clone() const override { ASTPtr clone() const override {
auto res = std::make_shared<ASTIndexDeclaration>(*this); auto res = std::make_shared<ASTIndexDeclaration>();
res->children.clear();
res->name = name;
res->granularity = granularity;
if (expr) if (expr)
res->set(res->expr, expr->clone()); res->set(res->expr, expr->clone());

View File

@ -138,6 +138,12 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
return true; 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) 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_primary_key("PRIMARY KEY");
ParserKeyword s_order_by("ORDER BY"); ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_sample_by("SAMPLE BY"); ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_index("INDEX"); ParserKeyword s_indexes("INDEXES");
ParserKeyword s_settings("SETTINGS"); ParserKeyword s_settings("SETTINGS");
ParserIdentifierWithOptionalParameters ident_with_optional_params_p; ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
ParserExpression expression_p; ParserExpression expression_p;
ParserSetQuery settings_p(/* parse_only_internals_ = */ true); ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
ParserIndexDeclaration index_p; ParserIndexDeclarationList indexes_p;
ASTPtr engine; ASTPtr engine;
ASTPtr partition_by; ASTPtr partition_by;
ASTPtr primary_key; ASTPtr primary_key;
ASTPtr order_by; ASTPtr order_by;
ASTPtr sample_by; ASTPtr sample_by;
ASTs indexes; ASTPtr indexes;
ASTPtr settings; ASTPtr settings;
if (!s_engine.ignore(pos, expected)) if (!s_engine.ignore(pos, expected))
@ -205,9 +211,8 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false; return false;
} }
if (s_index.ignore(pos, expected)) { if (s_indexes.ignore(pos, expected)) {
indexes.emplace_back(nullptr); if (indexes_p.parse(pos, indexes, expected))
if (index_p.parse(pos, indexes.back(), expected))
continue; continue;
else else
return false; 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->primary_key, primary_key);
storage->set(storage->order_by, order_by); storage->set(storage->order_by, order_by);
storage->set(storage->sample_by, sample_by); storage->set(storage->sample_by, sample_by);
storage->set(storage->indexes, indexes);
for (const auto& index : indexes) {
storage->indexes.emplace_back(nullptr);
storage->set(storage->indexes.back(), index);
}
storage->set(storage->settings, settings); storage->set(storage->settings, settings);

View File

@ -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 class ParserIndexDeclaration : public IParserBase
{ {
@ -228,14 +228,21 @@ public:
ParserIndexDeclaration() {} ParserIndexDeclaration() {}
protected: 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; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
/** /**
* ENGINE = name [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] * 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 class ParserStorage : public IParserBase
{ {

View File

@ -47,6 +47,7 @@
#include <algorithm> #include <algorithm>
#include <iomanip> #include <iomanip>
#include <set>
#include <thread> #include <thread>
#include <typeinfo> #include <typeinfo>
#include <typeindex> #include <typeindex>
@ -94,7 +95,7 @@ MergeTreeData::MergeTreeData(
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, const ASTPtr & sample_by_ast_,
const ASTs & indexes_ast_, const ASTPtr & indexes_ast_,
const MergingParams & merging_params_, const MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool require_part_metadata_, 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) { if (!indexes_asts)
for (const auto &index_ast : 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( indexes.push_back(
std::move(MergeTreeIndexFactory::instance().get( std::move(MergeTreeIndexFactory::instance().get(
*this, *this,
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast), std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast),
global_context))); 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);
} }
} }
} }

View File

@ -309,7 +309,7 @@ public:
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTs & indexes_ast_, const ASTPtr & indexes_ast_,
const MergingParams & merging_params_, const MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool require_part_metadata_, 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 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(); void initPartitionKey();

View File

@ -553,23 +553,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
Names all_column_names = data.getColumns().getNamesOfPhysical(); Names all_column_names = data.getColumns().getNamesOfPhysical();
NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
LOG_DEBUG(log, "Before extract");
NamesAndTypesList gathering_columns, merging_columns; NamesAndTypesList gathering_columns, merging_columns;
Names gathering_column_names, merging_column_names; Names gathering_column_names, merging_column_names;
extractMergingAndGatheringColumns( extractMergingAndGatheringColumns(
all_columns, data.sorting_key_expr, data.indexes, all_columns, data.sorting_key_expr, data.indexes,
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); 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>( MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
data, future_part.name, future_part.part_info); data, future_part.name, future_part.part_info);
new_data_part->partition.assign(future_part.getPartition()); new_data_part->partition.assign(future_part.getPartition());
new_data_part->relative_path = TMP_PREFIX + future_part.name; new_data_part->relative_path = TMP_PREFIX + future_part.name;
new_data_part->is_temp = true; 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; 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); MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate);

View File

@ -12,6 +12,12 @@
namespace DB { namespace DB {
namespace ErrorCodes
{
extern const int FILE_DOESNT_EXIST;
}
class MergeTreeTestIndex; class MergeTreeTestIndex;
struct MergeTreeTestGranule : public MergeTreeIndexGranule { struct MergeTreeTestGranule : public MergeTreeIndexGranule {
@ -24,6 +30,9 @@ struct MergeTreeTestGranule : public MergeTreeIndexGranule {
void deserializeBinary(ReadBuffer &istr) override { void deserializeBinary(ReadBuffer &istr) override {
readIntBinary(emp, istr); readIntBinary(emp, istr);
if (emp != 10) {
throw Exception("kek bad read", ErrorCodes::FILE_DOESNT_EXIST);
}
//std::cerr << "TESTINDEX: read " << emp << "\n"; //std::cerr << "TESTINDEX: read " << emp << "\n";
} }

View File

@ -340,7 +340,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
bool is_extended_storage_def = bool is_extended_storage_def =
args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by 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")); 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 order_by_ast;
ASTPtr primary_key_ast; ASTPtr primary_key_ast;
ASTPtr sample_by_ast; ASTPtr sample_by_ast;
ASTs indexes_ast; ASTPtr indexes_ast;
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
if (is_extended_storage_def) if (is_extended_storage_def)
@ -584,8 +584,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.storage_def->sample_by) if (args.storage_def->sample_by)
sample_by_ast = args.storage_def->sample_by->ptr(); sample_by_ast = args.storage_def->sample_by->ptr();
for (auto& index : args.storage_def->indexes) { if (args.storage_def->indexes) {
indexes_ast.push_back(index->ptr()); indexes_ast = args.storage_def->indexes->ptr();
} }
storage_settings.loadFromQuery(*args.storage_def); storage_settings.loadFromQuery(*args.storage_def);

View File

@ -95,7 +95,7 @@ StoragePtr StorageFactory::get(
} }
if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by 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")) && !endsWith(name, "MergeTree"))
{ {
throw Exception( throw Exception(

View File

@ -58,7 +58,7 @@ StorageMergeTree::StorageMergeTree(
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. 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 MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool has_force_restore_data_flag) bool has_force_restore_data_flag)

View File

@ -174,7 +174,7 @@ protected:
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. 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 MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool has_force_restore_data_flag); bool has_force_restore_data_flag);

View File

@ -206,7 +206,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, const ASTPtr & sample_by_ast_,
const ASTs & indexes_ast_, const ASTPtr & indexes_ast_,
const MergeTreeData::MergingParams & merging_params_, const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool has_force_restore_data_flag) bool has_force_restore_data_flag)
@ -228,9 +228,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
{ {
if (path_.empty()) if (path_.empty())
throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); 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() == '/') if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1); zookeeper_path.resize(zookeeper_path.size() - 1);

View File

@ -552,7 +552,7 @@ protected:
const ASTPtr & order_by_ast_, const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_, const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, const ASTPtr & sample_by_ast_,
const ASTs & indexes_ast_, const ASTPtr & indexes_ast_,
const MergeTreeData::MergingParams & merging_params_, const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_, const MergeTreeSettings & settings_,
bool has_force_restore_data_flag); bool has_force_restore_data_flag);