mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #21304 from amosbird/indexhint
Resurrect indexHint function.
This commit is contained in:
commit
d4e8f9676b
@ -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(<expression>)
|
||||
```
|
||||
|
||||
**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.
|
||||
|
@ -1133,6 +1133,111 @@ SELECT defaultValueOfTypeName('Nullable(Int8)')
|
||||
└──────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## indexHint {#indexhint}
|
||||
Возвращает все данные из диапазона, в который попадают данные, соответствующие указанному выражению.
|
||||
Переданное выражение не будет вычислено. Выбор диапазона производится по индексу.
|
||||
Индекс в ClickHouse разреженный, при чтении диапазона в ответ попадают «лишние» соседние данные.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
SELECT * FROM table WHERE indexHint(<expression>)
|
||||
```
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
Возвращает диапазон индекса, в котором выполняется заданное условие.
|
||||
|
||||
Тип: [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}
|
||||
|
||||
Создает массив, заполненный одним значением.
|
||||
|
@ -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}
|
||||
|
||||
使用单个值填充一个数组。
|
||||
|
70
src/Functions/indexHint.cpp
Normal file
70
src/Functions/indexHint.cpp
Normal file
@ -0,0 +1,70 @@
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
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<FunctionIndexHint>();
|
||||
}
|
||||
|
||||
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<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
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<FunctionIndexHint>();
|
||||
}
|
||||
|
||||
}
|
@ -39,7 +39,7 @@ struct DivideIntegralByConstantImpl
|
||||
|
||||
static ResultType process(A a, B b) { return Op::template apply<ResultType>(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"
|
||||
|
@ -43,7 +43,7 @@ struct ModuloByConstantImpl
|
||||
|
||||
static ResultType process(A a, B b) { return Op::template apply<ResultType>(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"
|
||||
|
@ -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);
|
||||
|
@ -303,6 +303,7 @@ SRCS(
|
||||
ignore.cpp
|
||||
ilike.cpp
|
||||
in.cpp
|
||||
indexHint.cpp
|
||||
initializeAggregation.cpp
|
||||
intDiv.cpp
|
||||
intDivOrZero.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())
|
||||
|
@ -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.
|
||||
|
@ -51,8 +51,10 @@ bool RequiredSourceColumnsMatcher::needChildVisit(const ASTPtr & node, const AST
|
||||
|
||||
if (const auto * f = node->as<ASTFunction>())
|
||||
{
|
||||
/// "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;
|
||||
}
|
||||
|
||||
|
@ -309,11 +309,11 @@ static const std::map<std::string, std::string> 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;
|
||||
|
@ -39,7 +39,6 @@
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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<const ASTExpressionList &>(*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;
|
||||
|
35
tests/queries/0_stateless/01739_index_hint.reference
Normal file
35
tests/queries/0_stateless/01739_index_hint.reference
Normal file
@ -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;
|
35
tests/queries/0_stateless/01739_index_hint.sql
Normal file
35
tests/queries/0_stateless/01739_index_hint.sql
Normal file
@ -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;
|
Loading…
Reference in New Issue
Block a user