diff --git a/src/DataTypes/EnumValues.h b/src/DataTypes/EnumValues.h index 17c292c5551..0747cd4aed8 100644 --- a/src/DataTypes/EnumValues.h +++ b/src/DataTypes/EnumValues.h @@ -29,7 +29,7 @@ private: void fillMaps(); public: - EnumValues(const Values & values_); + explicit EnumValues(const Values & values_); const Values & getValues() const { return values; } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 05c9133cb35..3457e01f98f 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -184,7 +184,9 @@ struct AggregationMethodOneNumber AggregationMethodOneNumber() = default; template - 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 - AggregationMethodString(const Other & other) : data(other.data) {} + explicit AggregationMethodString(const Other & other) : data(other.data) + { + } using State = ColumnsHashing::HashMethodString; @@ -247,7 +251,9 @@ struct AggregationMethodStringNoCache AggregationMethodStringNoCache() = default; template - AggregationMethodStringNoCache(const Other & other) : data(other.data) {} + explicit AggregationMethodStringNoCache(const Other & other) : data(other.data) + { + } using State = ColumnsHashing::HashMethodString; @@ -275,7 +281,9 @@ struct AggregationMethodFixedString AggregationMethodFixedString() = default; template - AggregationMethodFixedString(const Other & other) : data(other.data) {} + explicit AggregationMethodFixedString(const Other & other) : data(other.data) + { + } using State = ColumnsHashing::HashMethodFixedString; @@ -302,7 +310,9 @@ struct AggregationMethodFixedStringNoCache AggregationMethodFixedStringNoCache() = default; template - AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) {} + explicit AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) + { + } using State = ColumnsHashing::HashMethodFixedString; @@ -373,7 +383,9 @@ struct AggregationMethodKeysFixed AggregationMethodKeysFixed() = default; template - 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 - AggregationMethodSerialized(const Other & other) : data(other.data) {} + explicit AggregationMethodSerialized(const Other & other) : data(other.data) + { + } using State = ColumnsHashing::HashMethodSerialized; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 90e10abfa92..dd458967fc7 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -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); diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index ad8bd371f38..a5095a79ccb 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -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); diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index 9cdf9695172..a0a1bcbce2d 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -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; +} } diff --git a/src/Storages/IndicesDescription.h b/src/Storages/IndicesDescription.h index 7071ec89890..72e0748778f 100644 --- a/src/Storages/IndicesDescription.h +++ b/src/Storages/IndicesDescription.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -61,7 +62,7 @@ struct IndexDescription }; /// All secondary indices in storage -struct IndicesDescription : public std::vector +struct IndicesDescription : public std::vector, IHints<1, IndicesDescription> { /// Index with name exists bool has(const String & name) const; @@ -72,6 +73,9 @@ struct IndicesDescription : public std::vector /// Return common expression for all stored indices ExpressionActionsPtr getSingleExpressionForIndices(const ColumnsDescription & columns, ContextPtr context) const; + +public: + Names getAllRegisteredNames() const override; }; } diff --git a/tests/queries/0_stateless/02225_hints_for_indeices.reference b/tests/queries/0_stateless/02225_hints_for_indeices.reference new file mode 100644 index 00000000000..2c94e483710 --- /dev/null +++ b/tests/queries/0_stateless/02225_hints_for_indeices.reference @@ -0,0 +1,2 @@ +OK +OK diff --git a/tests/queries/0_stateless/02225_hints_for_indeices.sh b/tests/queries/0_stateless/02225_hints_for_indeices.sh new file mode 100755 index 00000000000..f4cfa17f8db --- /dev/null +++ b/tests/queries/0_stateless/02225_hints_for_indeices.sh @@ -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"