mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Merge branch 'master' into use-global-scalars-cache-analyzer
This commit is contained in:
commit
00557a9c6c
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -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));
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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";
|
||||
|
33
src/Functions/toMillisecond.cpp
Normal file
33
src/Functions/toMillisecond.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -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");
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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_,
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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";
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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 │
|
||||
|
@ -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
|
17
tests/queries/0_stateless/02998_to_milliseconds.sql
Normal file
17
tests/queries/0_stateless/02998_to_milliseconds.sql
Normal 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
|
@ -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}
|
@ -0,0 +1 @@
|
||||
1
|
12
tests/queries/0_stateless/03006_async_insert_deadlock_log.sh
Executable file
12
tests/queries/0_stateless/03006_async_insert_deadlock_log.sh
Executable 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"
|
6
tests/queries/0_stateless/03006_buffer_overflow_join.sql
Normal file
6
tests/queries/0_stateless/03006_buffer_overflow_join.sql
Normal 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;
|
@ -0,0 +1,3 @@
|
||||
\N 1000
|
||||
|
||||
\N 1000
|
@ -0,0 +1 @@
|
||||
SELECT count(NULL) IGNORE NULLS > avg(toDecimal32(NULL)) IGNORE NULLS, count() FROM numbers(1000) WITH TOTALS SETTINGS allow_experimental_analyzer = 1;
|
@ -2545,6 +2545,7 @@ toRelativeSecondNum
|
||||
toRelativeWeekNum
|
||||
toRelativeYearNum
|
||||
toSecond
|
||||
toMillisecond
|
||||
toStartOfDay
|
||||
toStartOfFifteenMinutes
|
||||
toStartOfFiveMinutes
|
||||
|
Loading…
Reference in New Issue
Block a user