Add name hints for data skipping indices

fix test
This commit is contained in:
feng lv 2022-02-20 11:45:13 +00:00
parent b75fbaa6a9
commit 07280e0ab1
8 changed files with 68 additions and 14 deletions

View File

@ -29,7 +29,7 @@ private:
void fillMaps();
public:
EnumValues(const Values & values_);
explicit EnumValues(const Values & values_);
const Values & getValues() const { return values; }

View File

@ -184,7 +184,9 @@ struct AggregationMethodOneNumber
AggregationMethodOneNumber() = default;
template <typename Other>
AggregationMethodOneNumber(const Other & other) : data(other.data) {}
explicit AggregationMethodOneNumber(const Other & other) : data(other.data)
{
}
/// To use one `Method` in different threads, use different `State`.
using State = ColumnsHashing::HashMethodOneNumber<typename Data::value_type,
@ -219,7 +221,9 @@ struct AggregationMethodString
AggregationMethodString() = default;
template <typename Other>
AggregationMethodString(const Other & other) : data(other.data) {}
explicit AggregationMethodString(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped>;
@ -247,7 +251,9 @@ struct AggregationMethodStringNoCache
AggregationMethodStringNoCache() = default;
template <typename Other>
AggregationMethodStringNoCache(const Other & other) : data(other.data) {}
explicit AggregationMethodStringNoCache(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped, true, false>;
@ -275,7 +281,9 @@ struct AggregationMethodFixedString
AggregationMethodFixedString() = default;
template <typename Other>
AggregationMethodFixedString(const Other & other) : data(other.data) {}
explicit AggregationMethodFixedString(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped>;
@ -302,7 +310,9 @@ struct AggregationMethodFixedStringNoCache
AggregationMethodFixedStringNoCache() = default;
template <typename Other>
AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) {}
explicit AggregationMethodFixedStringNoCache(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped, true, false>;
@ -373,7 +383,9 @@ struct AggregationMethodKeysFixed
AggregationMethodKeysFixed() = default;
template <typename Other>
AggregationMethodKeysFixed(const Other & other) : data(other.data) {}
explicit AggregationMethodKeysFixed(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodKeysFixed<
typename Data::value_type,
@ -462,7 +474,9 @@ struct AggregationMethodSerialized
AggregationMethodSerialized() = default;
template <typename Other>
AggregationMethodSerialized(const Other & other) : data(other.data) {}
explicit AggregationMethodSerialized(const Other & other) : data(other.data)
{
}
using State = ColumnsHashing::HashMethodSerialized<typename Data::value_type, Mapped>;

View File

@ -516,8 +516,13 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
});
if (insert_it == metadata.secondary_indices.end())
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
ErrorCodes::BAD_ARGUMENTS);
{
auto hints = metadata.secondary_indices.getHints(after_index_name);
auto hints_string = !hints.empty() ? ", may be you meant: " + toString(hints) : "";
throw Exception(
"Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after" + hints_string,
ErrorCodes::BAD_ARGUMENTS);
}
++insert_it;
}
@ -540,7 +545,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
{
if (if_exists)
return;
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.", ErrorCodes::BAD_ARGUMENTS);
auto hints = metadata.secondary_indices.getHints(index_name);
auto hints_string = !hints.empty() ? ", may be you meant: " + toString(hints) : "";
throw Exception(
"Wrong index name. Cannot find index " + backQuote(index_name) + " to drop" + hints_string, ErrorCodes::BAD_ARGUMENTS);
}
metadata.secondary_indices.erase(erase_it);
@ -582,7 +590,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
{
if (if_exists)
return;
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop",
ErrorCodes::BAD_ARGUMENTS);
}
constraints.erase(erase_it);

View File

@ -14,7 +14,7 @@ struct ConstraintsDescription
{
public:
ConstraintsDescription() { update(); }
ConstraintsDescription(const ASTs & constraints_);
explicit ConstraintsDescription(const ASTs & constraints_);
ConstraintsDescription(const ConstraintsDescription & other);
ConstraintsDescription & operator=(const ConstraintsDescription & other);

View File

@ -172,4 +172,13 @@ ExpressionActionsPtr IndicesDescription::getSingleExpressionForIndices(const Col
return ExpressionAnalyzer(combined_expr_list, syntax_result, context).getActions(false);
}
Names IndicesDescription::getAllRegisteredNames() const
{
Names result;
for (const auto & index : *this)
{
result.emplace_back(index.name);
}
return result;
}
}

View File

@ -8,6 +8,7 @@
#include <Interpreters/ExpressionActions.h>
#include <Parsers/IAST_fwd.h>
#include <Storages/ColumnsDescription.h>
#include <Common/NamePrompter.h>
namespace DB
{
@ -61,7 +62,7 @@ struct IndexDescription
};
/// All secondary indices in storage
struct IndicesDescription : public std::vector<IndexDescription>
struct IndicesDescription : public std::vector<IndexDescription>, IHints<1, IndicesDescription>
{
/// Index with name exists
bool has(const String & name) const;
@ -72,6 +73,9 @@ struct IndicesDescription : public std::vector<IndexDescription>
/// Return common expression for all stored indices
ExpressionActionsPtr getSingleExpressionForIndices(const ColumnsDescription & columns, ContextPtr context) const;
public:
Names getAllRegisteredNames() const override;
};
}

View File

@ -0,0 +1,2 @@
OK
OK

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t"
$CLICKHOUSE_CLIENT --query="CREATE TABLE t ENGINE=MergeTree ORDER BY n AS SELECT number AS n FROM numbers(10)"
$CLICKHOUSE_CLIENT --query="ALTER TABLE t ADD INDEX test_index n TYPE minmax GRANULARITY 32"
$CLICKHOUSE_CLIENT --query="ALTER TABLE t DROP INDEX test_indes" 2>&1 | grep -q "may be you meant: \['test_index'\]" && echo 'OK' || echo 'FAIL'
$CLICKHOUSE_CLIENT --query="ALTER TABLE t ADD INDEX test_index1 n TYPE minmax GRANULARITY 4 AFTER test_indes" 2>&1 | grep -q "may be you meant: \['test_index'\]" && echo 'OK' || echo 'FAIL'
$CLICKHOUSE_CLIENT --query="DROP TABLE t"