mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge branch 'master' into fix-assertions
This commit is contained in:
commit
d4a921bd7c
@ -170,18 +170,6 @@ else ()
|
||||
set(NO_WHOLE_ARCHIVE --no-whole-archive)
|
||||
endif ()
|
||||
|
||||
option(ENABLE_CURL_BUILD "Enable curl, azure, sentry build on by default except MacOS." ON)
|
||||
if (OS_DARWIN)
|
||||
# Disable the curl, azure, senry build on MacOS
|
||||
set (ENABLE_CURL_BUILD OFF)
|
||||
endif ()
|
||||
|
||||
option(ENABLE_ISAL_LIBRARY "Enable ISA-L library ON by default except on aarch64." ON)
|
||||
if (ARCH_AARCH64)
|
||||
# Disable ISA-L libray on aarch64.
|
||||
set (ENABLE_ISAL_LIBRARY OFF)
|
||||
endif ()
|
||||
|
||||
if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE")
|
||||
# Can be lld or ld-lld or lld-13 or /path/to/lld.
|
||||
if (LINKER_NAME MATCHES "lld")
|
||||
@ -399,9 +387,9 @@ else()
|
||||
endif ()
|
||||
|
||||
option (ENABLE_GWP_ASAN "Enable Gwp-Asan" ON)
|
||||
# We use mmap for allocations more heavily in debug builds,
|
||||
# but GWP-ASan also wants to use mmap frequently,
|
||||
# and due to a large number of memory mappings,
|
||||
# We use mmap for allocations more heavily in debug builds,
|
||||
# but GWP-ASan also wants to use mmap frequently,
|
||||
# and due to a large number of memory mappings,
|
||||
# it does not work together well.
|
||||
if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG"))
|
||||
set(ENABLE_GWP_ASAN OFF)
|
||||
|
16
contrib/CMakeLists.txt
vendored
16
contrib/CMakeLists.txt
vendored
@ -141,20 +141,19 @@ add_contrib (libuv-cmake libuv)
|
||||
add_contrib (liburing-cmake liburing)
|
||||
add_contrib (amqpcpp-cmake AMQP-CPP) # requires: libuv
|
||||
add_contrib (cassandra-cmake cassandra) # requires: libuv
|
||||
|
||||
if (ENABLE_CURL_BUILD)
|
||||
if (NOT OS_DARWIN)
|
||||
add_contrib (curl-cmake curl)
|
||||
add_contrib (azure-cmake azure)
|
||||
add_contrib (sentry-native-cmake sentry-native) # requires: curl
|
||||
endif()
|
||||
|
||||
add_contrib (fmtlib-cmake fmtlib)
|
||||
add_contrib (krb5-cmake krb5)
|
||||
add_contrib (cyrus-sasl-cmake cyrus-sasl) # for krb5
|
||||
add_contrib (libgsasl-cmake libgsasl) # requires krb5
|
||||
add_contrib (librdkafka-cmake librdkafka) # requires: libgsasl
|
||||
add_contrib (nats-io-cmake nats-io)
|
||||
add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5
|
||||
add_contrib (isa-l-cmake isa-l)
|
||||
add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5, isa-l
|
||||
add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift/avro/arrow/libhdfs3
|
||||
add_contrib (cppkafka-cmake cppkafka)
|
||||
add_contrib (libpqxx-cmake libpqxx)
|
||||
@ -178,23 +177,14 @@ add_contrib (s2geometry-cmake s2geometry)
|
||||
add_contrib (c-ares-cmake c-ares)
|
||||
add_contrib (qpl-cmake qpl)
|
||||
add_contrib (morton-nd-cmake morton-nd)
|
||||
|
||||
if (ARCH_S390X)
|
||||
add_contrib(crc32-s390x-cmake crc32-s390x)
|
||||
endif()
|
||||
|
||||
add_contrib (annoy-cmake annoy)
|
||||
|
||||
add_contrib (xxHash-cmake xxHash)
|
||||
|
||||
add_contrib (google-benchmark-cmake google-benchmark)
|
||||
|
||||
add_contrib (ulid-c-cmake ulid-c)
|
||||
|
||||
if (ENABLE_ISAL_LIBRARY)
|
||||
add_contrib (isa-l-cmake isa-l)
|
||||
endif()
|
||||
|
||||
# Put all targets defined here and in subdirectories under "contrib/<immediate-subdir>" folders in GUI-based IDEs.
|
||||
# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear
|
||||
# in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually,
|
||||
|
@ -1,3 +1,7 @@
|
||||
if (NOT ARCH_AMD64)
|
||||
return()
|
||||
endif ()
|
||||
|
||||
set(ISAL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/isa-l")
|
||||
|
||||
# The YASM and NASM assembers are somewhat mutually compatible. ISAL specifically needs NASM. If only YASM is installed, then check_language(ASM_NASM)
|
||||
|
@ -172,7 +172,7 @@ if (TARGET OpenSSL::SSL)
|
||||
target_link_libraries(_hdfs3 PRIVATE OpenSSL::Crypto OpenSSL::SSL)
|
||||
endif()
|
||||
|
||||
if (ENABLE_ISAL_LIBRARY)
|
||||
if (TARGET ch_contrib::isal)
|
||||
target_link_libraries(_hdfs3 PRIVATE ch_contrib::isal)
|
||||
add_definitions(-DHADOOP_ISAL_LIBRARY)
|
||||
endif()
|
||||
|
@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
esac
|
||||
|
||||
ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release"
|
||||
ARG VERSION="23.4.1.1943"
|
||||
ARG VERSION="23.4.2.11"
|
||||
ARG PACKAGES="clickhouse-keeper"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
# lts / testing / prestable / etc
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
|
||||
ARG VERSION="23.4.1.1943"
|
||||
ARG VERSION="23.4.2.11"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
|
||||
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
|
||||
ARG VERSION="23.4.1.1943"
|
||||
ARG VERSION="23.4.2.11"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# set non-empty deb_location_url url to create a docker image
|
||||
|
20
docs/changelogs/v23.4.2.11-stable.md
Normal file
20
docs/changelogs/v23.4.2.11-stable.md
Normal file
@ -0,0 +1,20 @@
|
||||
---
|
||||
sidebar_position: 1
|
||||
sidebar_label: 2023
|
||||
---
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### ClickHouse release v23.4.2.11-stable (b6442320f9d) FIXME as compared to v23.4.1.1943-stable (3920eb987f7)
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in an official stable release)
|
||||
|
||||
* Revert "Fix GCS native copy ([#48981](https://github.com/ClickHouse/ClickHouse/issues/48981))" [#49194](https://github.com/ClickHouse/ClickHouse/pull/49194) ([Raúl Marín](https://github.com/Algunenano)).
|
||||
* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
|
||||
#### NOT FOR CHANGELOG / INSIGNIFICANT
|
||||
|
||||
* Implement status comment [#48468](https://github.com/ClickHouse/ClickHouse/pull/48468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Update curl to 8.0.1 (for CVEs) [#48765](https://github.com/ClickHouse/ClickHouse/pull/48765) ([Boris Kuschel](https://github.com/bkuschel)).
|
||||
* Fallback auth gh api [#49314](https://github.com/ClickHouse/ClickHouse/pull/49314) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
@ -50,7 +50,7 @@ struct AggregateFunctionSequenceMatchData final
|
||||
bool sorted = true;
|
||||
PODArrayWithStackMemory<TimestampEvents, 64> events_list;
|
||||
/// sequenceMatch conditions met at least once in events_list
|
||||
std::bitset<max_events> conditions_met;
|
||||
Events conditions_met;
|
||||
|
||||
void add(const Timestamp timestamp, const Events & events)
|
||||
{
|
||||
@ -101,6 +101,11 @@ struct AggregateFunctionSequenceMatchData final
|
||||
size_t size;
|
||||
readBinary(size, buf);
|
||||
|
||||
/// If we lose these flags, functionality is broken
|
||||
/// If we serialize/deserialize these flags, we have compatibility issues
|
||||
/// If we set these flags to 1, we have a minor performance penalty, which seems acceptable
|
||||
conditions_met.set();
|
||||
|
||||
events_list.clear();
|
||||
events_list.reserve(size);
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
@ -25,7 +25,7 @@ class FunctionDateTrunc : public IFunction
|
||||
public:
|
||||
static constexpr auto name = "dateTrunc";
|
||||
|
||||
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {}
|
||||
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) { }
|
||||
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); }
|
||||
|
||||
@ -39,51 +39,58 @@ public:
|
||||
{
|
||||
/// The first argument is a constant string with the name of datepart.
|
||||
|
||||
auto result_type_is_date = false;
|
||||
intermediate_type_is_date = false;
|
||||
String datepart_param;
|
||||
auto check_first_argument = [&] {
|
||||
auto check_first_argument = [&]
|
||||
{
|
||||
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!datepart_column)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: "
|
||||
"name of datepart", getName());
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument for function {} must be constant string: "
|
||||
"name of datepart",
|
||||
getName());
|
||||
|
||||
datepart_param = datepart_column->getValue<String>();
|
||||
if (datepart_param.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty",
|
||||
getName());
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName());
|
||||
|
||||
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
|
||||
|
||||
result_type_is_date = (datepart_kind == IntervalKind::Year)
|
||||
|| (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month)
|
||||
|| (datepart_kind == IntervalKind::Week);
|
||||
intermediate_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter)
|
||||
|| (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week);
|
||||
};
|
||||
|
||||
bool second_argument_is_date = false;
|
||||
auto check_second_argument = [&] {
|
||||
auto check_second_argument = [&]
|
||||
{
|
||||
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. "
|
||||
"Should be a date or a date with time", arguments[1].type->getName(), getName());
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 2nd argument of function {}. "
|
||||
"Should be a date or a date with time",
|
||||
arguments[1].type->getName(),
|
||||
getName());
|
||||
|
||||
second_argument_is_date = isDate(arguments[1].type);
|
||||
|
||||
if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour)
|
||||
|| (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second)))
|
||||
if (second_argument_is_date
|
||||
&& ((datepart_kind == IntervalKind::Hour) || (datepart_kind == IntervalKind::Minute)
|
||||
|| (datepart_kind == IntervalKind::Second)))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
|
||||
};
|
||||
|
||||
auto check_timezone_argument = [&] {
|
||||
auto check_timezone_argument = [&]
|
||||
{
|
||||
if (!WhichDataType(arguments[2].type).isString())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. "
|
||||
"This argument is optional and must be a constant string with timezone name",
|
||||
arguments[2].type->getName(), getName());
|
||||
|
||||
if (second_argument_is_date && result_type_is_date)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} with datepart '{}' "
|
||||
"is allowed only when the 2nd argument has the type DateTime",
|
||||
getName(), datepart_param);
|
||||
arguments[2].type->getName(),
|
||||
getName());
|
||||
};
|
||||
|
||||
if (arguments.size() == 2)
|
||||
@ -99,15 +106,14 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
|
||||
getName(), arguments.size());
|
||||
getName(),
|
||||
arguments.size());
|
||||
}
|
||||
|
||||
if (result_type_is_date)
|
||||
return std::make_shared<DataTypeDate>();
|
||||
else
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
@ -124,26 +130,40 @@ public:
|
||||
|
||||
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
|
||||
|
||||
ColumnPtr truncated_column;
|
||||
auto date_type = std::make_shared<DataTypeDate>();
|
||||
|
||||
if (arguments.size() == 2)
|
||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
||||
truncated_column = to_start_of_interval->build(temp_columns)
|
||||
->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count);
|
||||
else
|
||||
{
|
||||
temp_columns[2] = arguments[2];
|
||||
truncated_column = to_start_of_interval->build(temp_columns)
|
||||
->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count);
|
||||
}
|
||||
|
||||
temp_columns[2] = arguments[2];
|
||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
||||
if (!intermediate_type_is_date)
|
||||
return truncated_column;
|
||||
|
||||
ColumnsWithTypeAndName temp_truncated_column(1);
|
||||
temp_truncated_column[0] = {truncated_column, date_type, ""};
|
||||
|
||||
auto to_date_time_or_default = FunctionFactory::instance().get("toDateTime", context);
|
||||
return to_date_time_or_default->build(temp_truncated_column)->execute(temp_truncated_column, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
bool hasInformationAboutMonotonicity() const override { return true; }
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
|
||||
{
|
||||
return { .is_monotonic = true, .is_always_monotonic = true };
|
||||
return {.is_monotonic = true, .is_always_monotonic = true};
|
||||
}
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
|
||||
mutable bool intermediate_type_is_date = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -33,7 +33,7 @@ String getExceptionEntryWithFileName(const ReadBuffer & in)
|
||||
if (filename.empty())
|
||||
return "";
|
||||
|
||||
return fmt::format("; While reading from: {}", filename);
|
||||
return fmt::format(": While reading from: {}", filename);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -246,18 +246,18 @@ toUnixTimestamp
|
||||
1426415400
|
||||
1426415400
|
||||
date_trunc
|
||||
2019-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-10-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-10-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-31 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-02 00:00:00
|
||||
@ -270,18 +270,18 @@ date_trunc
|
||||
2019-12-31 20:11:22
|
||||
2020-01-01 12:11:22
|
||||
2020-01-02 05:11:22
|
||||
2019-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-10-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-12-30
|
||||
2019-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-10-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2019-12-31 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-02 00:00:00
|
||||
@ -294,8 +294,8 @@ date_trunc
|
||||
2019-12-31 20:11:22
|
||||
2020-01-01 12:11:22
|
||||
2020-01-02 05:11:22
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2020-01-01
|
||||
2019-12-30
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
2019-12-30 00:00:00
|
||||
2020-01-01 00:00:00
|
||||
|
@ -135,13 +135,13 @@ Code: 43
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'year\', N, \'Asia/Istanbul\')
|
||||
Code: 43
|
||||
"Date","2019-01-01"
|
||||
"Date","2019-01-01"
|
||||
"DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
|
||||
"DateTime('Asia/Istanbul')","2019-01-01 00:00:00"
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'month\', N, \'Asia/Istanbul\')
|
||||
Code: 43
|
||||
"Date","2019-09-01"
|
||||
"Date","2019-09-01"
|
||||
"DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
|
||||
"DateTime('Asia/Istanbul')","2019-09-01 00:00:00"
|
||||
------------------------------------------
|
||||
SELECT date_trunc(\'day\', N, \'Asia/Istanbul\')
|
||||
"DateTime('Asia/Istanbul')","2019-09-16 00:00:00"
|
||||
|
@ -0,0 +1,3 @@
|
||||
serialized state is not used 1
|
||||
serialized state is used 1
|
||||
via Distributed 1
|
@ -0,0 +1,36 @@
|
||||
DROP TABLE IF EXISTS 02713_seqt;
|
||||
DROP TABLE IF EXISTS 02713_seqt_distr;
|
||||
|
||||
SELECT
|
||||
'serialized state is not used', sequenceMatch('(?1)(?2)')(time, number_ = 1, number_ = 0) AS seq
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
number AS time,
|
||||
number % 2 AS number_
|
||||
FROM numbers_mt(100)
|
||||
);
|
||||
|
||||
|
||||
CREATE TABLE 02713_seqt
|
||||
ENGINE = MergeTree
|
||||
ORDER BY n AS
|
||||
SELECT
|
||||
sequenceMatchState('(?1)(?2)')(time, number_ = 1, number_ = 0) AS seq,
|
||||
1 AS n
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
number AS time,
|
||||
number % 2 AS number_
|
||||
FROM numbers_mt(100)
|
||||
);
|
||||
|
||||
|
||||
SELECT 'serialized state is used', sequenceMatchMerge('(?1)(?2)')(seq) AS seq
|
||||
FROM 02713_seqt;
|
||||
|
||||
|
||||
CREATE TABLE 02713_seqt_distr ( seq AggregateFunction(sequenceMatch('(?1)(?2)'), UInt64, UInt8, UInt8) , n UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), '02713_seqt');
|
||||
|
||||
SELECT 'via Distributed', sequenceMatchMerge('(?1)(?2)')(seq) AS seq FROM 02713_seqt_distr;
|
@ -1,3 +1,4 @@
|
||||
v23.4.2.11-stable 2023-05-02
|
||||
v23.4.1.1943-stable 2023-04-27
|
||||
v23.3.2.37-lts 2023-04-22
|
||||
v23.3.1.2823-lts 2023-03-31
|
||||
|
|
Loading…
Reference in New Issue
Block a user