diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index c40aa3d1eae..9d7743e186f 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1192,6 +1192,109 @@ SELECT defaultValueOfTypeName('Nullable(Int8)') └──────────────────────────────────────────┘ ``` +## indexHint {#indexhint} +The function is intended for debugging and introspection purposes. The function ignores it's argument and always returns 1. Arguments are not even evaluated. + +But for the purpose of index analysis, the argument of this function is analyzed as if it was present directly without being wrapped inside `indexHint` function. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. + +**Syntax** + +```sql +SELECT * FROM table WHERE indexHint() +``` + +**Returned value** + +1. Type: [Uint8](https://clickhouse.yandex/docs/en/data_types/int_uint/#diapazony-uint). + +**Example** + +Here is the example of test data from the table [ontime](../../getting-started/example-datasets/ontime.md). + +Input table: + +```sql +SELECT count() FROM ontime +``` + +```text +┌─count()─┐ +│ 4276457 │ +└─────────┘ +``` + +The table has indexes on the fields `(FlightDate, (Year, FlightDate))`. + +Create a query, where the index is not used. + +Query: + +```sql +SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k +``` + +ClickHouse processed the entire table (`Processed 4.28 million rows`). + +Result: + +```text +┌──────────k─┬─count()─┐ +│ 2017-01-01 │ 13970 │ +│ 2017-01-02 │ 15882 │ +........................ +│ 2017-09-28 │ 16411 │ +│ 2017-09-29 │ 16384 │ +│ 2017-09-30 │ 12520 │ +└────────────┴─────────┘ +``` + +To apply the index, select a specific date. + +Query: + +```sql +SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k +``` + +By using the index, ClickHouse processed a significantly smaller number of rows (`Processed 32.74 thousand rows`). + +Result: + +```text +┌──────────k─┬─count()─┐ +│ 2017-09-15 │ 16428 │ +└────────────┴─────────┘ +``` + +Now wrap the expression `k = '2017-09-15'` into `indexHint` function. + +Query: + +```sql +SELECT + FlightDate AS k, + count() +FROM ontime +WHERE indexHint(k = '2017-09-15') +GROUP BY k +ORDER BY k ASC +``` + +ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). +The expression `k = '2017-09-15'` was not used when generating the result. +In examle the `indexHint` function allows to see adjacent dates. + +Result: + +```text +┌──────────k─┬─count()─┐ +│ 2017-09-14 │ 7071 │ +│ 2017-09-15 │ 16428 │ +│ 2017-09-16 │ 1077 │ +│ 2017-09-30 │ 8167 │ +└────────────┴─────────┘ +``` + ## replicate {#other-functions-replicate} Creates an array with a single value. diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index f9b3e5c3e68..84bbc6af968 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -1133,6 +1133,111 @@ SELECT defaultValueOfTypeName('Nullable(Int8)') └──────────────────────────────────────────┘ ``` +## indexHint {#indexhint} +Возвращает все данные из диапазона, в который попадают данные, соответствующие указанному выражению. +Переданное выражение не будет вычислено. Выбор диапазона производится по индексу. +Индекс в ClickHouse разреженный, при чтении диапазона в ответ попадают «лишние» соседние данные. + +**Синтаксис** + +```sql +SELECT * FROM table WHERE indexHint() +``` + +**Возвращаемое значение** + +Возвращает диапазон индекса, в котором выполняется заданное условие. + +Тип: [Uint8](https://clickhouse.yandex/docs/ru/data_types/int_uint/#diapazony-uint). + +**Пример** + +Рассмотрим пример с использованием тестовых данных таблицы [ontime](../../getting-started/example-datasets/ontime.md). + +Исходная таблица: + +```sql +SELECT count() FROM ontime +``` + +```text +┌─count()─┐ +│ 4276457 │ +└─────────┘ +``` + +В таблице есть индексы по полям `(FlightDate, (Year, FlightDate))`. + +Выполним выборку по дате, где индекс не используется. + +Запрос: + +```sql +SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k +``` + +ClickHouse обработал всю таблицу (`Processed 4.28 million rows`). + +Результат: + +```text +┌──────────k─┬─count()─┐ +│ 2017-01-01 │ 13970 │ +│ 2017-01-02 │ 15882 │ +........................ +│ 2017-09-28 │ 16411 │ +│ 2017-09-29 │ 16384 │ +│ 2017-09-30 │ 12520 │ +└────────────┴─────────┘ +``` + +Для подключения индекса выбираем конкретную дату. + +Запрос: + +```sql +SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k +``` + +При использовании индекса ClickHouse обработал значительно меньшее количество строк (`Processed 32.74 thousand rows`). + +Результат: + +```text +┌──────────k─┬─count()─┐ +│ 2017-09-15 │ 16428 │ +└────────────┴─────────┘ +``` + +Передадим в функцию `indexHint` выражение `k = '2017-09-15'`. + +Запрос: + +```sql +SELECT + FlightDate AS k, + count() +FROM ontime +WHERE indexHint(k = '2017-09-15') +GROUP BY k +ORDER BY k ASC +``` + +ClickHouse применил индекс по аналогии с примером выше (`Processed 32.74 thousand rows`). +Выражение `k = '2017-09-15'` не используется при формировании результата. +Функция `indexHint` позволяет увидеть соседние данные. + +Результат: + +```text +┌──────────k─┬─count()─┐ +│ 2017-09-14 │ 7071 │ +│ 2017-09-15 │ 16428 │ +│ 2017-09-16 │ 1077 │ +│ 2017-09-30 │ 8167 │ +└────────────┴─────────┘ +``` + ## replicate {#other-functions-replicate} Создает массив, заполненный одним значением. diff --git a/docs/zh/sql-reference/functions/other-functions.md b/docs/zh/sql-reference/functions/other-functions.md index b17a5e89332..c58c4bd1510 100644 --- a/docs/zh/sql-reference/functions/other-functions.md +++ b/docs/zh/sql-reference/functions/other-functions.md @@ -477,6 +477,103 @@ FROM 1 rows in set. Elapsed: 0.002 sec. + +## indexHint {#indexhint} +输出符合索引选择范围内的所有数据,同时不实用参数中的表达式进行过滤。 + +传递给函数的表达式参数将不会被计算,但ClickHouse使用参数中的表达式进行索引过滤。 + +**返回值** + +- 1。 + +**示例** + +这是一个包含[ontime](../../getting-started/example-datasets/ontime.md)测试数据集的测试表。 + +``` +SELECT count() FROM ontime + +┌─count()─┐ +│ 4276457 │ +└─────────┘ +``` + +该表使用`(FlightDate, (Year, FlightDate))`作为索引。 + +对该表进行如下的查询: + +``` +:) SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k + +SELECT + FlightDate AS k, + count() +FROM ontime +GROUP BY k +ORDER BY k ASC + +┌──────────k─┬─count()─┐ +│ 2017-01-01 │ 13970 │ +│ 2017-01-02 │ 15882 │ +........................ +│ 2017-09-28 │ 16411 │ +│ 2017-09-29 │ 16384 │ +│ 2017-09-30 │ 12520 │ +└────────────┴─────────┘ + +273 rows in set. Elapsed: 0.072 sec. Processed 4.28 million rows, 8.55 MB (59.00 million rows/s., 118.01 MB/s.) +``` + +在这个查询中,由于没有使用索引,所以ClickHouse将处理整个表的所有数据(`Processed 4.28 million rows`)。使用下面的查询尝试使用索引进行查询: + +``` +:) SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k + +SELECT + FlightDate AS k, + count() +FROM ontime +WHERE k = '2017-09-15' +GROUP BY k +ORDER BY k ASC + +┌──────────k─┬─count()─┐ +│ 2017-09-15 │ 16428 │ +└────────────┴─────────┘ + +1 rows in set. Elapsed: 0.014 sec. Processed 32.74 thousand rows, 65.49 KB (2.31 million rows/s., 4.63 MB/s.) +``` + +在最后一行的显示中,通过索引ClickHouse处理的行数明显减少(`Processed 32.74 thousand rows`)。 + +现在将表达式`k = '2017-09-15'`传递给`indexHint`函数: + +``` +:) SELECT FlightDate AS k, count() FROM ontime WHERE indexHint(k = '2017-09-15') GROUP BY k ORDER BY k + +SELECT + FlightDate AS k, + count() +FROM ontime +WHERE indexHint(k = '2017-09-15') +GROUP BY k +ORDER BY k ASC + +┌──────────k─┬─count()─┐ +│ 2017-09-14 │ 7071 │ +│ 2017-09-15 │ 16428 │ +│ 2017-09-16 │ 1077 │ +│ 2017-09-30 │ 8167 │ +└────────────┴─────────┘ + +4 rows in set. Elapsed: 0.004 sec. Processed 32.74 thousand rows, 65.49 KB (8.97 million rows/s., 17.94 MB/s.) +``` + +对于这个请求,根据ClickHouse显示ClickHouse与上一次相同的方式应用了索引(`Processed 32.74 thousand rows`)。但是,最终返回的结果集中并没有根据`k = '2017-09-15'`表达式进行过滤结果。 + +由于ClickHouse中使用稀疏索引,因此在读取范围时(本示例中为相邻日期),"额外"的数据将包含在索引结果中。使用`indexHint`函数可以查看到它们。 + ## 复制 {#replicate} 使用单个值填充一个数组。 diff --git a/src/Functions/indexHint.cpp b/src/Functions/indexHint.cpp new file mode 100644 index 00000000000..3a893201896 --- /dev/null +++ b/src/Functions/indexHint.cpp @@ -0,0 +1,70 @@ +#include +#include +#include + + +namespace DB +{ + + +/** The `indexHint` function takes any number of any arguments and always returns one. + * + * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) + * - the expressions inside it are not evaluated; + * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, + * as if instead of using it the expression itself would be. + * + * Example: WHERE something AND indexHint(CounterID = 34) + * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. + * + * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. + */ +class FunctionIndexHint : public IFunction +{ +public: + static constexpr auto name = "indexHint"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + bool isSuitableForConstantFolding() const override { return false; } + + String getName() const override + { + return name; + } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeUInt8().createColumnConst(input_rows_count, 1u); + } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName &) const override + { + return DataTypeUInt8().createColumnConst(1, 1u); + } +}; + + +void registerFunctionIndexHint(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index 79e35a19283..14df8af66ca 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -39,7 +39,7 @@ struct DivideIntegralByConstantImpl static ResultType process(A a, B b) { return Op::template apply(a, b); } - static NO_INLINE void vectorConstant(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size) + static void NO_INLINE NO_SANITIZE_UNDEFINED vectorConstant(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size) { #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index 6ac0c82bd3e..ed68982cd71 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -43,7 +43,7 @@ struct ModuloByConstantImpl static ResultType process(A a, B b) { return Op::template apply(a, b); } - static void NO_INLINE vectorConstant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size) + static void NO_INLINE NO_SANITIZE_UNDEFINED vectorConstant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size) { #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index d00d4a42db0..3403390ea72 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -28,6 +28,7 @@ void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); void registerFunctionIgnore(FunctionFactory &); +void registerFunctionIndexHint(FunctionFactory &); void registerFunctionIdentity(FunctionFactory &); void registerFunctionArrayJoin(FunctionFactory &); void registerFunctionReplicate(FunctionFactory &); @@ -101,6 +102,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); registerFunctionIgnore(factory); + registerFunctionIndexHint(factory); registerFunctionIdentity(factory); registerFunctionArrayJoin(factory); registerFunctionReplicate(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 0f4f7a321b2..2a541369ff4 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -303,6 +303,7 @@ SRCS( ignore.cpp ilike.cpp in.cpp + indexHint.cpp initializeAggregation.cpp intDiv.cpp intDivOrZero.cpp diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4d2465b6e93..2e6a6398291 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -181,7 +181,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( } } - /// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant. + /// Some functions like ignore(), indexHint() or getTypeName() always return constant result even if arguments are not constant. /// We can't do constant folding, but can specify in sample block that function result is constant to avoid /// unnecessary materialization. if (!node.column && node.function_base->isSuitableForConstantFolding()) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 2ad1d0804c4..b7d679a81f1 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -811,6 +811,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & } } + /// A special function `indexHint`. Everything that is inside it is not calculated + if (node.name == "indexHint") + { + // Arguments are removed. We add function instead of constant column to avoid constant folding. + data.addFunction(FunctionFactory::instance().get("indexHint", data.getContext()), {}, column_name); + return; + } + if (node.is_window_function) { // Also add columns from PARTITION BY and ORDER BY of window functions. diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 54883043d30..2f2a68656bc 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -51,8 +51,10 @@ bool RequiredSourceColumnsMatcher::needChildVisit(const ASTPtr & node, const AST if (const auto * f = node->as()) { + /// "indexHint" is a special function for index analysis. + /// Everything that is inside it is not calculated. See KeyCondition /// "lambda" visit children itself. - if (f->name == "lambda") + if (f->name == "indexHint" || f->name == "lambda") return false; } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 43419f9ce5e..da36b7008bd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -309,11 +309,11 @@ static const std::map inverse_relations = { bool isLogicalOperator(const String & func_name) { - return (func_name == "and" || func_name == "or" || func_name == "not"); + return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint"); } /// The node can be one of: -/// - Logical operator (AND, OR, NOT) +/// - Logical operator (AND, OR, NOT and indexHint() - logical NOOP) /// - An "atom" (relational operator, constant, expression) /// - A logical constant expression /// - Any other function @@ -330,7 +330,8 @@ ASTPtr cloneASTWithInversionPushDown(const ASTPtr node, const bool need_inversio const auto result_node = makeASTFunction(func->name); - if (need_inversion) + /// indexHint() is a special case - logical NOOP function + if (result_node->name != "indexHint" && need_inversion) { result_node->name = (result_node->name == "and") ? "or" : "and"; } @@ -965,6 +966,8 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( { const auto & args = (*it)->arguments->children; auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context); + if (!func_builder) + return false; ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; @@ -1277,6 +1280,8 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out) { /// Functions AND, OR, NOT. + /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses + /// (or, the same thing - calling the function `and` from one argument). const ASTs & args = func->arguments->children; if (func->name == "not") @@ -1288,7 +1293,7 @@ bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNE } else { - if (func->name == "and") + if (func->name == "and" || func->name == "indexHint") out.function = RPNElement::FUNCTION_AND; else if (func->name == "or") out.function = RPNElement::FUNCTION_OR; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index af72b3e53f2..1340332350f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -39,7 +39,6 @@ #include #include #include -#include namespace ProfileEvents { diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 53ee063486b..ff875b185e9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -392,7 +392,7 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) func->name = "__bitSwapLastTwo"; } - else if (func->name == "and") + else if (func->name == "and" || func->name == "indexHint") { auto last_arg = args.back(); args.pop_back(); @@ -448,7 +448,7 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr & node, bool atomi const ASTs & args = func->arguments->children; - if (func->name == "and") + if (func->name == "and" || func->name == "indexHint") return checkASTUseless(args[0], atomic) && checkASTUseless(args[1], atomic); else if (func->name == "or") return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 50f8266a2a0..3e2e77d5de7 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -339,6 +339,10 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) c if ("globalIn" == function_ptr->name || "globalNotIn" == function_ptr->name) return true; + + /// indexHint is a special function that it does not make sense to transfer to PREWHERE + if ("indexHint" == function_ptr->name) + return true; } else if (auto opt_name = IdentifierSemantic::getColumnName(ptr)) { diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index d41c53eb106..d63781db67d 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -90,6 +90,8 @@ private: bool operatorFromAST(const ASTFunction * func, RPNElement & out) { /// Functions AND, OR, NOT. + /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses + /// (or, the same thing - calling the function `and` from one argument). const ASTs & args = typeid_cast(*func->arguments).children; if (func->name == "not") @@ -101,7 +103,7 @@ private: } else { - if (func->name == "and") + if (func->name == "and" || func->name == "indexHint") out.function = RPNElement::FUNCTION_AND; else if (func->name == "or") out.function = RPNElement::FUNCTION_OR; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference new file mode 100644 index 00000000000..6aa40c5d302 --- /dev/null +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -0,0 +1,35 @@ +-- { echo } + +drop table if exists tbl; +create table tbl (p Int64, t Int64, f Float64) Engine=MergeTree partition by p order by t settings index_granularity=1; +insert into tbl select number / 4, number, 0 from numbers(16); +select * from tbl WHERE indexHint(t = 1) order by t; +0 0 0 +0 1 0 +select * from tbl WHERE indexHint(t in (select toInt64(number) + 2 from numbers(3))) order by t; +0 1 0 +0 2 0 +0 3 0 +1 4 0 +select * from tbl WHERE indexHint(p = 2) order by t; +2 8 0 +2 9 0 +2 10 0 +2 11 0 +select * from tbl WHERE indexHint(p in (select toInt64(number) - 2 from numbers(3))) order by t; +0 0 0 +0 1 0 +0 2 0 +0 3 0 +drop table tbl; +drop table if exists XXXX; +create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128; +insert into XXXX select number*60, 0 from numbers(100000); +SELECT count() FROM XXXX WHERE indexHint(t = 42); +128 +drop table if exists XXXX; +create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; +insert into XXXX select number*60, 0 from numbers(100000); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +100000 +drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql new file mode 100644 index 00000000000..28395c2dc1d --- /dev/null +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -0,0 +1,35 @@ +-- { echo } + +drop table if exists tbl; + +create table tbl (p Int64, t Int64, f Float64) Engine=MergeTree partition by p order by t settings index_granularity=1; + +insert into tbl select number / 4, number, 0 from numbers(16); + +select * from tbl WHERE indexHint(t = 1) order by t; + +select * from tbl WHERE indexHint(t in (select toInt64(number) + 2 from numbers(3))) order by t; + +select * from tbl WHERE indexHint(p = 2) order by t; + +select * from tbl WHERE indexHint(p in (select toInt64(number) - 2 from numbers(3))) order by t; + +drop table tbl; + +drop table if exists XXXX; + +create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=128; + +insert into XXXX select number*60, 0 from numbers(100000); + +SELECT count() FROM XXXX WHERE indexHint(t = 42); + +drop table if exists XXXX; + +create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; + +insert into XXXX select number*60, 0 from numbers(100000); + +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); + +drop table XXXX;