mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add name hints for data skipping indices
fix test
This commit is contained in:
parent
b75fbaa6a9
commit
07280e0ab1
@ -29,7 +29,7 @@ private:
|
||||
void fillMaps();
|
||||
|
||||
public:
|
||||
EnumValues(const Values & values_);
|
||||
explicit EnumValues(const Values & values_);
|
||||
|
||||
const Values & getValues() const { return values; }
|
||||
|
||||
|
@ -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>;
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,2 @@
|
||||
OK
|
||||
OK
|
17
tests/queries/0_stateless/02225_hints_for_indeices.sh
Executable file
17
tests/queries/0_stateless/02225_hints_for_indeices.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user