Merge branch 'master' into use-global-scalars-cache-analyzer

This commit is contained in:
Nikolai Kochetov 2024-03-08 12:30:04 +00:00
commit 00557a9c6c
34 changed files with 501 additions and 166 deletions

View File

@ -394,8 +394,7 @@ Result:
## toYear
Converts a date or date with time to the year number (AD) as `UInt16` value.
Returns the year component (AD) of a date or date with time.
**Syntax**
@ -431,7 +430,7 @@ Result:
## toQuarter
Converts a date or date with time to the quarter number (1-4) as `UInt8` value.
Returns the quarter (1-4) of a date or date with time.
**Syntax**
@ -465,10 +464,9 @@ Result:
└──────────────────────────────────────────────┘
```
## toMonth
Converts a date or date with time to the month number (1-12) as `UInt8` value.
Returns the month component (1-12) of a date or date with time.
**Syntax**
@ -504,7 +502,7 @@ Result:
## toDayOfYear
Converts a date or date with time to the number of the day of the year (1-366) as `UInt16` value.
Returns the number of the day within the year (1-366) of a date or date with time.
**Syntax**
@ -540,7 +538,7 @@ Result:
## toDayOfMonth
Converts a date or date with time to the number of the day in the month (1-31) as `UInt8` value.
Returns the number of the day within the month (1-31) of a date or date with time.
**Syntax**
@ -576,7 +574,7 @@ Result:
## toDayOfWeek
Converts a date or date with time to the number of the day in the week as `UInt8` value.
Returns the number of the day within the week of a date or date with time.
The two-argument form of `toDayOfWeek()` enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or 1 to 7. If the mode argument is omitted, the default mode is 0. The time zone of the date can be specified as the third argument.
@ -627,7 +625,7 @@ Result:
## toHour
Converts a date with time to the number of the hour in 24-hour time (0-23) as `UInt8` value.
Returns the hour component (0-24) of a date with time.
Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always exactly when it occurs - it depends on the timezone).
@ -641,7 +639,7 @@ Alias: `HOUR`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -665,7 +663,7 @@ Result:
## toMinute
Converts a date with time to the number of the minute of the hour (0-59) as `UInt8` value.
Returns the minute component (0-59) a date with time.
**Syntax**
@ -677,7 +675,7 @@ Alias: `MINUTE`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -701,7 +699,7 @@ Result:
## toSecond
Converts a date with time to the second in the minute (0-59) as `UInt8` value. Leap seconds are not considered.
Returns the second component (0-59) of a date with time. Leap seconds are not considered.
**Syntax**
@ -713,7 +711,7 @@ Alias: `SECOND`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -735,6 +733,40 @@ Result:
└─────────────────────────────────────────────┘
```
## toMillisecond
Returns the millisecond component (0-999) of a date with time.
**Syntax**
```sql
toMillisecond(value)
```
*Arguments**
- `value` - [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
Alias: `MILLISECOND`
```sql
SELECT toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))
```
Result:
```response
┌──toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))─┐
│ 456 │
└────────────────────────────────────────────────────────────┘
```
**Returned value**
- The millisecond in the minute (0 - 59) of the given date/time
Type: `UInt16`
## toUnixTimestamp
Converts a string, a date or a date with time to the [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time) in `UInt32` representation.

View File

@ -5,7 +5,12 @@ sidebar_label: Window Functions
title: Window Functions
---
ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported:
Windows functions let you perform calculations across a set of rows that are related to the current row.
Some of the calculations that you can do are similar to those that can be done with an aggregate function, but a window function doesn't cause rows to be grouped into a single output - the individual rows are still returned.
## Standard Window Functions
ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported.
| Feature | Support or workaround |
|------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
@ -25,6 +30,8 @@ ClickHouse supports the standard grammar for defining windows and window functio
## ClickHouse-specific Window Functions
There are also the following window function that's specific to ClickHouse:
### nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL X UNITS])
Finds non-negative derivative for given `metric_column` by `timestamp_column`.
@ -33,40 +40,6 @@ The computed value is the following for each row:
- `0` for 1st row,
- ${metric_i - metric_{i-1} \over timestamp_i - timestamp_{i-1}} * interval$ for $i_th$ row.
## References
### GitHub Issues
The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097).
All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag.
### Tests
These tests contain the examples of the currently supported grammar:
https://github.com/ClickHouse/ClickHouse/blob/master/tests/performance/window_functions.xml
https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/01591_window_functions.sql
### Postgres Docs
https://www.postgresql.org/docs/current/sql-select.html#SQL-WINDOW
https://www.postgresql.org/docs/devel/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS
https://www.postgresql.org/docs/devel/functions-window.html
https://www.postgresql.org/docs/devel/tutorial-window.html
### MySQL Docs
https://dev.mysql.com/doc/refman/8.0/en/window-function-descriptions.html
https://dev.mysql.com/doc/refman/8.0/en/window-functions-usage.html
https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html
## Syntax
```text
@ -80,20 +53,7 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]
- `PARTITION BY` - defines how to break a resultset into groups.
- `ORDER BY` - defines how to order rows inside the group during calculation aggregate_function.
- `ROWS or RANGE` - defines bounds of a frame, aggregate_function is calculated within a frame.
- `WINDOW` - allows to reuse a window definition with multiple expressions.
### Functions
These functions can be used only as a window function.
- `row_number()` - Number the current row within its partition starting from 1.
- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame.
- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame.
- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame.
- `rank()` - Rank the current row within its partition with gaps.
- `dense_rank()` - Rank the current row within its partition without gaps.
- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame.
- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame.
- `WINDOW` - allows multiple expressions to use the same window definition.
```text
PARTITION
@ -112,8 +72,23 @@ These functions can be used only as a window function.
└─────────────────┘ <--- UNBOUNDED FOLLOWING (END of the PARTITION)
```
### Functions
These functions can be used only as a window function.
- `row_number()` - Number the current row within its partition starting from 1.
- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame.
- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame.
- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame.
- `rank()` - Rank the current row within its partition with gaps.
- `dense_rank()` - Rank the current row within its partition without gaps.
- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame.
- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame.
## Examples
Let's have a look at some examples of how window functions can be used.
```sql
CREATE TABLE wf_partition
(
@ -589,6 +564,41 @@ ORDER BY
└──────────────┴─────────────────────┴───────┴─────────────────────────┘
```
## References
### GitHub Issues
The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097).
All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag.
### Tests
These tests contain the examples of the currently supported grammar:
https://github.com/ClickHouse/ClickHouse/blob/master/tests/performance/window_functions.xml
https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/01591_window_functions.sql
### Postgres Docs
https://www.postgresql.org/docs/current/sql-select.html#SQL-WINDOW
https://www.postgresql.org/docs/devel/sql-expressions.html#SYNTAX-WINDOW-FUNCTIONS
https://www.postgresql.org/docs/devel/functions-window.html
https://www.postgresql.org/docs/devel/tutorial-window.html
### MySQL Docs
https://dev.mysql.com/doc/refman/8.0/en/window-function-descriptions.html
https://dev.mysql.com/doc/refman/8.0/en/window-functions-usage.html
https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html
## Related Content
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)

View File

@ -1,5 +1,7 @@
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Common/assert_cast.h>
#include <Common/FieldVisitorToString.h>
#include <Common/SipHash.h>
@ -38,6 +40,70 @@ ConstantNode::ConstantNode(Field value_)
: ConstantNode(value_, applyVisitor(FieldToDataType(), value_))
{}
bool ConstantNode::requiresCastCall() const
{
const auto & constant_value_literal = constant_value->getValue();
bool need_to_add_cast_function = false;
auto constant_value_literal_type = constant_value_literal.getType();
WhichDataType constant_value_type(constant_value->getType());
switch (constant_value_literal_type)
{
case Field::Types::String:
{
need_to_add_cast_function = !constant_value_type.isString();
break;
}
case Field::Types::UInt64:
case Field::Types::Int64:
case Field::Types::Float64:
{
WhichDataType constant_value_field_type(applyVisitor(FieldToDataType(), constant_value_literal));
need_to_add_cast_function = constant_value_field_type.idx != constant_value_type.idx;
break;
}
case Field::Types::Int128:
case Field::Types::UInt128:
case Field::Types::Int256:
case Field::Types::UInt256:
case Field::Types::Decimal32:
case Field::Types::Decimal64:
case Field::Types::Decimal128:
case Field::Types::Decimal256:
case Field::Types::AggregateFunctionState:
case Field::Types::Array:
case Field::Types::Tuple:
case Field::Types::Map:
case Field::Types::UUID:
case Field::Types::Bool:
case Field::Types::Object:
case Field::Types::IPv4:
case Field::Types::IPv6:
case Field::Types::Null:
case Field::Types::CustomType:
{
need_to_add_cast_function = true;
break;
}
}
// Add cast if constant was created as a result of constant folding.
// Constant folding may lead to type transformation and literal on shard
// may have a different type.
return need_to_add_cast_function || source_expression != nullptr;
}
bool ConstantNode::receivedFromInitiatorServer() const
{
if (!hasSourceExpression())
return false;
auto * cast_function = getSourceExpression()->as<FunctionNode>();
if (!cast_function || cast_function->getFunctionName() != "_CAST")
return false;
return true;
}
void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const
{
buffer << std::string(indent, ' ') << "CONSTANT id: " << format_state.getNodeId(this);
@ -89,54 +155,7 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const
if (!options.add_cast_for_constants)
return constant_value_ast;
bool need_to_add_cast_function = false;
auto constant_value_literal_type = constant_value_literal.getType();
WhichDataType constant_value_type(constant_value->getType());
switch (constant_value_literal_type)
{
case Field::Types::String:
{
need_to_add_cast_function = !constant_value_type.isString();
break;
}
case Field::Types::UInt64:
case Field::Types::Int64:
case Field::Types::Float64:
{
WhichDataType constant_value_field_type(applyVisitor(FieldToDataType(), constant_value_literal));
need_to_add_cast_function = constant_value_field_type.idx != constant_value_type.idx;
break;
}
case Field::Types::Int128:
case Field::Types::UInt128:
case Field::Types::Int256:
case Field::Types::UInt256:
case Field::Types::Decimal32:
case Field::Types::Decimal64:
case Field::Types::Decimal128:
case Field::Types::Decimal256:
case Field::Types::AggregateFunctionState:
case Field::Types::Array:
case Field::Types::Tuple:
case Field::Types::Map:
case Field::Types::UUID:
case Field::Types::Bool:
case Field::Types::Object:
case Field::Types::IPv4:
case Field::Types::IPv6:
case Field::Types::Null:
case Field::Types::CustomType:
{
need_to_add_cast_function = true;
break;
}
}
// Add cast if constant was created as a result of constant folding.
// Constant folding may lead to type transformation and literal on shard
// may have a different type.
if (need_to_add_cast_function || source_expression != nullptr)
if (requiresCastCall())
{
auto constant_type_name_ast = std::make_shared<ASTLiteral>(constant_value->getType()->getName());
return makeASTFunction("_CAST", std::move(constant_value_ast), std::move(constant_type_name_ast));

View File

@ -75,6 +75,12 @@ public:
return constant_value->getType();
}
/// Check if conversion to AST requires wrapping with _CAST function.
bool requiresCastCall() const;
/// Check if constant is a result of _CAST function constant folding.
bool receivedFromInitiatorServer() const;
void setMaskId(size_t id)
{
mask_id = id;

View File

@ -3,13 +3,13 @@
#include <base/DayNum.h>
#include <base/defines.h>
#include <base/types.h>
#include <Core/DecimalFunctions.h>
#include <ctime>
#include <cassert>
#include <string>
#include <type_traits>
#define DATE_SECONDS_PER_DAY 86400 /// Number of seconds in a day, 60 * 60 * 24
#define DATE_LUT_MIN_YEAR 1900 /// 1900 since majority of financial organizations consider 1900 as an initial year.
@ -280,9 +280,9 @@ private:
static_assert(std::is_integral_v<DateOrTime> && std::is_integral_v<Divisor>);
assert(divisor > 0);
if (likely(offset_is_whole_number_of_hours_during_epoch))
if (offset_is_whole_number_of_hours_during_epoch) [[likely]]
{
if (likely(x >= 0))
if (x >= 0) [[likely]]
return static_cast<DateOrTime>(x / divisor * divisor);
/// Integer division for negative numbers rounds them towards zero (up).
@ -576,10 +576,10 @@ public:
unsigned toSecond(Time t) const
{
if (likely(offset_is_whole_number_of_minutes_during_epoch))
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
{
Time res = t % 60;
if (likely(res >= 0))
if (res >= 0) [[likely]]
return static_cast<unsigned>(res);
return static_cast<unsigned>(res) + 60;
}
@ -593,6 +593,30 @@ public:
return time % 60;
}
template <typename DateOrTime>
unsigned toMillisecond(const DateOrTime & datetime, Int64 scale_multiplier) const
{
constexpr Int64 millisecond_multiplier = 1'000;
constexpr Int64 microsecond_multiplier = 1'000 * millisecond_multiplier;
constexpr Int64 divider = microsecond_multiplier / millisecond_multiplier;
auto components = DB::DecimalUtils::splitWithScaleMultiplier(datetime, scale_multiplier);
if (datetime.value < 0 && components.fractional)
{
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
--components.whole;
}
Int64 fractional = components.fractional;
if (scale_multiplier > microsecond_multiplier)
fractional = fractional / (scale_multiplier / microsecond_multiplier);
else if (scale_multiplier < microsecond_multiplier)
fractional = fractional * (microsecond_multiplier / scale_multiplier);
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
return millisecond;
}
unsigned toMinute(Time t) const
{
if (t >= 0 && offset_is_whole_number_of_hours_during_epoch)
@ -1122,9 +1146,9 @@ public:
DateOrTime toStartOfMinuteInterval(DateOrTime t, UInt64 minutes) const
{
Int64 divisor = 60 * minutes;
if (likely(offset_is_whole_number_of_minutes_during_epoch))
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
{
if (likely(t >= 0))
if (t >= 0) [[likely]]
return static_cast<DateOrTime>(t / divisor * divisor);
return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
}
@ -1339,7 +1363,7 @@ public:
UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
{
if (likely(day_of_month <= 28))
if (day_of_month <= 28) [[likely]]
return day_of_month;
UInt8 days_in_month = daysInMonth(year, month);

View File

@ -10,16 +10,17 @@ namespace ErrorCodes
void throwDateIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", name);
}
void throwDateTimeIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type Date for function {}", name);
}
void throwDate32IsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date32 of argument for function {}", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type Date32 for function {}", name);
}
void throwDateTimeIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type DateTime for function {}", name);
}
}

View File

@ -6,6 +6,7 @@
#include <Common/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <Common/IntervalKind.h>
#include "base/Decimal.h"
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnVector.h>
@ -54,8 +55,8 @@ constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; // 2149-06-06 23:59:59 U
constexpr time_t MAX_DATETIME_DAY_NUM = 49710; // 2106-02-07
[[noreturn]] void throwDateIsNotSupported(const char * name);
[[noreturn]] void throwDateTimeIsNotSupported(const char * name);
[[noreturn]] void throwDate32IsNotSupported(const char * name);
[[noreturn]] void throwDateTimeIsNotSupported(const char * name);
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
@ -481,7 +482,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Nanosecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -516,7 +517,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Microsecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -559,7 +560,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Millisecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -602,7 +603,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Second>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
{
@ -623,7 +624,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Minute>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
{
@ -644,7 +645,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Hour>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
{
@ -777,7 +778,7 @@ struct ToTimeImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -802,7 +803,7 @@ struct ToStartOfMinuteImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -849,7 +850,7 @@ struct ToStartOfSecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -897,7 +898,7 @@ struct ToStartOfMillisecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -941,7 +942,7 @@ struct ToStartOfMicrosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -979,7 +980,7 @@ struct ToStartOfNanosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1004,7 +1005,7 @@ struct ToStartOfFiveMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1036,7 +1037,7 @@ struct ToStartOfTenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1068,7 +1069,7 @@ struct ToStartOfFifteenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1103,7 +1104,7 @@ struct TimeSlotImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1142,7 +1143,7 @@ struct ToStartOfHourImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1429,7 +1430,7 @@ struct ToHourImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1456,7 +1457,7 @@ struct TimezoneOffsetImpl
static time_t execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static time_t execute(UInt16, const DateLUTImpl &)
@ -1482,7 +1483,7 @@ struct ToMinuteImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1507,7 +1508,7 @@ struct ToSecondImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1518,6 +1519,32 @@ struct ToSecondImpl
using FactorTransform = ToStartOfMinuteImpl;
};
struct ToMillisecondImpl
{
static constexpr auto name = "toMillisecond";
static UInt16 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl & time_zone)
{
return time_zone.toMillisecond<DateTime64>(datetime64, scale_multiplier);
}
static UInt16 execute(UInt32, const DateLUTImpl &)
{
return 0;
}
static UInt16 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
}
static UInt16 execute(UInt16, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
}
static constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
struct ToISOYearImpl
{
static constexpr auto name = "toISOYear";

View File

@ -0,0 +1,33 @@
#include <Common/FunctionDocumentation.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
using FunctionToMillisecond = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToMillisecondImpl>;
REGISTER_FUNCTION(ToMillisecond)
{
factory.registerFunction<FunctionToMillisecond>(
FunctionDocumentation{
.description=R"(
Returns the millisecond component (0-999) of a date with time.
)",
.syntax="toMillisecond(value)",
.arguments={{"value", "DateTime or DateTime64"}},
.returned_value="The millisecond in the minute (0 - 59) of the given date/time",
.examples{
{"toMillisecond", "SELECT toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3)", "456"}},
.categories{"Dates and Times"}
}
);
/// MySQL compatibility alias.
factory.registerAlias("MILLISECOND", "toMillisecond", FunctionFactory::CaseInsensitive);
}
}

View File

@ -1362,7 +1362,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
size_t num_result_columns = result.size();
if (mode == MatchColumnsMode::Position && num_input_columns != num_result_columns)
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match");
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match (source: {} and result: {})", num_input_columns, num_result_columns);
if (add_casted_columns && mode != MatchColumnsMode::Name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name");

View File

@ -142,8 +142,9 @@ Block getHeaderForProcessingStage(
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto storage = std::make_shared<StorageDummy>(
storage_snapshot->storage.getStorageID(), storage_snapshot->metadata->getColumns(), storage_snapshot);
auto storage = std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(),
storage_snapshot->getAllColumnsDescription(),
storage_snapshot);
InterpreterSelectQueryAnalyzer interpreter(query, context, storage, SelectQueryOptions(processed_stage).analyze());
result = interpreter.getSampleBlock();
}

View File

@ -1181,7 +1181,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node,
if (select_query_options.is_subquery)
updateContextForSubqueryExecution(mutable_context);
return std::make_shared<PlannerContext>(mutable_context, std::move(global_planner_context));
return std::make_shared<PlannerContext>(mutable_context, std::move(global_planner_context), select_query_options);
}
Planner::Planner(const QueryTreeNodePtr & query_tree_,

View File

@ -44,6 +44,27 @@ namespace ErrorCodes
namespace
{
/* Calculates Action node name for ConstantNode.
*
* If converting to AST will add a '_CAST' function call,
* the result action name will also include it.
*/
String calculateActionNodeNameWithCastIfNeeded(const ConstantNode & constant_node)
{
WriteBufferFromOwnString buffer;
if (constant_node.requiresCastCall())
buffer << "_CAST(";
buffer << calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType());
if (constant_node.requiresCastCall())
{
buffer << ", '" << constant_node.getResultType()->getName() << "'_String)";
}
return buffer.str();
}
class ActionNodeNameHelper
{
public:
@ -88,7 +109,49 @@ public:
case QueryTreeNodeType::CONSTANT:
{
const auto & constant_node = node->as<ConstantNode &>();
/* To ensure that headers match during distributed query we need to simulate action node naming on
* secondary servers. If we don't do that headers will mismatch due to constant folding.
*
* +--------+
* -----------------| Server |----------------
* / +--------+ \
* / \
* v v
* +-----------+ +-----------+
* | Initiator | ------ | Secondary |------
* +-----------+ / +-----------+ \
* | / \
* | / \
* v / \
* +---------------+ v v
* | Wrap in _CAST | +----------------------------+ +----------------------+
* | if needed | | Constant folded from _CAST | | Constant folded from |
* +---------------+ +----------------------------+ | another expression |
* | +----------------------+
* v |
* +----------------------------+ v
* | Name ConstantNode the same | +--------------------------+
* | as on initiator server | | Generate action name for |
* | (wrap in _CAST if needed) | | original expression |
* +----------------------------+ +--------------------------+
*/
if (planner_context.isASTLevelOptimizationAllowed())
{
result = calculateActionNodeNameWithCastIfNeeded(constant_node);
}
else
{
// Need to check if constant folded from QueryNode until https://github.com/ClickHouse/ClickHouse/issues/60847 is fixed.
if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY)
{
if (constant_node.receivedFromInitiatorServer())
result = calculateActionNodeNameWithCastIfNeeded(constant_node);
else
result = calculateActionNodeName(constant_node.getSourceExpression());
}
else
result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType());
}
break;
}
case QueryTreeNodeType::FUNCTION:
@ -530,7 +593,52 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
const auto & constant_literal = constant_node.getValue();
const auto & constant_type = constant_node.getResultType();
auto constant_node_name = calculateConstantActionNodeName(constant_literal, constant_type);
auto constant_node_name = [&]()
{
/* To ensure that headers match during distributed query we need to simulate action node naming on
* secondary servers. If we don't do that headers will mismatch due to constant folding.
*
* +--------+
* -----------------| Server |----------------
* / +--------+ \
* / \
* v v
* +-----------+ +-----------+
* | Initiator | ------ | Secondary |------
* +-----------+ / +-----------+ \
* | / \
* | / \
* v / \
* +---------------+ v v
* | Wrap in _CAST | +----------------------------+ +----------------------+
* | if needed | | Constant folded from _CAST | | Constant folded from |
* +---------------+ +----------------------------+ | another expression |
* | +----------------------+
* v |
* +----------------------------+ v
* | Name ConstantNode the same | +--------------------------+
* | as on initiator server | | Generate action name for |
* | (wrap in _CAST if needed) | | original expression |
* +----------------------------+ +--------------------------+
*/
if (planner_context->isASTLevelOptimizationAllowed())
{
return calculateActionNodeNameWithCastIfNeeded(constant_node);
}
else
{
// Need to check if constant folded from QueryNode until https://github.com/ClickHouse/ClickHouse/issues/60847 is fixed.
if (constant_node.hasSourceExpression() && constant_node.getSourceExpression()->getNodeType() != QueryTreeNodeType::QUERY)
{
if (constant_node.receivedFromInitiatorServer())
return calculateActionNodeNameWithCastIfNeeded(constant_node);
else
return action_node_name_helper.calculateActionNodeName(constant_node.getSourceExpression());
}
else
return calculateConstantActionNodeName(constant_literal, constant_type);
}
}();
ColumnWithTypeAndName column;
column.name = constant_node_name;

View File

@ -3,6 +3,7 @@
#include <Analyzer/TableNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -41,9 +42,10 @@ bool GlobalPlannerContext::hasColumnIdentifier(const ColumnIdentifier & column_i
return column_identifiers.contains(column_identifier);
}
PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_)
PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_, const SelectQueryOptions & select_query_options_)
: query_context(std::move(query_context_))
, global_planner_context(std::move(global_planner_context_))
, is_ast_level_optimization_allowed(!(query_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY || select_query_options_.ignore_ast_optimizations))
{}
TableExpressionData & PlannerContext::getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node)

View File

@ -10,6 +10,7 @@
#include <Analyzer/IQueryTreeNode.h>
#include <Planner/TableExpressionData.h>
#include <Interpreters/SelectQueryOptions.h>
namespace DB
{
@ -78,7 +79,7 @@ class PlannerContext
{
public:
/// Create planner context with query context and global planner context
PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_);
PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_, const SelectQueryOptions & select_query_options_);
/// Get planner context query context
ContextPtr getQueryContext() const
@ -165,6 +166,12 @@ public:
static SetKey createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node);
PreparedSets & getPreparedSets() { return prepared_sets; }
/// Returns false if any of following conditions met:
/// 1. Query is executed on a follower node.
/// 2. ignore_ast_optimizations is set.
bool isASTLevelOptimizationAllowed() const { return is_ast_level_optimization_allowed; }
private:
/// Query context
ContextMutablePtr query_context;
@ -172,6 +179,8 @@ private:
/// Global planner context
GlobalPlannerContextPtr global_planner_context;
bool is_ast_level_optimization_allowed;
/// Column node to column identifier
std::unordered_map<QueryTreeNodePtr, ColumnIdentifier> column_node_to_column_identifier;

View File

@ -951,6 +951,8 @@ void TCPHandler::processInsertQuery()
auto result = processAsyncInsertQuery(*insert_queue);
if (result.status == AsynchronousInsertQueue::PushResult::OK)
{
/// Reset pipeline because it may hold write lock for some storages.
state.io.pipeline.reset();
if (settings.wait_for_async_insert)
{
size_t timeout_ms = settings.wait_for_async_insert_timeout.totalMilliseconds();
@ -983,7 +985,7 @@ void TCPHandler::processInsertQuery()
else
{
PushingPipelineExecutor executor(state.io.pipeline);
run_executor(executor, processed_block);
run_executor(executor, std::move(processed_block));
}
sendInsertProfileEvents();

View File

@ -296,7 +296,6 @@ VirtualColumnsDescription StorageDistributed::createVirtuals()
StorageInMemoryMetadata metadata;
auto desc = MergeTreeData::createVirtuals(metadata);
desc.addEphemeral("_table", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "Name of a table");
desc.addEphemeral("_shard_num", std::make_shared<DataTypeUInt32>(), "Deprecated. Use function shardNum instead");
return desc;

View File

@ -1047,7 +1047,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources(
Block pipe_header = builder->getHeader();
if (has_database_virtual_column && !pipe_header.has("_database"))
if (has_database_virtual_column && common_header.has("_database") && !pipe_header.has("_database"))
{
ColumnWithTypeAndName column;
column.name = "_database";
@ -1062,7 +1062,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources(
{ return std::make_shared<ExpressionTransform>(stream_header, adding_column_actions); });
}
if (has_table_virtual_column && !pipe_header.has("_table"))
if (has_table_virtual_column && common_header.has("_table") && !pipe_header.has("_table"))
{
ColumnWithTypeAndName column;
column.name = "_table";

View File

@ -69,6 +69,14 @@ std::shared_ptr<StorageSnapshot> StorageSnapshot::clone(DataPtr data_) const
return res;
}
ColumnsDescription StorageSnapshot::getAllColumnsDescription() const
{
auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals();
auto column_names_and_types = getColumns(get_column_options);
return ColumnsDescription{column_names_and_types};
}
NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options) const
{
auto all_columns = getMetadataForQuery()->getColumns().get(options);

View File

@ -55,6 +55,9 @@ struct StorageSnapshot
std::shared_ptr<StorageSnapshot> clone(DataPtr data_) const;
/// Get columns description
ColumnsDescription getAllColumnsDescription() const;
/// Get all available columns with types according to options.
NamesAndTypesList getColumns(const GetColumnsOptions & options) const;

View File

@ -71,6 +71,9 @@ std::vector<size_t> TableFunctionExecutable::skipAnalysisForArguments(const Quer
const auto & table_function_node_arguments = table_function_node.getArguments().getNodes();
size_t table_function_node_arguments_size = table_function_node_arguments.size();
if (table_function_node_arguments_size <= 2)
return {};
std::vector<size_t> result_indexes;
result_indexes.reserve(table_function_node_arguments_size - 2);
for (size_t i = 2; i < table_function_node_arguments_size; ++i)

View File

@ -2,4 +2,3 @@ test_build_sets_from_multiple_threads/test.py::test_set
test_concurrent_backups_s3/test.py::test_concurrent_backups
test_distributed_type_object/test.py::test_distributed_type_object
test_merge_table_over_distributed/test.py::test_global_in
test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed

View File

@ -36,7 +36,7 @@ Header: avgWeighted(x, y) Nullable(Float64)
Header: x Nullable(Nothing)
y UInt8
Expression (Projection)
Header: NULL_Nullable(Nothing) Nullable(Nothing)
Header: _CAST(NULL_Nullable(Nothing), \'Nullable(Nothing)\'_String) Nullable(Nothing)
1_UInt8 UInt8
Expression (Change column names to column identifiers)
Header: __table5.dummy UInt8

View File

@ -60,7 +60,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom
│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │
│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │
│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │
│ _table │ LowCardinality(String) │ │ │ Name of a table │ │ │ 1 │
│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 1 │
└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘
SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
@ -94,7 +93,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom
│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │
│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │
│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │
│ _table │ LowCardinality(String) │ │ │ Name of a table │ │ │ 0 │ 1 │
│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 0 │ 1 │
│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │
│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │
@ -160,7 +158,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom
│ _part_offset │ UInt64 │ 1 │
│ _row_exists │ UInt8 │ 1 │
│ _block_number │ UInt64 │ 1 │
│ _table │ LowCardinality(String) │ 1 │
│ _shard_num │ UInt32 │ 1 │
└────────────────┴───────────────────────────┴────────────┘
SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
@ -194,7 +191,6 @@ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCom
│ _part_offset │ UInt64 │ 0 │ 1 │
│ _row_exists │ UInt8 │ 0 │ 1 │
│ _block_number │ UInt64 │ 0 │ 1 │
│ _table │ LowCardinality(String) │ 0 │ 1 │
│ _shard_num │ UInt32 │ 0 │ 1 │
│ arr.size0 │ UInt64 │ 1 │ 0 │
│ t.a │ String │ 1 │ 0 │

View File

@ -0,0 +1,8 @@
2023-04-21 10:20:30 0 0
2023-04-21 10:20:30 0 0
2023-04-21 10:20:30.123 123 123
2023-04-21 10:20:30.123456 123 123
2023-04-21 10:20:30.123456789 123 123
120
2023-04-21 10:20:30 0
2023-04-21 10:20:30 0

View File

@ -0,0 +1,17 @@
-- Negative tests
SELECT toMillisecond(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT toMillisecond('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toMillisecond(toDate('2024-02-28')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toMillisecond(toDate32('2024-02-28')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Tests with constant and non-constant arguments
SELECT toDateTime('2023-04-21 10:20:30') AS dt, toMillisecond(dt), toMillisecond(materialize(dt));
SELECT toDateTime64('2023-04-21 10:20:30', 0) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123', 3) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123456', 6) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123456789', 9) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
-- Special cases
SELECT MILLISECOND(toDateTime64('2023-04-21 10:20:30.123456', 2)); -- Alias
SELECT toNullable(toDateTime('2023-04-21 10:20:30')) AS dt, toMillisecond(dt); -- Nullable
SELECT toLowCardinality(toDateTime('2023-04-21 10:20:30')) AS dt, toMillisecond(dt); -- LowCardinality

View File

@ -0,0 +1,4 @@
SELECT
toFixedString(toFixedString(toLowCardinality(toFixedString('--------------------', toNullable(20))), toLowCardinality(20)), 20),
*
FROM executable('data String', SETTINGS max_command_execution_time = 100); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query "CREATE TABLE t_async_insert_deadlock (a UInt64) ENGINE = Log"
echo '{"a": 1}' | $CLICKHOUSE_CLIENT --async_insert 1 --wait_for_async_insert 1 --query "INSERT INTO t_async_insert_deadlock FORMAT JSONEachRow"
$CLICKHOUSE_CLIENT --query "SELECT * FROM t_async_insert_deadlock ORDER BY a"
$CLICKHOUSE_CLIENT --query "DROP TABLE t_async_insert_deadlock"

View File

@ -0,0 +1,6 @@
CREATE TABLE 03006_buffer_overflow_l (`a` String, `b` Tuple(String, String)) ENGINE = Memory;
INSERT INTO 03006_buffer_overflow_l SELECT * FROM generateRandom() limit 1000;
CREATE TABLE 03006_buffer_overflow_r (`a` LowCardinality(Nullable(String)), `c` Tuple(LowCardinality(String), LowCardinality(String))) ENGINE = Memory;
INSERT INTO 03006_buffer_overflow_r SELECT * FROM generateRandom() limit 1000;
SELECT a FROM 03006_buffer_overflow_l RIGHT JOIN 03006_buffer_overflow_r USING (a) ORDER BY a ASC NULLS FIRST FORMAT Null;

View File

@ -0,0 +1,3 @@
\N 1000
\N 1000

View File

@ -0,0 +1 @@
SELECT count(NULL) IGNORE NULLS > avg(toDecimal32(NULL)) IGNORE NULLS, count() FROM numbers(1000) WITH TOTALS SETTINGS allow_experimental_analyzer = 1;

View File

@ -2545,6 +2545,7 @@ toRelativeSecondNum
toRelativeWeekNum
toRelativeYearNum
toSecond
toMillisecond
toStartOfDay
toStartOfFifteenMinutes
toStartOfFiveMinutes