mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into add-more-logging
This commit is contained in:
commit
f0dac3dfd5
@ -314,13 +314,13 @@ static int read_unicode(json_stream *json)
|
||||
|
||||
if (l < 0xdc00 || l > 0xdfff) {
|
||||
json_error(json, "invalid surrogate pair continuation \\u%04lx out "
|
||||
"of range (dc00-dfff)", l);
|
||||
"of range (dc00-dfff)", (unsigned long)l);
|
||||
return -1;
|
||||
}
|
||||
|
||||
cp = ((h - 0xd800) * 0x400) + ((l - 0xdc00) + 0x10000);
|
||||
} else if (cp >= 0xdc00 && cp <= 0xdfff) {
|
||||
json_error(json, "dangling surrogate \\u%04lx", cp);
|
||||
json_error(json, "dangling surrogate \\u%04lx", (unsigned long)cp);
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
2
contrib/NuRaft
vendored
2
contrib/NuRaft
vendored
@ -1 +1 @@
|
||||
Subproject commit 4a12f99dfc9d47c687ff7700b927cc76856225d1
|
||||
Subproject commit 08ac76ea80a37f89b12109c805eafe9f1dc9b991
|
@ -32,6 +32,7 @@ set(SRCS
|
||||
"${LIBRARY_DIR}/src/handle_custom_notification.cxx"
|
||||
"${LIBRARY_DIR}/src/handle_vote.cxx"
|
||||
"${LIBRARY_DIR}/src/launcher.cxx"
|
||||
"${LIBRARY_DIR}/src/log_entry.cxx"
|
||||
"${LIBRARY_DIR}/src/srv_config.cxx"
|
||||
"${LIBRARY_DIR}/src/snapshot_sync_req.cxx"
|
||||
"${LIBRARY_DIR}/src/snapshot_sync_ctx.cxx"
|
||||
|
@ -47,7 +47,7 @@ An example:
|
||||
<engine>ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024</engine>
|
||||
-->
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<max_size_rows>1048576</max_size>
|
||||
<max_size_rows>1048576</max_size_rows>
|
||||
<reserved_size_rows>8192</reserved_size_rows>
|
||||
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
|
||||
<flush_on_crash>false</flush_on_crash>
|
||||
|
@ -483,7 +483,7 @@ Where:
|
||||
|
||||
- `r1`- the number of unique visitors who visited the site during 2020-01-01 (the `cond1` condition).
|
||||
- `r2`- the number of unique visitors who visited the site during a specific time period between 2020-01-01 and 2020-01-02 (`cond1` and `cond2` conditions).
|
||||
- `r3`- the number of unique visitors who visited the site during a specific time period between 2020-01-01 and 2020-01-03 (`cond1` and `cond3` conditions).
|
||||
- `r3`- the number of unique visitors who visited the site during a specific time period on 2020-01-01 and 2020-01-03 (`cond1` and `cond3` conditions).
|
||||
|
||||
## uniqUpTo(N)(x)
|
||||
|
||||
|
@ -1670,7 +1670,7 @@ Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../.
|
||||
|
||||
## age
|
||||
|
||||
Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond.
|
||||
Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 nanosecond.
|
||||
E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit.
|
||||
|
||||
For an alternative to `age`, see function `date\_diff`.
|
||||
@ -1686,16 +1686,17 @@ age('unit', startdate, enddate, [timezone])
|
||||
- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
|
||||
Possible values:
|
||||
|
||||
- `microsecond` `microseconds` `us` `u`
|
||||
- `millisecond` `milliseconds` `ms`
|
||||
- `second` `seconds` `ss` `s`
|
||||
- `minute` `minutes` `mi` `n`
|
||||
- `hour` `hours` `hh` `h`
|
||||
- `day` `days` `dd` `d`
|
||||
- `week` `weeks` `wk` `ww`
|
||||
- `month` `months` `mm` `m`
|
||||
- `quarter` `quarters` `qq` `q`
|
||||
- `year` `years` `yyyy` `yy`
|
||||
- `nanosecond`, `nanoseconds`, `ns`
|
||||
- `microsecond`, `microseconds`, `us`, `u`
|
||||
- `millisecond`, `milliseconds`, `ms`
|
||||
- `second`, `seconds`, `ss`, `s`
|
||||
- `minute`, `minutes`, `mi`, `n`
|
||||
- `hour`, `hours`, `hh`, `h`
|
||||
- `day`, `days`, `dd`, `d`
|
||||
- `week`, `weeks`, `wk`, `ww`
|
||||
- `month`, `months`, `mm`, `m`
|
||||
- `quarter`, `quarters`, `qq`, `q`
|
||||
- `year`, `years`, `yyyy`, `yy`
|
||||
|
||||
- `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
|
||||
|
||||
@ -1763,16 +1764,17 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_
|
||||
- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
|
||||
Possible values:
|
||||
|
||||
- `microsecond` `microseconds` `us` `u`
|
||||
- `millisecond` `milliseconds` `ms`
|
||||
- `second` `seconds` `ss` `s`
|
||||
- `minute` `minutes` `mi` `n`
|
||||
- `hour` `hours` `hh` `h`
|
||||
- `day` `days` `dd` `d`
|
||||
- `week` `weeks` `wk` `ww`
|
||||
- `month` `months` `mm` `m`
|
||||
- `quarter` `quarters` `qq` `q`
|
||||
- `year` `years` `yyyy` `yy`
|
||||
- `nanosecond`, `nanoseconds`, `ns`
|
||||
- `microsecond`, `microseconds`, `us`, `u`
|
||||
- `millisecond`, `milliseconds`, `ms`
|
||||
- `second`, `seconds`, `ss`, `s`
|
||||
- `minute`, `minutes`, `mi`, `n`
|
||||
- `hour`, `hours`, `hh`, `h`
|
||||
- `day`, `days`, `dd`, `d`
|
||||
- `week`, `weeks`, `wk`, `ww`
|
||||
- `month`, `months`, `mm`, `m`
|
||||
- `quarter`, `quarters`, `qq`, `q`
|
||||
- `year`, `years`, `yyyy`, `yy`
|
||||
|
||||
- `startdate` — The first time value to subtract (the subtrahend). [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
|
||||
|
||||
|
@ -56,7 +56,9 @@ Entries for finished mutations are not deleted right away (the number of preserv
|
||||
|
||||
For non-replicated tables, all `ALTER` queries are performed synchronously. For replicated tables, the query just adds instructions for the appropriate actions to `ZooKeeper`, and the actions themselves are performed as soon as possible. However, the query can wait for these actions to be completed on all the replicas.
|
||||
|
||||
For all `ALTER` queries, you can use the [alter_sync](/docs/en/operations/settings/settings.md/#alter-sync) setting to set up waiting.
|
||||
For `ALTER` queries that creates mutations (e.g.: including, but not limited to `UPDATE`, `DELETE`, `MATERIALIZE INDEX`, `MATERIALIZE PROJECTION`, `MATERIALIZE COLUMN`, `APPLY DELETED MASK`, `CLEAR STATISTIC`, `MATERIALIZE STATISTIC`) the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting.
|
||||
|
||||
For other `ALTER` queries which only modify the metadata, you can use the [alter_sync](/docs/en/operations/settings/settings.md/#alter-sync) setting to set up waiting.
|
||||
|
||||
You can specify how long (in seconds) to wait for inactive replicas to execute all `ALTER` queries with the [replication_wait_for_inactive_replica_timeout](/docs/en/operations/settings/settings.md/#replication-wait-for-inactive-replica-timeout) setting.
|
||||
|
||||
@ -64,8 +66,6 @@ You can specify how long (in seconds) to wait for inactive replicas to execute a
|
||||
For all `ALTER` queries, if `alter_sync = 2` and some replicas are not active for more than the time, specified in the `replication_wait_for_inactive_replica_timeout` setting, then an exception `UNFINISHED` is thrown.
|
||||
:::
|
||||
|
||||
For `ALTER TABLE ... UPDATE|DELETE|MATERIALIZE INDEX|MATERIALIZE PROJECTION|MATERIALIZE COLUMN` queries the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting.
|
||||
|
||||
## Related content
|
||||
|
||||
- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse)
|
||||
|
@ -64,6 +64,14 @@ RELOAD FUNCTIONS [ON CLUSTER cluster_name]
|
||||
RELOAD FUNCTION [ON CLUSTER cluster_name] function_name
|
||||
```
|
||||
|
||||
## RELOAD ASYNCHRONOUS METRICS
|
||||
|
||||
Re-calculates all [asynchronous metrics](../../operations/system-tables/asynchronous_metrics.md). Since asynchronous metrics are periodically updated based on setting [asynchronous_metrics_update_period_s](../../operations/server-configuration-parameters/settings.md), updating them manually using this statement is typically not necessary.
|
||||
|
||||
```sql
|
||||
RELOAD ASYNCHRONOUS METRICS [ON CLUSTER cluster_name]
|
||||
```
|
||||
|
||||
## DROP DNS CACHE
|
||||
|
||||
Clears ClickHouse’s internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries).
|
||||
|
@ -476,7 +476,7 @@ FROM
|
||||
|
||||
- `r1` - количество уникальных посетителей за 2020-01-01 (`cond1`).
|
||||
- `r2` - количество уникальных посетителей в период между 2020-01-01 и 2020-01-02 (`cond1` и `cond2`).
|
||||
- `r3` - количество уникальных посетителей в период между 2020-01-01 и 2020-01-03 (`cond1` и `cond3`).
|
||||
- `r3` - количество уникальных посетителей в период за 2020-01-01 и 2020-01-03 (`cond1` и `cond3`).
|
||||
|
||||
## uniqUpTo(N)(x) {#uniquptonx}
|
||||
|
||||
|
@ -627,7 +627,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
|
||||
|
||||
## age
|
||||
|
||||
Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду.
|
||||
Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 наносекунду.
|
||||
Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`.
|
||||
|
||||
**Синтаксис**
|
||||
@ -641,6 +641,7 @@ age('unit', startdate, enddate, [timezone])
|
||||
- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
|
||||
Возможные значения:
|
||||
|
||||
- `nanosecond` (возможные сокращения: `ns`)
|
||||
- `microsecond` (возможные сокращения: `us`, `u`)
|
||||
- `millisecond` (возможные сокращения: `ms`)
|
||||
- `second` (возможные сокращения: `ss`, `s`)
|
||||
@ -716,6 +717,7 @@ date_diff('unit', startdate, enddate, [timezone])
|
||||
- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
|
||||
Возможные значения:
|
||||
|
||||
- `nanosecond` (возможные сокращения: `ns`)
|
||||
- `microsecond` (возможные сокращения: `us`, `u`)
|
||||
- `millisecond` (возможные сокращения: `ms`)
|
||||
- `second` (возможные сокращения: `ss`, `s`)
|
||||
|
@ -472,7 +472,7 @@ FROM
|
||||
|
||||
- `r1`-2020-01-01期间访问该网站的独立访问者数量( `cond1` 条件)。
|
||||
- `r2`-在2020-01-01和2020-01-02之间的特定时间段内访问该网站的唯一访问者的数量 (`cond1` 和 `cond2` 条件)。
|
||||
- `r3`-在2020-01-01和2020-01-03之间的特定时间段内访问该网站的唯一访问者的数量 (`cond1` 和 `cond3` 条件)。
|
||||
- `r3`-在2020-01-01和2020-01-03 网站的独立访客数量 (`cond1` 和 `cond3` 条件)。
|
||||
|
||||
## uniqUpTo(N)(x) {#uniquptonx}
|
||||
|
||||
|
@ -643,6 +643,7 @@ date_diff('unit', startdate, enddate, [timezone])
|
||||
- `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。
|
||||
可能的值:
|
||||
|
||||
- `nanosecond`
|
||||
- `microsecond`
|
||||
- `millisecond`
|
||||
- `second`
|
||||
|
@ -22,7 +22,7 @@ void protectMemoryRegion(void * addr, size_t len, int prot)
|
||||
}
|
||||
#endif
|
||||
|
||||
size_t byte_size(size_t num_elements, size_t element_size)
|
||||
ALWAYS_INLINE size_t byte_size(size_t num_elements, size_t element_size)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_mul_overflow(num_elements, element_size, &amount))
|
||||
@ -30,7 +30,7 @@ size_t byte_size(size_t num_elements, size_t element_size)
|
||||
return amount;
|
||||
}
|
||||
|
||||
size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right)
|
||||
ALWAYS_INLINE size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_add_overflow(byte_size(num_elements, element_size), pad_left + pad_right, &amount))
|
||||
|
@ -2219,7 +2219,7 @@ uint64_t Changelog::getStartIndex() const
|
||||
LogEntryPtr Changelog::getLastEntry() const
|
||||
{
|
||||
/// This entry treaded in special way by NuRaft
|
||||
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(sizeof(uint64_t)));
|
||||
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(0));
|
||||
|
||||
auto entry = entry_storage.getEntry(max_log_id);
|
||||
if (entry == nullptr)
|
||||
|
@ -16,7 +16,7 @@ ptr<log_entry> makeClone(const ptr<log_entry> & entry)
|
||||
InMemoryLogStore::InMemoryLogStore()
|
||||
: start_idx(1)
|
||||
{
|
||||
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(sizeof(uint64_t));
|
||||
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(0);
|
||||
logs[0] = nuraft::cs_new<nuraft::log_entry>(0, buf);
|
||||
}
|
||||
|
||||
|
@ -109,7 +109,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
}
|
||||
else
|
||||
{
|
||||
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, LoadingStrictnessLevel::ATTACH);
|
||||
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, mode);
|
||||
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
}
|
||||
}
|
||||
|
@ -153,7 +153,6 @@ public:
|
||||
return dir_path / entry->path().filename();
|
||||
}
|
||||
|
||||
|
||||
String name() const override { return entry->path().filename(); }
|
||||
|
||||
private:
|
||||
|
@ -22,8 +22,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr auto microsecond_multiplier = 1000000;
|
||||
static constexpr auto millisecond_multiplier = 1000;
|
||||
static constexpr auto millisecond_multiplier = 1'000;
|
||||
static constexpr auto microsecond_multiplier = 1'000'000;
|
||||
static constexpr auto nanosecond_multiplier = 1'000'000'000;
|
||||
|
||||
static constexpr FormatSettings::DateTimeOverflowBehavior default_date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore;
|
||||
|
||||
@ -1903,9 +1904,10 @@ struct ToRelativeSubsecondNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeSubsecondNumImpl";
|
||||
|
||||
static Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &)
|
||||
static Int64 execute(const DateTime64 & t, const DateTime64::NativeType scale, const DateLUTImpl &)
|
||||
{
|
||||
static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000);
|
||||
static_assert(
|
||||
scale_multiplier == millisecond_multiplier || scale_multiplier == microsecond_multiplier || scale_multiplier == nanosecond_multiplier);
|
||||
if (scale == scale_multiplier)
|
||||
return t.value;
|
||||
if (scale > scale_multiplier)
|
||||
@ -2031,13 +2033,14 @@ struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeCompon
|
||||
{
|
||||
UInt16 millisecond;
|
||||
UInt16 microsecond;
|
||||
UInt16 nanosecond;
|
||||
};
|
||||
|
||||
struct ToDateTimeComponentsImpl
|
||||
{
|
||||
static constexpr auto name = "toDateTimeComponents";
|
||||
|
||||
static DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone)
|
||||
static DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, const DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone)
|
||||
{
|
||||
auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
|
||||
|
||||
@ -2046,28 +2049,33 @@ struct ToDateTimeComponentsImpl
|
||||
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);
|
||||
|
||||
constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier;
|
||||
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
|
||||
UInt16 microsecond = static_cast<UInt16>(fractional % divider);
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond};
|
||||
// Normalize the dividers between microseconds and nanoseconds w.r.t. the scale.
|
||||
Int64 microsecond_divider = (millisecond_multiplier * scale_multiplier) / microsecond_multiplier;
|
||||
Int64 nanosecond_divider = scale_multiplier / microsecond_multiplier;
|
||||
|
||||
// Protect against division by zero for smaller scale multipliers.
|
||||
microsecond_divider = (microsecond_divider ? microsecond_divider : 1);
|
||||
nanosecond_divider = (nanosecond_divider ? nanosecond_divider : 1);
|
||||
|
||||
const Int64 & fractional = components.fractional;
|
||||
UInt16 millisecond = static_cast<UInt16>(fractional / microsecond_divider);
|
||||
UInt16 microsecond = static_cast<UInt16>((fractional % microsecond_divider) / nanosecond_divider);
|
||||
UInt16 nanosecond = static_cast<UInt16>(fractional % nanosecond_divider);
|
||||
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond, nanosecond};
|
||||
}
|
||||
static DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast<DateLUTImpl::Time>(t)), 0, 0};
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast<DateLUTImpl::Time>(t)), 0, 0, 0};
|
||||
}
|
||||
static DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0};
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0, 0};
|
||||
}
|
||||
static DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0};
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0, 0};
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
|
@ -177,10 +177,10 @@ public:
|
||||
DateTimeComponentsWithFractionalPart a_comp;
|
||||
DateTimeComponentsWithFractionalPart b_comp;
|
||||
Int64 adjust_value;
|
||||
auto x_microseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
auto y_microseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
auto x_nanoseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<nanosecond_multiplier>>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
auto y_nanoseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<nanosecond_multiplier>>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
|
||||
if (x_microseconds <= y_microseconds)
|
||||
if (x_nanoseconds <= y_nanoseconds)
|
||||
{
|
||||
a_comp = TransformDateTime64<ToDateTimeComponentsImpl>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
b_comp = TransformDateTime64<ToDateTimeComponentsImpl>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
@ -193,7 +193,6 @@ public:
|
||||
adjust_value = 1;
|
||||
}
|
||||
|
||||
|
||||
if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeYearNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.date.month > b_comp.date.month)
|
||||
@ -202,7 +201,8 @@ public:
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeQuarterNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -215,7 +215,8 @@ public:
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeMonthNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -225,7 +226,8 @@ public:
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeWeekNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -237,7 +239,8 @@ public:
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeDayNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -246,7 +249,8 @@ public:
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeHourNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -254,25 +258,34 @@ public:
|
||||
if ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeMinuteNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSecondNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSubsecondNumImpl<1000>>>)
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSubsecondNumImpl<millisecond_multiplier>>>)
|
||||
{
|
||||
if (a_comp.microsecond > b_comp.microsecond)
|
||||
if ((a_comp.microsecond > b_comp.microsecond)
|
||||
|| ((a_comp.microsecond == b_comp.microsecond) && (a_comp.nanosecond > b_comp.nanosecond)))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSubsecondNumImpl<microsecond_multiplier>>>)
|
||||
{
|
||||
if (a_comp.nanosecond > b_comp.nanosecond)
|
||||
res += adjust_value;
|
||||
}
|
||||
return res;
|
||||
@ -401,6 +414,8 @@ public:
|
||||
impl.template dispatchForColumns<ToRelativeSubsecondNumImpl<millisecond_multiplier>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else if (unit == "microsecond" || unit == "microseconds" || unit == "us" || unit == "u")
|
||||
impl.template dispatchForColumns<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else if (unit == "nanosecond" || unit == "nanoseconds" || unit == "ns")
|
||||
impl.template dispatchForColumns<ToRelativeSubsecondNumImpl<nanosecond_multiplier>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Function {} does not support '{}' unit", getName(), unit);
|
||||
|
@ -996,6 +996,14 @@ private:
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No object created and no exception raised for {}", type_name);
|
||||
|
||||
if (!info->object && new_object)
|
||||
{
|
||||
/// If we loaded the object for the first time then we should set `invalidate_query_response` to the current value.
|
||||
/// Otherwise we will immediately try to reload the object again despite the fact that it was just loaded.
|
||||
bool is_modified = new_object->isModified();
|
||||
LOG_TRACE(log, "Object '{}' was{} modified", name, (is_modified ? "" : " not"));
|
||||
}
|
||||
|
||||
/// Saving the result of the loading.
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
|
@ -2298,7 +2298,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
&& !settings.allow_experimental_query_deduplication
|
||||
&& !settings.empty_result_for_aggregation_by_empty_set
|
||||
&& storage
|
||||
&& storage->supportsTrivialCountOptimization()
|
||||
&& storage->supportsTrivialCountOptimization(storage_snapshot, getContext())
|
||||
&& query_info.filter_asts.empty()
|
||||
&& query_analyzer->hasAggregation()
|
||||
&& (query_analyzer->aggregates().size() == 1)
|
||||
|
@ -225,7 +225,8 @@ bool applyTrivialCountIfPossible(
|
||||
return false;
|
||||
|
||||
const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
|
||||
if (!storage->supportsTrivialCountOptimization())
|
||||
if (!storage->supportsTrivialCountOptimization(
|
||||
table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(), query_context))
|
||||
return false;
|
||||
|
||||
auto storage_id = storage->getStorageID();
|
||||
@ -262,9 +263,6 @@ bool applyTrivialCountIfPossible(
|
||||
if (main_query_node.hasGroupBy() || main_query_node.hasPrewhere() || main_query_node.hasWhere())
|
||||
return false;
|
||||
|
||||
if (storage->hasLightweightDeletedMask())
|
||||
return false;
|
||||
|
||||
if (settings.allow_experimental_query_deduplication
|
||||
|| settings.empty_result_for_aggregation_by_empty_set)
|
||||
return false;
|
||||
|
@ -92,7 +92,7 @@ public:
|
||||
|
||||
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
protected:
|
||||
friend class HDFSSource;
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
|
||||
|
@ -265,7 +265,10 @@ public:
|
||||
|
||||
/// Return true if the trivial count query could be optimized without reading the data at all
|
||||
/// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method.
|
||||
virtual bool supportsTrivialCountOptimization() const { return false; }
|
||||
virtual bool supportsTrivialCountOptimization(const StorageSnapshotPtr & /*storage_snapshot*/, ContextPtr /*query_context*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
private:
|
||||
StorageID storage_id;
|
||||
|
@ -8221,6 +8221,11 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart
|
||||
DB::updateObjectColumns(object_columns, columns, part->getColumns());
|
||||
}
|
||||
|
||||
bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const
|
||||
{
|
||||
return !hasLightweightDeletedMask();
|
||||
}
|
||||
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const
|
||||
{
|
||||
auto snapshot_data = std::make_unique<SnapshotData>();
|
||||
|
@ -440,7 +440,7 @@ public:
|
||||
|
||||
bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return !hasLightweightDeletedMask(); }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override;
|
||||
|
||||
/// Snapshot for MergeTree contains the current set of data parts
|
||||
/// at the moment of the start of query.
|
||||
|
@ -93,7 +93,7 @@ public:
|
||||
bool supportsDelete() const override { return true; }
|
||||
|
||||
/// To turn on the optimization optimize_trivial_approximate_count_query=1 should be set for a query.
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
||||
|
||||
|
@ -100,7 +100,7 @@ public:
|
||||
|
||||
bool supportsSubsetOfColumns(const ContextPtr & context) const;
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
bool prefersLargeBlocks() const override;
|
||||
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
void updateBeforeRead(const ContextPtr & /*context*/) override {}
|
||||
|
@ -134,7 +134,7 @@ public:
|
||||
const ContextPtr & context,
|
||||
size_t & total_bytes_to_read);
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
protected:
|
||||
friend class StorageFileSource;
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
|
||||
|
@ -85,7 +85,7 @@ public:
|
||||
|
||||
const Names & getKeyNames() const { return key_names; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
|
||||
void drop() override { nested_storage->drop(); }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return false; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return false; }
|
||||
|
||||
IndexSizeByName getSecondaryIndexSizes() const override
|
||||
{
|
||||
|
@ -1675,9 +1675,9 @@ std::tuple<bool /* is_regexp */, ASTPtr> StorageMerge::evaluateDatabaseName(cons
|
||||
return {false, ast};
|
||||
}
|
||||
|
||||
bool StorageMerge::supportsTrivialCountOptimization() const
|
||||
bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr ctx) const
|
||||
{
|
||||
return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(); }) == nullptr;
|
||||
return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(storage_snapshot, ctx); }) == nullptr;
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageMerge::totalRows(const Settings & settings) const
|
||||
|
@ -76,7 +76,7 @@ public:
|
||||
/// Evaluate database name or regexp for StorageMerge and TableFunction merge
|
||||
static std::tuple<bool /* is_regexp */, ASTPtr> evaluateDatabaseName(const ASTPtr & node, ContextPtr context);
|
||||
|
||||
bool supportsTrivialCountOptimization() const override;
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
||||
std::optional<UInt64> totalBytes(const Settings & settings) const override;
|
||||
|
@ -352,7 +352,7 @@ public:
|
||||
|
||||
using KeysWithInfo = StorageS3Source::KeysWithInfo;
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
protected:
|
||||
virtual Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context);
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
protected:
|
||||
void updateConfigurationIfChanged(ContextPtr local_context);
|
||||
|
@ -247,6 +247,8 @@ void StorageSetOrJoinBase::restore()
|
||||
static const char * file_suffix = ".bin";
|
||||
static const auto file_suffix_size = strlen(".bin");
|
||||
|
||||
using FilePriority = std::pair<UInt64, String>;
|
||||
std::priority_queue<FilePriority, std::vector<FilePriority>, std::greater<>> backup_files;
|
||||
for (auto dir_it{disk->iterateDirectory(path)}; dir_it->isValid(); dir_it->next())
|
||||
{
|
||||
const auto & name = dir_it->name();
|
||||
@ -261,9 +263,18 @@ void StorageSetOrJoinBase::restore()
|
||||
if (file_num > increment)
|
||||
increment = file_num;
|
||||
|
||||
restoreFromFile(dir_it->path());
|
||||
backup_files.push({file_num, file_path});
|
||||
}
|
||||
}
|
||||
|
||||
/// Restore in the same order as blocks were written
|
||||
/// It may be important for storage Join, user expect to get the first row (unless `join_any_take_last_row` setting is set)
|
||||
/// but after restart we may have different order of blocks in memory.
|
||||
while (!backup_files.empty())
|
||||
{
|
||||
restoreFromFile(backup_files.top().second);
|
||||
backup_files.pop();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -127,7 +127,7 @@ protected:
|
||||
|
||||
bool parallelizeOutputAfterReading(ContextPtr context) const override;
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromDataImpl(
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
|
||||
|
||||
private:
|
||||
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
|
||||
|
@ -7,6 +7,13 @@ The `gh` CLI preferred over the PyGithub to have an easy way to rollback bad
|
||||
release in command line by simple execution giving rollback commands
|
||||
|
||||
On another hand, PyGithub is used for convenient getting commit's status from API
|
||||
|
||||
To run this script on a freshly installed Ubuntu 22.04 system, it is enough to do the following commands:
|
||||
|
||||
sudo apt install pip
|
||||
pip install requests boto3 github PyGithub
|
||||
sudo snap install gh
|
||||
gh auth login
|
||||
"""
|
||||
|
||||
|
||||
|
@ -69,6 +69,8 @@ TEST_FILE_EXTENSIONS = [".sql", ".sql.j2", ".sh", ".py", ".expect"]
|
||||
|
||||
VERSION_PATTERN = r"^((\d+\.)?(\d+\.)?(\d+\.)?\d+)$"
|
||||
|
||||
TEST_MAX_RUN_TIME_IN_SECONDS = 120
|
||||
|
||||
|
||||
class SharedEngineReplacer:
|
||||
ENGINES_NON_REPLICATED_REGEXP = r"[ =]((Collapsing|VersionedCollapsing|Summing|Replacing|Aggregating|)MergeTree\(?\)?)"
|
||||
@ -681,7 +683,9 @@ class FailureReason(enum.Enum):
|
||||
STDERR = "having stderror: "
|
||||
EXCEPTION = "having exception in stdout: "
|
||||
RESULT_DIFF = "result differs with reference: "
|
||||
TOO_LONG = "Test runs too long (> 60s). Make it faster."
|
||||
TOO_LONG = (
|
||||
f"Test runs too long (> {TEST_MAX_RUN_TIME_IN_SECONDS}s). Make it faster."
|
||||
)
|
||||
INTERNAL_QUERY_FAIL = "Internal query (CREATE/DROP DATABASE) failed:"
|
||||
|
||||
# SKIPPED reasons
|
||||
@ -1420,7 +1424,7 @@ class TestCase:
|
||||
|
||||
if (
|
||||
self.testcase_args.test_runs > 1
|
||||
and total_time > 120
|
||||
and total_time > TEST_MAX_RUN_TIME_IN_SECONDS
|
||||
and "long" not in self.tags
|
||||
):
|
||||
if debug_log:
|
||||
|
@ -122,6 +122,9 @@ def test_dependent_tables(started_cluster):
|
||||
)
|
||||
query("create table system.join (n int, m int) engine=Join(any, left, n)")
|
||||
query("insert into system.join values (1, 1)")
|
||||
for i in range(2, 100):
|
||||
query(f"insert into system.join values (1, {i})")
|
||||
|
||||
query(
|
||||
"create table src (n int, m default joinGet('system.join', 'm', 1::int),"
|
||||
"t default dictGetOrNull('a.d', 'm', toUInt64(3)),"
|
||||
|
@ -93,10 +93,18 @@ def test_join_s3(cluster):
|
||||
"CREATE TABLE testLocalJoin(`id` UInt64, `val` String) ENGINE = Join(ANY, LEFT, id)"
|
||||
)
|
||||
node.query(
|
||||
"CREATE TABLE testS3Join(`id` UInt64, `val` String) ENGINE = Join(ANY, LEFT, id) SETTINGS disk='s3'"
|
||||
"CREATE TABLE testS3Join(`id` UInt64, `val` String) ENGINE = Join(ANY, LEFT, id) SETTINGS disk='s3', join_any_take_last_row = 1"
|
||||
)
|
||||
|
||||
node.query("INSERT INTO testLocalJoin VALUES (1, 'a')")
|
||||
for i in range(1, 10):
|
||||
c = chr(ord("a") + i)
|
||||
node.query(f"INSERT INTO testLocalJoin VALUES (1, '{c}')")
|
||||
|
||||
# because of `join_any_take_last_row = 1` we expect the last row with 'a' value
|
||||
for i in range(1, 10):
|
||||
c = chr(ord("a") + i)
|
||||
node.query(f"INSERT INTO testS3Join VALUES (1, '{c}')")
|
||||
node.query("INSERT INTO testS3Join VALUES (1, 'a')")
|
||||
|
||||
assert (
|
||||
@ -105,7 +113,7 @@ def test_join_s3(cluster):
|
||||
)
|
||||
== "\t\na\ta\n\t\n"
|
||||
)
|
||||
assert_objects_count(cluster, 1)
|
||||
assert_objects_count(cluster, 10)
|
||||
|
||||
node.query("INSERT INTO testLocalJoin VALUES (2, 'b')")
|
||||
node.query("INSERT INTO testS3Join VALUES (2, 'b')")
|
||||
@ -116,7 +124,7 @@ def test_join_s3(cluster):
|
||||
)
|
||||
== "\t\na\ta\nb\tb\n"
|
||||
)
|
||||
assert_objects_count(cluster, 2)
|
||||
assert_objects_count(cluster, 11)
|
||||
|
||||
node.restart_clickhouse()
|
||||
assert (
|
||||
|
3
tests/performance/function_tokens.xml
Normal file
3
tests/performance/function_tokens.xml
Normal file
@ -0,0 +1,3 @@
|
||||
<test>
|
||||
<query>with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByChar(' ', materialize(s)) as w from numbers(1000000)</query>
|
||||
</test>
|
@ -203,7 +203,7 @@ select
|
||||
with 0.16 as threshold
|
||||
select
|
||||
'noisy Trace messages',
|
||||
greatest(coalesce(((select message_format_string, count() from logs where level = 'Trace' and message_format_string not in ('Access granted: {}{}', '{} -> {}', 'Query {} to stage {}{}')
|
||||
greatest(coalesce(((select message_format_string, count() from logs where level = 'Trace' and message_format_string not in ('Access granted: {}{}', '{} -> {}', 'Query {} to stage {}{}', 'Query {} from stage {} to stage {}{}')
|
||||
group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r,
|
||||
r <= threshold ? '' : top_message.1;
|
||||
|
||||
|
@ -8,3 +8,4 @@
|
||||
-63072000
|
||||
-63072000000
|
||||
-63072000000000
|
||||
-63072000000000000
|
||||
|
@ -8,3 +8,4 @@ SELECT dateDiff('minutes', toDateTime('2017-12-31', 'UTC'), toDateTime('2016-01-
|
||||
SELECT dateDiff('seconds', toDateTime('2017-12-31', 'UTC'), toDateTime('2016-01-01', 'UTC'));
|
||||
SELECT dateDiff('milliseconds', toDateTime('2017-12-31', 'UTC'), toDateTime('2016-01-01', 'UTC'));
|
||||
SELECT dateDiff('microseconds', toDateTime('2017-12-31', 'UTC'), toDateTime('2016-01-01', 'UTC'));
|
||||
SELECT dateDiff('nanoseconds', toDateTime('2017-12-31', 'UTC'), toDateTime('2016-01-01', 'UTC'));
|
||||
|
@ -1,10 +1,14 @@
|
||||
-- { echo }
|
||||
|
||||
-- DateTime64 vs DateTime64 with fractional part
|
||||
SELECT age('nanosecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
5100200000
|
||||
SELECT age('nanosecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
5100199999
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
5100200
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
5100200
|
||||
5100199
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC'));
|
||||
5100
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC'));
|
||||
|
@ -1,6 +1,9 @@
|
||||
-- { echo }
|
||||
|
||||
-- DateTime64 vs DateTime64 with fractional part
|
||||
SELECT age('nanosecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
SELECT age('nanosecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user