Merge branch 'master' into read-cgroup-memory-usage-async-metrics

This commit is contained in:
Antonio Andelic 2024-07-26 15:55:42 +02:00
commit 91e0df33db
77 changed files with 589 additions and 423 deletions

View File

@ -2102,14 +2102,14 @@ Result:
└─────────────────┘ └─────────────────┘
``` ```
## filesystemFree ## filesystemUnreserved
Returns the total amount of the free space on the filesystem hosting the database persistence. See also `filesystemAvailable` Returns the total amount of the free space on the filesystem hosting the database persistence. (previously `filesystemFree`). See also [`filesystemAvailable`](#filesystemavailable).
**Syntax** **Syntax**
```sql ```sql
filesystemFree() filesystemUnreserved()
``` ```
**Returned value** **Returned value**
@ -2121,7 +2121,7 @@ filesystemFree()
Query: Query:
```sql ```sql
SELECT formatReadableSize(filesystemFree()) AS "Free space"; SELECT formatReadableSize(filesystemUnreserved()) AS "Free space";
``` ```
Result: Result:

View File

@ -21,7 +21,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2], name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2],
... ...
) ENGINE = engine ) ENGINE = engine
COMMENT 'comment for table' [COMMENT 'comment for table']
``` ```
Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine. Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine.
@ -626,11 +626,6 @@ SELECT * FROM base.t1;
You can add a comment to the table when you creating it. You can add a comment to the table when you creating it.
:::note
The comment clause is supported by all table engines except [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) and [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md).
:::
**Syntax** **Syntax**
``` sql ``` sql

View File

@ -16,6 +16,7 @@ Syntax:
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ... AS SELECT ...
[COMMENT 'comment']
``` ```
Normal views do not store any data. They just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause. Normal views do not store any data. They just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause.
@ -57,6 +58,7 @@ SELECT * FROM view(column1=value1, column2=value2 ...)
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ... AS SELECT ...
[COMMENT 'comment']
``` ```
:::tip :::tip
@ -161,6 +163,7 @@ RANDOMIZE FOR interval
DEPENDS ON [db.]name [, [db.]name [, ...]] DEPENDS ON [db.]name [, [db.]name [, ...]]
[TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY] [TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY]
AS SELECT ... AS SELECT ...
[COMMENT 'comment']
``` ```
where `interval` is a sequence of simple intervals: where `interval` is a sequence of simple intervals:
```sql ```sql
@ -267,7 +270,10 @@ This is an experimental feature that may change in backwards-incompatible ways i
::: :::
``` sql ``` sql
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] AS SELECT ... GROUP BY time_window_function CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE]
AS SELECT ...
GROUP BY time_window_function
[COMMENT 'comment']
``` ```
Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query. Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query.

View File

@ -195,7 +195,7 @@ bool SingleValueDataFixed<T>::isEqualTo(const IColumn & column, size_t index) co
template <typename T> template <typename T>
bool SingleValueDataFixed<T>::isEqualTo(const SingleValueDataFixed<T> & to) const bool SingleValueDataFixed<T>::isEqualTo(const SingleValueDataFixed<T> & to) const
{ {
return has() && to.value == value; return has() && to.has() && to.value == value;
} }
template <typename T> template <typename T>
@ -904,6 +904,7 @@ bool SingleValueDataNumeric<T>::isEqualTo(const DB::IColumn & column, size_t ind
template <typename T> template <typename T>
bool SingleValueDataNumeric<T>::isEqualTo(const DB::SingleValueDataBase & to) const bool SingleValueDataNumeric<T>::isEqualTo(const DB::SingleValueDataBase & to) const
{ {
/// to.has() is checked in memory.get().isEqualTo
auto const & other = assert_cast<const Self &>(to); auto const & other = assert_cast<const Self &>(to);
return memory.get().isEqualTo(other.memory.get()); return memory.get().isEqualTo(other.memory.get());
} }
@ -917,6 +918,7 @@ void SingleValueDataNumeric<T>::set(const DB::IColumn & column, size_t row_num,
template <typename T> template <typename T>
void SingleValueDataNumeric<T>::set(const DB::SingleValueDataBase & to, DB::Arena * arena) void SingleValueDataNumeric<T>::set(const DB::SingleValueDataBase & to, DB::Arena * arena)
{ {
/// to.has() is checked in memory.get().set
auto const & other = assert_cast<const Self &>(to); auto const & other = assert_cast<const Self &>(to);
return memory.get().set(other.memory.get(), arena); return memory.get().set(other.memory.get(), arena);
} }
@ -924,6 +926,7 @@ void SingleValueDataNumeric<T>::set(const DB::SingleValueDataBase & to, DB::Aren
template <typename T> template <typename T>
bool SingleValueDataNumeric<T>::setIfSmaller(const DB::SingleValueDataBase & to, DB::Arena * arena) bool SingleValueDataNumeric<T>::setIfSmaller(const DB::SingleValueDataBase & to, DB::Arena * arena)
{ {
/// to.has() is checked in memory.get().setIfSmaller
auto const & other = assert_cast<const Self &>(to); auto const & other = assert_cast<const Self &>(to);
return memory.get().setIfSmaller(other.memory.get(), arena); return memory.get().setIfSmaller(other.memory.get(), arena);
} }
@ -931,6 +934,7 @@ bool SingleValueDataNumeric<T>::setIfSmaller(const DB::SingleValueDataBase & to,
template <typename T> template <typename T>
bool SingleValueDataNumeric<T>::setIfGreater(const DB::SingleValueDataBase & to, DB::Arena * arena) bool SingleValueDataNumeric<T>::setIfGreater(const DB::SingleValueDataBase & to, DB::Arena * arena)
{ {
/// to.has() is checked in memory.get().setIfGreater
auto const & other = assert_cast<const Self &>(to); auto const & other = assert_cast<const Self &>(to);
return memory.get().setIfGreater(other.memory.get(), arena); return memory.get().setIfGreater(other.memory.get(), arena);
} }
@ -1191,7 +1195,7 @@ bool SingleValueDataString::isEqualTo(const DB::IColumn & column, size_t row_num
bool SingleValueDataString::isEqualTo(const SingleValueDataBase & other) const bool SingleValueDataString::isEqualTo(const SingleValueDataBase & other) const
{ {
auto const & to = assert_cast<const Self &>(other); auto const & to = assert_cast<const Self &>(other);
return has() && to.getStringRef() == getStringRef(); return has() && to.has() && to.getStringRef() == getStringRef();
} }
void SingleValueDataString::set(const IColumn & column, size_t row_num, Arena * arena) void SingleValueDataString::set(const IColumn & column, size_t row_num, Arena * arena)
@ -1291,7 +1295,7 @@ bool SingleValueDataGeneric::isEqualTo(const IColumn & column, size_t row_num) c
bool SingleValueDataGeneric::isEqualTo(const DB::SingleValueDataBase & other) const bool SingleValueDataGeneric::isEqualTo(const DB::SingleValueDataBase & other) const
{ {
auto const & to = assert_cast<const Self &>(other); auto const & to = assert_cast<const Self &>(other);
return has() && to.value == value; return has() && to.has() && to.value == value;
} }
void SingleValueDataGeneric::set(const IColumn & column, size_t row_num, Arena *) void SingleValueDataGeneric::set(const IColumn & column, size_t row_num, Arena *)

View File

@ -2241,6 +2241,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
ASTPtr parsed_query; ASTPtr parsed_query;
std::unique_ptr<Exception> current_exception; std::unique_ptr<Exception> current_exception;
size_t retries_count = 0;
while (true) while (true)
{ {
auto stage = analyzeMultiQueryText(this_query_begin, this_query_end, all_queries_end, auto stage = analyzeMultiQueryText(this_query_begin, this_query_end, all_queries_end,
@ -2321,7 +2323,12 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
// Check whether the error (or its absence) matches the test hints // Check whether the error (or its absence) matches the test hints
// (or their absence). // (or their absence).
bool error_matches_hint = true; bool error_matches_hint = true;
if (have_error) bool need_retry = test_hint.needRetry(server_exception, &retries_count);
if (need_retry)
{
std::this_thread::sleep_for(std::chrono::seconds(1));
}
else if (have_error)
{ {
if (test_hint.hasServerErrors()) if (test_hint.hasServerErrors())
{ {
@ -2415,7 +2422,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
if (have_error && !ignore_error) if (have_error && !ignore_error)
return is_interactive; return is_interactive;
this_query_begin = this_query_end; if (!need_retry)
this_query_begin = this_query_end;
break; break;
} }
} }

View File

@ -10,6 +10,7 @@
namespace DB::ErrorCodes namespace DB::ErrorCodes
{ {
extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_TEXT;
extern const int OK;
} }
namespace DB namespace DB
@ -62,9 +63,28 @@ bool TestHint::hasExpectedServerError(int error)
return std::find(server_errors.begin(), server_errors.end(), error) != server_errors.end(); return std::find(server_errors.begin(), server_errors.end(), error) != server_errors.end();
} }
bool TestHint::needRetry(const std::unique_ptr<Exception> & server_exception, size_t * retries_counter)
{
chassert(retries_counter);
if (max_retries <= *retries_counter)
return false;
++*retries_counter;
int error = ErrorCodes::OK;
if (server_exception)
error = server_exception->code();
if (retry_until)
return !hasExpectedServerError(error); /// retry until we get the expected error
else
return hasExpectedServerError(error); /// retry while we have the expected error
}
void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint)
{ {
std::unordered_set<std::string_view> commands{"echo", "echoOn", "echoOff"}; std::unordered_set<std::string_view> commands{"echo", "echoOn", "echoOff", "retry"};
std::unordered_set<std::string_view> command_errors{ std::unordered_set<std::string_view> command_errors{
"serverError", "serverError",
@ -73,6 +93,9 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint)
for (Token token = comment_lexer.nextToken(); !token.isEnd(); token = comment_lexer.nextToken()) for (Token token = comment_lexer.nextToken(); !token.isEnd(); token = comment_lexer.nextToken())
{ {
if (token.type == TokenType::Whitespace)
continue;
String item = String(token.begin, token.end); String item = String(token.begin, token.end);
if (token.type == TokenType::BareWord && commands.contains(item)) if (token.type == TokenType::BareWord && commands.contains(item))
{ {
@ -82,6 +105,30 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint)
echo.emplace(true); echo.emplace(true);
if (item == "echoOff") if (item == "echoOff")
echo.emplace(false); echo.emplace(false);
if (item == "retry")
{
token = comment_lexer.nextToken();
while (token.type == TokenType::Whitespace)
token = comment_lexer.nextToken();
if (token.type != TokenType::Number)
throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Could not parse the number of retries: {}",
std::string_view(token.begin, token.end));
max_retries = std::stoul(std::string(token.begin, token.end));
token = comment_lexer.nextToken();
while (token.type == TokenType::Whitespace)
token = comment_lexer.nextToken();
if (token.type != TokenType::BareWord ||
(std::string_view(token.begin, token.end) != "until" &&
std::string_view(token.begin, token.end) != "while"))
throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected 'until' or 'while' after the number of retries, got: {}",
std::string_view(token.begin, token.end));
retry_until = std::string_view(token.begin, token.end) == "until";
}
} }
else if (!is_leading_hint && token.type == TokenType::BareWord && command_errors.contains(item)) else if (!is_leading_hint && token.type == TokenType::BareWord && command_errors.contains(item))
{ {
@ -133,6 +180,9 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint)
break; break;
} }
} }
if (max_retries && server_errors.size() != 1)
throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected one serverError after the 'retry N while|until' command");
} }
} }

View File

@ -6,6 +6,7 @@
#include <fmt/format.h> #include <fmt/format.h>
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/Exception.h>
namespace DB namespace DB
@ -65,12 +66,17 @@ public:
bool hasExpectedClientError(int error); bool hasExpectedClientError(int error);
bool hasExpectedServerError(int error); bool hasExpectedServerError(int error);
bool needRetry(const std::unique_ptr<Exception> & server_exception, size_t * retries_counter);
private: private:
const String & query; const String & query;
ErrorVector server_errors{}; ErrorVector server_errors{};
ErrorVector client_errors{}; ErrorVector client_errors{};
std::optional<bool> echo; std::optional<bool> echo;
size_t max_retries = 0;
bool retry_until = false;
void parse(Lexer & comment_lexer, bool is_leading_hint); void parse(Lexer & comment_lexer, bool is_leading_hint);
bool allErrorsExpected(int actual_server_error, int actual_client_error) const bool allErrorsExpected(int actual_server_error, int actual_client_error) const

View File

@ -154,6 +154,7 @@ StoragePtr DatabaseSQLite::fetchTable(const String & table_name, ContextPtr loca
table_name, table_name,
ColumnsDescription{*columns}, ColumnsDescription{*columns},
ConstraintsDescription{}, ConstraintsDescription{},
/* comment = */ "",
local_context); local_context);
return storage; return storage;

View File

@ -84,7 +84,7 @@ public:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()); return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{ {
Generator generator; Generator generator;
generator.init(arguments, max_substrings_includes_remaining_string); generator.init(arguments, max_substrings_includes_remaining_string);
@ -107,18 +107,17 @@ public:
const ColumnString::Chars & src_chars = col_str->getChars(); const ColumnString::Chars & src_chars = col_str->getChars();
const ColumnString::Offsets & src_offsets = col_str->getOffsets(); const ColumnString::Offsets & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size()); res_offsets.reserve(input_rows_count);
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random. res_strings_offsets.reserve(input_rows_count * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size()); res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr; Pos token_begin = nullptr;
Pos token_end = nullptr; Pos token_end = nullptr;
size_t size = src_offsets.size();
ColumnString::Offset current_src_offset = 0; ColumnString::Offset current_src_offset = 0;
ColumnArray::Offset current_dst_offset = 0; ColumnArray::Offset current_dst_offset = 0;
ColumnString::Offset current_dst_strings_offset = 0; ColumnString::Offset current_dst_strings_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]); Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
current_src_offset = src_offsets[i]; current_src_offset = src_offsets[i];

View File

@ -50,7 +50,7 @@ public:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat32>())); return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat32>()));
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
ColumnPtr array_ptr = arguments[0].column; ColumnPtr array_ptr = arguments[0].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get()); const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
@ -79,7 +79,7 @@ public:
ColumnArray::Offset prev_src_offset = 0; ColumnArray::Offset prev_src_offset = 0;
for (size_t i = 0; i < src_offsets.size(); ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
UInt64 period; UInt64 period;
auto period_ptr = arguments[1].column->convertToFullColumnIfConst(); auto period_ptr = arguments[1].column->convertToFullColumnIfConst();

View File

@ -55,7 +55,7 @@ public:
template <typename DataType> template <typename DataType>
bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars, size_t input_rows_count) const
{ {
const ColumnConst & col_times_const = checkAndGetColumn<ColumnConst>(*col_times); const ColumnConst & col_times_const = checkAndGetColumn<ColumnConst>(*col_times);
@ -71,12 +71,12 @@ public:
checkRepeatTime(times); checkRepeatTime(times);
res_offsets.resize(col_times->size()); res_offsets.resize(input_rows_count);
res_chars.resize(col_times->size() * (times + 1)); res_chars.resize(input_rows_count * (times + 1));
size_t pos = 0; size_t pos = 0;
for (size_t i = 0; i < col_times->size(); ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
memset(res_chars.begin() + pos, space, times); memset(res_chars.begin() + pos, space, times);
pos += times; pos += times;
@ -92,20 +92,20 @@ public:
template <typename DataType> template <typename DataType>
bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars, size_t input_rows_count) const
{ {
auto * col_times = checkAndGetColumn<typename DataType::ColumnType>(col_times_.get()); auto * col_times = checkAndGetColumn<typename DataType::ColumnType>(col_times_.get());
if (!col_times) if (!col_times)
return false; return false;
res_offsets.resize(col_times->size()); res_offsets.resize(input_rows_count);
res_chars.resize(col_times->size() * 10); /// heuristic res_chars.resize(input_rows_count * 10); /// heuristic
const PaddedPODArray<typename DataType::FieldType> & times_data = col_times->getData(); const PaddedPODArray<typename DataType::FieldType> & times_data = col_times->getData();
size_t pos = 0; size_t pos = 0;
for (size_t i = 0; i < col_times->size(); ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
typename DataType::FieldType times = times_data[i]; typename DataType::FieldType times = times_data[i];
@ -132,7 +132,7 @@ public:
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
const auto & col_num = arguments[0].column; const auto & col_num = arguments[0].column;
@ -143,26 +143,26 @@ public:
if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get())) if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get()))
{ {
if ((executeConstant<DataTypeUInt8>(col_num, res_offsets, res_chars)) if ((executeConstant<DataTypeUInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeUInt16>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeUInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeUInt32>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeUInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeUInt64>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeUInt64>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt8>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt16>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt32>(col_num, res_offsets, res_chars)) || (executeConstant<DataTypeInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt64>(col_num, res_offsets, res_chars))) || (executeConstant<DataTypeInt64>(col_num, res_offsets, res_chars, input_rows_count)))
return col_res; return col_res;
} }
else else
{ {
if ((executeVector<DataTypeUInt8>(col_num, res_offsets, res_chars)) if ((executeVector<DataTypeUInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeUInt16>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeUInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeUInt32>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeUInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeUInt64>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeUInt64>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt8>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt16>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt32>(col_num, res_offsets, res_chars)) || (executeVector<DataTypeInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt64>(col_num, res_offsets, res_chars))) || (executeVector<DataTypeInt64>(col_num, res_offsets, res_chars, input_rows_count)))
return col_res; return col_res;
} }

View File

@ -32,7 +32,8 @@ struct StemImpl
const ColumnString::Offsets & offsets, const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets, ColumnString::Offsets & res_offsets,
const String & language) const String & language,
size_t input_rows_count)
{ {
sb_stemmer * stemmer = sb_stemmer_new(language.data(), "UTF_8"); sb_stemmer * stemmer = sb_stemmer_new(language.data(), "UTF_8");
@ -45,7 +46,7 @@ struct StemImpl
res_offsets.assign(offsets); res_offsets.assign(offsets);
UInt64 data_size = 0; UInt64 data_size = 0;
for (UInt64 i = 0; i < offsets.size(); ++i) for (UInt64 i = 0; i < input_rows_count; ++i)
{ {
/// Note that accessing -1th element is valid for PaddedPODArray. /// Note that accessing -1th element is valid for PaddedPODArray.
size_t original_size = offsets[i] - offsets[i - 1]; size_t original_size = offsets[i] - offsets[i - 1];
@ -101,7 +102,7 @@ public:
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
const auto & langcolumn = arguments[0].column; const auto & langcolumn = arguments[0].column;
const auto & strcolumn = arguments[1].column; const auto & strcolumn = arguments[1].column;
@ -119,7 +120,7 @@ public:
String language = lang_col->getValue<String>(); String language = lang_col->getValue<String>();
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
StemImpl::vector(words_col->getChars(), words_col->getOffsets(), col_res->getChars(), col_res->getOffsets(), language); StemImpl::vector(words_col->getChars(), words_col->getOffsets(), col_res->getChars(), col_res->getOffsets(), language, input_rows_count);
return col_res; return col_res;
} }
}; };

View File

@ -40,7 +40,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
static bool tryExecuteString(const IColumn * col, ColumnPtr & col_res) static bool tryExecuteString(const IColumn * col, ColumnPtr & col_res, size_t input_rows_count)
{ {
const ColumnString * col_str_in = checkAndGetColumn<ColumnString>(col); const ColumnString * col_str_in = checkAndGetColumn<ColumnString>(col);
@ -53,8 +53,7 @@ public:
const ColumnString::Chars & in_vec = col_str_in->getChars(); const ColumnString::Chars & in_vec = col_str_in->getChars();
const ColumnString::Offsets & in_offsets = col_str_in->getOffsets(); const ColumnString::Offsets & in_offsets = col_str_in->getOffsets();
size_t size = in_offsets.size(); out_offsets.resize(input_rows_count);
out_offsets.resize(size);
out_vec.resize(in_vec.size()); out_vec.resize(in_vec.size());
char * begin = reinterpret_cast<char *>(out_vec.data()); char * begin = reinterpret_cast<char *>(out_vec.data());
@ -62,7 +61,7 @@ public:
ColumnString::Offset current_in_offset = 0; ColumnString::Offset current_in_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const char * pos_in = reinterpret_cast<const char *>(&in_vec[current_in_offset]); const char * pos_in = reinterpret_cast<const char *>(&in_vec[current_in_offset]);
size_t current_size = strlen(pos_in); size_t current_size = strlen(pos_in);
@ -87,7 +86,7 @@ public:
} }
} }
static bool tryExecuteFixedString(const IColumn * col, ColumnPtr & col_res) static bool tryExecuteFixedString(const IColumn * col, ColumnPtr & col_res, size_t input_rows_count)
{ {
const ColumnFixedString * col_fstr_in = checkAndGetColumn<ColumnFixedString>(col); const ColumnFixedString * col_fstr_in = checkAndGetColumn<ColumnFixedString>(col);
@ -99,10 +98,8 @@ public:
const ColumnString::Chars & in_vec = col_fstr_in->getChars(); const ColumnString::Chars & in_vec = col_fstr_in->getChars();
size_t size = col_fstr_in->size(); out_offsets.resize(input_rows_count);
out_vec.resize(in_vec.size() + input_rows_count);
out_offsets.resize(size);
out_vec.resize(in_vec.size() + size);
char * begin = reinterpret_cast<char *>(out_vec.data()); char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin; char * pos = begin;
@ -110,7 +107,7 @@ public:
size_t n = col_fstr_in->getN(); size_t n = col_fstr_in->getN();
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
size_t current_size = strnlen(pos_in, n); size_t current_size = strnlen(pos_in, n);
memcpySmallAllowReadWriteOverflow15(pos, pos_in, current_size); memcpySmallAllowReadWriteOverflow15(pos, pos_in, current_size);
@ -133,12 +130,12 @@ public:
} }
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
const IColumn * column = arguments[0].column.get(); const IColumn * column = arguments[0].column.get();
ColumnPtr res_column; ColumnPtr res_column;
if (tryExecuteFixedString(column, res_column) || tryExecuteString(column, res_column)) if (tryExecuteFixedString(column, res_column, input_rows_count) || tryExecuteString(column, res_column, input_rows_count))
return res_column; return res_column;
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",

View File

@ -68,7 +68,7 @@ namespace
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
ColumnPtr column_string = arguments[0].column; ColumnPtr column_string = arguments[0].column;
ColumnPtr column_delim = arguments[1].column; ColumnPtr column_delim = arguments[1].column;
@ -110,10 +110,10 @@ namespace
if (is_count_const) if (is_count_const)
{ {
Int64 count = column_count->getInt(0); Int64 count = column_count->getInt(0);
vectorConstant(col_str, delim, count, vec_res, offsets_res); vectorConstant(col_str, delim, count, vec_res, offsets_res, input_rows_count);
} }
else else
vectorVector(col_str, delim, column_count.get(), vec_res, offsets_res); vectorVector(col_str, delim, column_count.get(), vec_res, offsets_res, input_rows_count);
} }
return column_res; return column_res;
} }
@ -124,18 +124,18 @@ namespace
const String & delim, const String & delim,
const IColumn * count_column, const IColumn * count_column,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{ {
size_t rows = str_column->size();
res_data.reserve(str_column->getChars().size() / 2); res_data.reserve(str_column->getChars().size() / 2);
res_offsets.reserve(rows); res_offsets.reserve(input_rows_count);
bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size()); && isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size()); = !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
StringRef str_ref = str_column->getDataAt(i); StringRef str_ref = str_column->getDataAt(i);
Int64 count = count_column->getInt(i); Int64 count = count_column->getInt(i);
@ -157,18 +157,18 @@ namespace
const String & delim, const String & delim,
Int64 count, Int64 count,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{ {
size_t rows = str_column->size();
res_data.reserve(str_column->getChars().size() / 2); res_data.reserve(str_column->getChars().size() / 2);
res_offsets.reserve(rows); res_offsets.reserve(input_rows_count);
bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size()) bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size()); && isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size()); = !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
StringRef str_ref = str_column->getDataAt(i); StringRef str_ref = str_column->getDataAt(i);

View File

@ -6,6 +6,7 @@ namespace DB
{ {
using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval<SubtractNanosecondsImpl>; using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval<SubtractNanosecondsImpl>;
REGISTER_FUNCTION(SubtractNanoseconds) REGISTER_FUNCTION(SubtractNanoseconds)
{ {
factory.registerFunction<FunctionSubtractNanoseconds>(); factory.registerFunction<FunctionSubtractNanoseconds>();

View File

@ -152,7 +152,7 @@ private:
return nullptr; return nullptr;
} }
bool allow_custom_error_code_argument; const bool allow_custom_error_code_argument;
}; };
} }

View File

@ -41,18 +41,17 @@ struct TimeSlotsImpl
/// The following three methods process DateTime type /// The following three methods process DateTime type
static void vectorVector( static void vectorVector(
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<UInt32> & durations, UInt32 time_slot_size, const PaddedPODArray<UInt32> & starts, const PaddedPODArray<UInt32> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets,
size_t input_rows_count)
{ {
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = starts.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
ColumnArray::Offset current_offset = 0; ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + durations[i]) / time_slot_size; value <= end; ++value) for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + durations[i]) / time_slot_size; value <= end; ++value)
{ {
@ -66,18 +65,17 @@ struct TimeSlotsImpl
static void vectorConstant( static void vectorConstant(
const PaddedPODArray<UInt32> & starts, UInt32 duration, UInt32 time_slot_size, const PaddedPODArray<UInt32> & starts, UInt32 duration, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets,
size_t input_rows_count)
{ {
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = starts.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
ColumnArray::Offset current_offset = 0; ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; ++value) for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; ++value)
{ {
@ -91,18 +89,17 @@ struct TimeSlotsImpl
static void constantVector( static void constantVector(
UInt32 start, const PaddedPODArray<UInt32> & durations, UInt32 time_slot_size, UInt32 start, const PaddedPODArray<UInt32> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets) PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets,
size_t input_rows_count)
{ {
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = durations.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
ColumnArray::Offset current_offset = 0; ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (UInt32 value = start / time_slot_size, end = (start + durations[i]) / time_slot_size; value <= end; ++value) for (UInt32 value = start / time_slot_size, end = (start + durations[i]) / time_slot_size; value <= end; ++value)
{ {
@ -120,12 +117,11 @@ struct TimeSlotsImpl
*/ */
static NO_SANITIZE_UNDEFINED void vectorVector( static NO_SANITIZE_UNDEFINED void vectorVector(
const PaddedPODArray<DateTime64> & starts, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size, const PaddedPODArray<DateTime64> & starts, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size,
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale,
size_t input_rows_count)
{ {
size_t size = starts.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
/// Modify all units to have same scale /// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -139,7 +135,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1) for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1)
{ {
@ -152,12 +148,11 @@ struct TimeSlotsImpl
static NO_SANITIZE_UNDEFINED void vectorConstant( static NO_SANITIZE_UNDEFINED void vectorConstant(
const PaddedPODArray<DateTime64> & starts, Decimal64 duration, Decimal64 time_slot_size, const PaddedPODArray<DateTime64> & starts, Decimal64 duration, Decimal64 time_slot_size,
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale,
size_t input_rows_count)
{ {
size_t size = starts.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
/// Modify all units to have same scale /// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -172,7 +167,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + duration) / time_slot_size; value <= end; value += 1) for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] * dt_multiplier + duration) / time_slot_size; value <= end; value += 1)
{ {
@ -185,12 +180,11 @@ struct TimeSlotsImpl
static NO_SANITIZE_UNDEFINED void constantVector( static NO_SANITIZE_UNDEFINED void constantVector(
DateTime64 start, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size, DateTime64 start, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size,
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale) PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale,
size_t input_rows_count)
{ {
size_t size = durations.size(); result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
result_offsets.resize(size);
result_values.reserve(size);
/// Modify all units to have same scale /// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale}); UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -205,7 +199,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0) if (time_slot_size == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
for (DateTime64 value = start / time_slot_size, end = (start + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1) for (DateTime64 value = start / time_slot_size, end = (start + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1)
{ {
@ -282,7 +276,7 @@ public:
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
if (WhichDataType(arguments[0].type).isDateTime()) if (WhichDataType(arguments[0].type).isDateTime())
{ {
@ -308,17 +302,17 @@ public:
if (dt_starts && durations) if (dt_starts && durations)
{ {
TimeSlotsImpl::vectorVector(dt_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets()); TimeSlotsImpl::vectorVector(dt_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), input_rows_count);
return res; return res;
} }
else if (dt_starts && const_durations) else if (dt_starts && const_durations)
{ {
TimeSlotsImpl::vectorConstant(dt_starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets()); TimeSlotsImpl::vectorConstant(dt_starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets(), input_rows_count);
return res; return res;
} }
else if (dt_const_starts && durations) else if (dt_const_starts && durations)
{ {
TimeSlotsImpl::constantVector(dt_const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets()); TimeSlotsImpl::constantVector(dt_const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets(), input_rows_count);
return res; return res;
} }
} }
@ -353,21 +347,21 @@ public:
if (starts && durations) if (starts && durations)
{ {
TimeSlotsImpl::vectorVector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), TimeSlotsImpl::vectorVector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(),
start_time_scale, duration_scale, time_slot_scale); start_time_scale, duration_scale, time_slot_scale, input_rows_count);
return res; return res;
} }
else if (starts && const_durations) else if (starts && const_durations)
{ {
TimeSlotsImpl::vectorConstant( TimeSlotsImpl::vectorConstant(
starts->getData(), const_durations->getValue<Decimal64>(), time_slot_size, res_values, res->getOffsets(), starts->getData(), const_durations->getValue<Decimal64>(), time_slot_size, res_values, res->getOffsets(),
start_time_scale, duration_scale, time_slot_scale); start_time_scale, duration_scale, time_slot_scale, input_rows_count);
return res; return res;
} }
else if (const_starts && durations) else if (const_starts && durations)
{ {
TimeSlotsImpl::constantVector( TimeSlotsImpl::constantVector(
const_starts->getValue<DateTime64>(), durations->getData(), time_slot_size, res_values, res->getOffsets(), const_starts->getValue<DateTime64>(), durations->getData(), time_slot_size, res_values, res->getOffsets(),
start_time_scale, duration_scale, time_slot_scale); start_time_scale, duration_scale, time_slot_scale, input_rows_count);
return res; return res;
} }
} }

View File

@ -54,9 +54,9 @@ private:
/// For operations with Integer/Float /// For operations with Integer/Float
template <typename FromVectorType> template <typename FromVectorType>
void vectorConstant(const FromVectorType & vec_from, UInt8 precision, void vectorConstant(const FromVectorType & vec_from, UInt8 precision,
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets,
size_t input_rows_count) const
{ {
size_t input_rows_count = vec_from.size();
result_offsets.resize(input_rows_count); result_offsets.resize(input_rows_count);
/// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated,
@ -74,9 +74,9 @@ private:
template <typename FirstArgVectorType> template <typename FirstArgVectorType>
void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision, void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision,
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets,
size_t input_rows_count) const
{ {
size_t input_rows_count = vec_from.size();
result_offsets.resize(input_rows_count); result_offsets.resize(input_rows_count);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to); WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
@ -98,7 +98,8 @@ private:
/// For operations with Decimal /// For operations with Decimal
template <typename FirstArgVectorType> template <typename FirstArgVectorType>
void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision,
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale,
size_t input_rows_count) const
{ {
/// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77.
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10; constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
@ -107,7 +108,6 @@ private:
"Too many fractional digits requested for Decimal, must not be more than {}", max_digits); "Too many fractional digits requested for Decimal, must not be more than {}", max_digits);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to); WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
size_t input_rows_count = vec_from.size();
result_offsets.resize(input_rows_count); result_offsets.resize(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i) for (size_t i = 0; i < input_rows_count; ++i)
@ -121,9 +121,9 @@ private:
template <typename FirstArgVectorType> template <typename FirstArgVectorType>
void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision, void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision,
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale,
size_t input_rows_count) const
{ {
size_t input_rows_count = vec_from.size();
result_offsets.resize(input_rows_count); result_offsets.resize(input_rows_count);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to); WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
@ -182,28 +182,28 @@ private:
} }
public: public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
switch (arguments[0].type->getTypeId()) switch (arguments[0].type->getTypeId())
{ {
case TypeIndex::UInt8: return executeType<UInt8>(arguments); case TypeIndex::UInt8: return executeType<UInt8>(arguments, input_rows_count);
case TypeIndex::UInt16: return executeType<UInt16>(arguments); case TypeIndex::UInt16: return executeType<UInt16>(arguments, input_rows_count);
case TypeIndex::UInt32: return executeType<UInt32>(arguments); case TypeIndex::UInt32: return executeType<UInt32>(arguments, input_rows_count);
case TypeIndex::UInt64: return executeType<UInt64>(arguments); case TypeIndex::UInt64: return executeType<UInt64>(arguments, input_rows_count);
case TypeIndex::UInt128: return executeType<UInt128>(arguments); case TypeIndex::UInt128: return executeType<UInt128>(arguments, input_rows_count);
case TypeIndex::UInt256: return executeType<UInt256>(arguments); case TypeIndex::UInt256: return executeType<UInt256>(arguments, input_rows_count);
case TypeIndex::Int8: return executeType<Int8>(arguments); case TypeIndex::Int8: return executeType<Int8>(arguments, input_rows_count);
case TypeIndex::Int16: return executeType<Int16>(arguments); case TypeIndex::Int16: return executeType<Int16>(arguments, input_rows_count);
case TypeIndex::Int32: return executeType<Int32>(arguments); case TypeIndex::Int32: return executeType<Int32>(arguments, input_rows_count);
case TypeIndex::Int64: return executeType<Int64>(arguments); case TypeIndex::Int64: return executeType<Int64>(arguments, input_rows_count);
case TypeIndex::Int128: return executeType<Int128>(arguments); case TypeIndex::Int128: return executeType<Int128>(arguments, input_rows_count);
case TypeIndex::Int256: return executeType<Int256>(arguments); case TypeIndex::Int256: return executeType<Int256>(arguments, input_rows_count);
case TypeIndex::Float32: return executeType<Float32>(arguments); case TypeIndex::Float32: return executeType<Float32>(arguments, input_rows_count);
case TypeIndex::Float64: return executeType<Float64>(arguments); case TypeIndex::Float64: return executeType<Float64>(arguments, input_rows_count);
case TypeIndex::Decimal32: return executeType<Decimal32>(arguments); case TypeIndex::Decimal32: return executeType<Decimal32>(arguments, input_rows_count);
case TypeIndex::Decimal64: return executeType<Decimal64>(arguments); case TypeIndex::Decimal64: return executeType<Decimal64>(arguments, input_rows_count);
case TypeIndex::Decimal128: return executeType<Decimal128>(arguments); case TypeIndex::Decimal128: return executeType<Decimal128>(arguments, input_rows_count);
case TypeIndex::Decimal256: return executeType<Decimal256>(arguments); case TypeIndex::Decimal256: return executeType<Decimal256>(arguments, input_rows_count);
default: default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
arguments[0].column->getName(), getName()); arguments[0].column->getName(), getName());
@ -212,7 +212,7 @@ public:
private: private:
template <typename T> template <typename T>
ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const ColumnPtr executeType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const
{ {
const auto * precision_col = checkAndGetColumn<ColumnVector<UInt8>>(arguments[1].column.get()); const auto * precision_col = checkAndGetColumn<ColumnVector<UInt8>>(arguments[1].column.get());
const auto * precision_col_const = checkAndGetColumnConst<ColumnVector<UInt8>>(arguments[1].column.get()); const auto * precision_col_const = checkAndGetColumnConst<ColumnVector<UInt8>>(arguments[1].column.get());
@ -230,9 +230,9 @@ private:
{ {
UInt8 from_scale = from_col->getScale(); UInt8 from_scale = from_col->getScale();
if (precision_col_const) if (precision_col_const)
vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets, from_scale); vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets, from_scale, input_rows_count);
else if (precision_col) else if (precision_col)
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale, input_rows_count);
else else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName());
} }
@ -245,9 +245,9 @@ private:
if (from_col) if (from_col)
{ {
if (precision_col_const) if (precision_col_const)
vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets); vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets, input_rows_count);
else if (precision_col) else if (precision_col)
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, input_rows_count);
else else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName());

View File

@ -147,19 +147,20 @@ public:
std::unreachable(); std::unreachable();
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{ {
const auto & time_column = arguments[0]; const auto & time_column = arguments[0];
const auto & interval_column = arguments[1]; const auto & interval_column = arguments[1];
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone); auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone, input_rows_count);
return result_column; return result_column;
} }
private: private:
ColumnPtr dispatchForTimeColumn( ColumnPtr dispatchForTimeColumn(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone) const const DataTypePtr & result_type, const DateLUTImpl & time_zone,
size_t input_rows_count) const
{ {
const auto & time_column_type = *time_column.type.get(); const auto & time_column_type = *time_column.type.get();
const auto & time_column_col = *time_column.column.get(); const auto & time_column_col = *time_column.column.get();
@ -170,19 +171,19 @@ private:
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale(); auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
if (time_column_vec) if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale); return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count, scale);
} }
else if (isDateTime(time_column_type)) else if (isDateTime(time_column_type))
{ {
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col); const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col);
if (time_column_vec) if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone); return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
} }
else if (isDate(time_column_type)) else if (isDate(time_column_type))
{ {
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(&time_column_col); const auto * time_column_vec = checkAndGetColumn<ColumnDate>(&time_column_col);
if (time_column_vec) if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone); return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
} }
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName());
} }
@ -190,7 +191,7 @@ private:
template <typename TimeDataType, typename TimeColumnType> template <typename TimeDataType, typename TimeColumnType>
ColumnPtr dispatchForIntervalColumn( ColumnPtr dispatchForIntervalColumn(
const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale = 1) const
{ {
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get()); const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
if (!interval_type) if (!interval_type)
@ -207,27 +208,27 @@ private:
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{ {
case IntervalKind::Kind::Nanosecond: case IntervalKind::Kind::Nanosecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Nanosecond>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Nanosecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Microsecond: case IntervalKind::Kind::Microsecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Microsecond>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Microsecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Millisecond: case IntervalKind::Kind::Millisecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Millisecond>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Millisecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Second: case IntervalKind::Kind::Second:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Second>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Second>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Minute: case IntervalKind::Kind::Minute:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Minute>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Minute>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Hour: case IntervalKind::Kind::Hour:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Hour>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Hour>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Day: case IntervalKind::Kind::Day:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Day>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Day>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Week: case IntervalKind::Kind::Week:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Week>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Week>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Month: case IntervalKind::Kind::Month:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Month>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Month>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Quarter: case IntervalKind::Kind::Quarter:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Quarter>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Quarter>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
case IntervalKind::Kind::Year: case IntervalKind::Kind::Year:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Year>(time_data_type, time_column, num_units, result_type, time_zone, scale); return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Year>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
} }
std::unreachable(); std::unreachable();
@ -236,22 +237,21 @@ private:
template <typename TimeDataType, typename TimeColumnType, typename ResultDataType, IntervalKind::Kind unit> template <typename TimeDataType, typename TimeColumnType, typename ResultDataType, IntervalKind::Kind unit>
ColumnPtr execute( ColumnPtr execute(
const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale) const
{ {
using ResultColumnType = typename ResultDataType::ColumnType; using ResultColumnType = typename ResultDataType::ColumnType;
using ResultFieldType = typename ResultDataType::FieldType; using ResultFieldType = typename ResultDataType::FieldType;
const auto & time_data = time_column_type.getData(); const auto & time_data = time_column_type.getData();
size_t size = time_data.size();
auto result_col = result_type->createColumn(); auto result_col = result_type->createColumn();
auto * col_to = assert_cast<ResultColumnType *>(result_col.get()); auto * col_to = assert_cast<ResultColumnType *>(result_col.get());
auto & result_data = col_to->getData(); auto & result_data = col_to->getData();
result_data.resize(size); result_data.resize(input_rows_count);
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale); Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
for (size_t i = 0; i != size; ++i) for (size_t i = 0; i != input_rows_count; ++i)
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier)); result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
return result_col; return result_col;

View File

@ -73,7 +73,7 @@ public:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()); return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
auto column_offsets = ColumnArray::ColumnOffsets::create(); auto column_offsets = ColumnArray::ColumnOffsets::create();
@ -90,9 +90,9 @@ public:
auto input_column = arguments[0].column; auto input_column = arguments[0].column;
if (const auto * column_string = checkAndGetColumn<ColumnString>(input_column.get())) if (const auto * column_string = checkAndGetColumn<ColumnString>(input_column.get()))
executeImpl(extractor, *column_string, *result_column_string, *column_offsets); executeImpl(extractor, *column_string, *result_column_string, *column_offsets, input_rows_count);
else if (const auto * column_fixed_string = checkAndGetColumn<ColumnFixedString>(input_column.get())) else if (const auto * column_fixed_string = checkAndGetColumn<ColumnFixedString>(input_column.get()))
executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets); executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets, input_rows_count);
return ColumnArray::create(std::move(result_column_string), std::move(column_offsets)); return ColumnArray::create(std::move(result_column_string), std::move(column_offsets));
} }
@ -105,9 +105,9 @@ public:
auto input_column = arguments[0].column; auto input_column = arguments[0].column;
if (const auto * column_string = checkAndGetColumn<ColumnString>(input_column.get())) if (const auto * column_string = checkAndGetColumn<ColumnString>(input_column.get()))
executeImpl(extractor, *column_string, *result_column_string, *column_offsets); executeImpl(extractor, *column_string, *result_column_string, *column_offsets, input_rows_count);
else if (const auto * column_fixed_string = checkAndGetColumn<ColumnFixedString>(input_column.get())) else if (const auto * column_fixed_string = checkAndGetColumn<ColumnFixedString>(input_column.get()))
executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets); executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets, input_rows_count);
return ColumnArray::create(std::move(result_column_string), std::move(column_offsets)); return ColumnArray::create(std::move(result_column_string), std::move(column_offsets));
} }
@ -120,15 +120,15 @@ private:
const ExtractorType & extractor, const ExtractorType & extractor,
StringColumnType & input_data_column, StringColumnType & input_data_column,
ResultStringColumnType & result_data_column, ResultStringColumnType & result_data_column,
ColumnArray::ColumnOffsets & offsets_column) const ColumnArray::ColumnOffsets & offsets_column,
size_t input_rows_count) const
{ {
size_t current_tokens_size = 0; size_t current_tokens_size = 0;
auto & offsets_data = offsets_column.getData(); auto & offsets_data = offsets_column.getData();
size_t column_size = input_data_column.size(); offsets_data.resize(input_rows_count);
offsets_data.resize(column_size);
for (size_t i = 0; i < column_size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
auto data = input_data_column.getDataAt(i); auto data = input_data_column.getDataAt(i);

View File

@ -173,30 +173,30 @@ namespace
} }
else if (cache.table_num_to_idx) else if (cache.table_num_to_idx)
{ {
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const, *in_casted) if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const, *in_casted) && !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const, *in_casted)) && !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const, *in_casted, input_rows_count))
{ {
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
} }
} }
else if (cache.table_string_to_idx) else if (cache.table_string_to_idx)
{ {
if (!executeString(in, *column_result, default_non_const, *in_casted)) if (!executeString(in, *column_result, default_non_const, *in_casted, input_rows_count))
executeContiguous(in, *column_result, default_non_const, *in_casted); executeContiguous(in, *column_result, default_non_const, *in_casted, input_rows_count);
} }
else if (cache.table_anything_to_idx) else if (cache.table_anything_to_idx)
{ {
executeAnything(in, *column_result, default_non_const, *in_casted); executeAnything(in, *column_result, default_non_const, *in_casted, input_rows_count);
} }
else else
throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized");
@ -217,12 +217,11 @@ namespace
return impl->execute(args, result_type, input_rows_count); return impl->execute(args, result_type, input_rows_count);
} }
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
{ {
const size_t size = in->size();
const auto & table = *cache.table_anything_to_idx; const auto & table = *cache.table_anything_to_idx;
column_result.reserve(size); column_result.reserve(input_rows_count);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
SipHash hash; SipHash hash;
in->updateHashWithValue(i, hash); in->updateHashWithValue(i, hash);
@ -239,12 +238,11 @@ namespace
} }
} }
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
{ {
const size_t size = in->size();
const auto & table = *cache.table_string_to_idx; const auto & table = *cache.table_string_to_idx;
column_result.reserve(size); column_result.reserve(input_rows_count);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const auto * it = table.find(in->getDataAt(i)); const auto * it = table.find(in->getDataAt(i));
if (it) if (it)
@ -259,7 +257,7 @@ namespace
} }
template <typename T> template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
{ {
const auto * const in = checkAndGetColumn<T>(in_untyped); const auto * const in = checkAndGetColumn<T>(in_untyped);
if (!in) if (!in)
@ -269,24 +267,23 @@ namespace
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>) if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
in_scale = in->getScale(); in_scale = in->getScale();
if (!executeNumToString(pod, column_result, default_non_const) if (!executeNumToString(pod, column_result, default_non_const, input_rows_count)
&& !executeNumToNum<ColumnVector<UInt8>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<UInt8>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<UInt16>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<UInt16>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<UInt32>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<UInt32>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<UInt64>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<UInt64>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Int8>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Int8>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Int16>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Int16>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Int32>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Int32>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Int64>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Int64>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Float32>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Float32>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnVector<Float64>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnVector<Float64>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnDecimal<Decimal32>>(pod, column_result, default_non_const, in_scale) && !executeNumToNum<ColumnDecimal<Decimal32>>(pod, column_result, default_non_const, in_scale, input_rows_count)
&& !executeNumToNum<ColumnDecimal<Decimal64>>(pod, column_result, default_non_const, in_scale)) && !executeNumToNum<ColumnDecimal<Decimal64>>(pod, column_result, default_non_const, in_scale, input_rows_count))
{ {
const size_t size = pod.size();
const auto & table = *cache.table_num_to_idx; const auto & table = *cache.table_num_to_idx;
column_result.reserve(size); column_result.reserve(input_rows_count);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const auto * it = table.find(bit_cast<UInt64>(pod[i])); const auto * it = table.find(bit_cast<UInt64>(pod[i]));
if (it) if (it)
@ -303,14 +300,13 @@ namespace
} }
template <typename T> template <typename T>
bool executeNumToString(const PaddedPODArray<T> & pod, IColumn & column_result, const ColumnPtr default_non_const) const bool executeNumToString(const PaddedPODArray<T> & pod, IColumn & column_result, const ColumnPtr default_non_const, size_t input_rows_count) const
{ {
auto * out = typeid_cast<ColumnString *>(&column_result); auto * out = typeid_cast<ColumnString *>(&column_result);
if (!out) if (!out)
return false; return false;
auto & out_offs = out->getOffsets(); auto & out_offs = out->getOffsets();
const size_t size = pod.size(); out_offs.resize(input_rows_count);
out_offs.resize(size);
auto & out_chars = out->getChars(); auto & out_chars = out->getChars();
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get()); const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
@ -325,14 +321,14 @@ namespace
const auto & def_offs = def->getOffsets(); const auto & def_offs = def->getOffsets();
const auto * def_data = def_chars.data(); const auto * def_data = def_chars.data();
auto def_size = def_offs[0]; auto def_size = def_offs[0];
executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_data, def_size, size); executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_data, def_size, input_rows_count);
} }
else else
{ {
const auto * def = assert_cast<const ColumnString *>(default_non_const.get()); const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars(); const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets(); const auto & def_offs = def->getOffsets();
executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, input_rows_count);
} }
return true; return true;
} }
@ -347,10 +343,10 @@ namespace
const ColumnString::Offsets & to_offsets, const ColumnString::Offsets & to_offsets,
const DefData & def_data, const DefData & def_data,
const DefOffs & def_offsets, const DefOffs & def_offsets,
const size_t size) const size_t input_rows_count) const
{ {
size_t out_cur_off = 0; size_t out_cur_off = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const char8_t * to = nullptr; const char8_t * to = nullptr;
size_t to_size = 0; size_t to_size = 0;
@ -382,14 +378,13 @@ namespace
template <typename T, typename U> template <typename T, typename U>
bool executeNumToNum( bool executeNumToNum(
const PaddedPODArray<U> & pod, IColumn & column_result, const ColumnPtr default_non_const, const UInt32 in_scale) const const PaddedPODArray<U> & pod, IColumn & column_result, ColumnPtr default_non_const, UInt32 in_scale, size_t input_rows_count) const
{ {
auto * out = typeid_cast<T *>(&column_result); auto * out = typeid_cast<T *>(&column_result);
if (!out) if (!out)
return false; return false;
auto & out_pod = out->getData(); auto & out_pod = out->getData();
const size_t size = pod.size(); out_pod.resize(input_rows_count);
out_pod.resize(size);
UInt32 out_scale = 0; UInt32 out_scale = 0;
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>) if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
out_scale = out->getScale(); out_scale = out->getScale();
@ -399,15 +394,15 @@ namespace
if (cache.default_column) if (cache.default_column)
{ {
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0]; const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, size, out_scale, out_scale); executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, input_rows_count, out_scale, out_scale);
} }
else if (default_non_const) else if (default_non_const)
{ {
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData(); const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, size, out_scale, out_scale); executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, input_rows_count, out_scale, out_scale);
} }
else else
executeNumToNumHelper(table, pod, out_pod, to_pod, pod, size, out_scale, in_scale); executeNumToNumHelper(table, pod, out_pod, to_pod, pod, input_rows_count, out_scale, in_scale);
return true; return true;
} }
@ -418,11 +413,11 @@ namespace
PaddedPODArray<Out> & out_pod, PaddedPODArray<Out> & out_pod,
const PaddedPODArray<Out> & to_pod, const PaddedPODArray<Out> & to_pod,
const Def & def, const Def & def,
const size_t size, size_t input_rows_count,
const UInt32 out_scale, UInt32 out_scale,
const UInt32 def_scale) const UInt32 def_scale) const
{ {
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const auto * it = table.find(bit_cast<UInt64>(pod[i])); const auto * it = table.find(bit_cast<UInt64>(pod[i]));
if (it) if (it)
@ -450,7 +445,7 @@ namespace
} }
} }
bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
{ {
const auto * const in = checkAndGetColumn<ColumnString>(in_untyped); const auto * const in = checkAndGetColumn<ColumnString>(in_untyped);
if (!in) if (!in)
@ -458,19 +453,19 @@ namespace
const auto & data = in->getChars(); const auto & data = in->getChars();
const auto & offsets = in->getOffsets(); const auto & offsets = in->getOffsets();
if (!executeStringToString(data, offsets, column_result, default_non_const) if (!executeStringToString(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt8>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<UInt8>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt16>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<UInt16>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt32>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<UInt32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt64>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<UInt64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int8>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Int8>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int16>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Int16>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int32>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Int32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int64>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Int64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Float32>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Float32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Float64>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnVector<Float64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnDecimal<Decimal32>>(data, offsets, column_result, default_non_const) && !executeStringToNum<ColumnDecimal<Decimal32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnDecimal<Decimal64>>(data, offsets, column_result, default_non_const)) && !executeStringToNum<ColumnDecimal<Decimal64>>(data, offsets, column_result, default_non_const, input_rows_count))
{ {
const size_t size = offsets.size(); const size_t size = offsets.size();
const auto & table = *cache.table_string_to_idx; const auto & table = *cache.table_string_to_idx;
@ -497,14 +492,14 @@ namespace
const ColumnString::Chars & data, const ColumnString::Chars & data,
const ColumnString::Offsets & offsets, const ColumnString::Offsets & offsets,
IColumn & column_result, IColumn & column_result,
const ColumnPtr default_non_const) const const ColumnPtr default_non_const,
size_t input_rows_count) const
{ {
auto * out = typeid_cast<ColumnString *>(&column_result); auto * out = typeid_cast<ColumnString *>(&column_result);
if (!out) if (!out)
return false; return false;
auto & out_offs = out->getOffsets(); auto & out_offs = out->getOffsets();
const size_t size = offsets.size(); out_offs.resize(input_rows_count);
out_offs.resize(size);
auto & out_chars = out->getChars(); auto & out_chars = out->getChars();
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get()); const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
@ -519,18 +514,18 @@ namespace
const auto & def_offs = def->getOffsets(); const auto & def_offs = def->getOffsets();
const auto * def_data = def_chars.data(); const auto * def_data = def_chars.data();
auto def_size = def_offs[0]; auto def_size = def_offs[0];
executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_data, def_size, size); executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_data, def_size, input_rows_count);
} }
else if (default_non_const) else if (default_non_const)
{ {
const auto * def = assert_cast<const ColumnString *>(default_non_const.get()); const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars(); const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets(); const auto & def_offs = def->getOffsets();
executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, input_rows_count);
} }
else else
{ {
executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, data, offsets, size); executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, data, offsets, input_rows_count);
} }
return true; return true;
} }
@ -546,11 +541,11 @@ namespace
const ColumnString::Offsets & to_offsets, const ColumnString::Offsets & to_offsets,
const DefData & def_data, const DefData & def_data,
const DefOffs & def_offsets, const DefOffs & def_offsets,
const size_t size) const size_t input_rows_count) const
{ {
ColumnString::Offset current_offset = 0; ColumnString::Offset current_offset = 0;
size_t out_cur_off = 0; size_t out_cur_off = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const char8_t * to = nullptr; const char8_t * to = nullptr;
size_t to_size = 0; size_t to_size = 0;
@ -587,26 +582,26 @@ namespace
const ColumnString::Chars & data, const ColumnString::Chars & data,
const ColumnString::Offsets & offsets, const ColumnString::Offsets & offsets,
IColumn & column_result, IColumn & column_result,
const ColumnPtr default_non_const) const const ColumnPtr default_non_const,
size_t input_rows_count) const
{ {
auto * out = typeid_cast<T *>(&column_result); auto * out = typeid_cast<T *>(&column_result);
if (!out) if (!out)
return false; return false;
auto & out_pod = out->getData(); auto & out_pod = out->getData();
const size_t size = offsets.size(); out_pod.resize(input_rows_count);
out_pod.resize(size);
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData(); const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
const auto & table = *cache.table_string_to_idx; const auto & table = *cache.table_string_to_idx;
if (cache.default_column) if (cache.default_column)
{ {
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0]; const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, size); executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, input_rows_count);
} }
else else
{ {
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData(); const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, size); executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, input_rows_count);
} }
return true; return true;
} }
@ -619,10 +614,10 @@ namespace
PaddedPODArray<Out> & out_pod, PaddedPODArray<Out> & out_pod,
const PaddedPODArray<Out> & to_pod, const PaddedPODArray<Out> & to_pod,
const Def & def, const Def & def,
const size_t size) const size_t input_rows_count) const
{ {
ColumnString::Offset current_offset = 0; ColumnString::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1}; const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
current_offset = offsets[i]; current_offset = offsets[i];

View File

@ -52,7 +52,8 @@ struct TranslateImpl
const std::string & map_from, const std::string & map_from,
const std::string & map_to, const std::string & map_to,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{ {
Map map; Map map;
fillMapWithValues(map, map_from, map_to); fillMapWithValues(map, map_from, map_to);
@ -62,7 +63,7 @@ struct TranslateImpl
UInt8 * dst = res_data.data(); UInt8 * dst = res_data.data();
for (UInt64 i = 0; i < offsets.size(); ++i) for (UInt64 i = 0; i < input_rows_count; ++i)
{ {
const UInt8 * src = data.data() + offsets[i - 1]; const UInt8 * src = data.data() + offsets[i - 1];
const UInt8 * src_end = data.data() + offsets[i] - 1; const UInt8 * src_end = data.data() + offsets[i] - 1;
@ -175,19 +176,20 @@ struct TranslateUTF8Impl
const std::string & map_from, const std::string & map_from,
const std::string & map_to, const std::string & map_to,
ColumnString::Chars & res_data, ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets) ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{ {
MapASCII map_ascii; MapASCII map_ascii;
MapUTF8 map; MapUTF8 map;
fillMapWithValues(map_ascii, map, map_from, map_to); fillMapWithValues(map_ascii, map, map_from, map_to);
res_data.resize(data.size()); res_data.resize(data.size());
res_offsets.resize(offsets.size()); res_offsets.resize(input_rows_count);
UInt8 * dst = res_data.data(); UInt8 * dst = res_data.data();
UInt64 data_size = 0; UInt64 data_size = 0;
for (UInt64 i = 0; i < offsets.size(); ++i) for (UInt64 i = 0; i < input_rows_count; ++i)
{ {
const UInt8 * src = data.data() + offsets[i - 1]; const UInt8 * src = data.data() + offsets[i - 1];
const UInt8 * src_end = data.data() + offsets[i] - 1; const UInt8 * src_end = data.data() + offsets[i] - 1;
@ -311,7 +313,7 @@ public:
} }
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
const ColumnPtr column_src = arguments[0].column; const ColumnPtr column_src = arguments[0].column;
const ColumnPtr column_map_from = arguments[1].column; const ColumnPtr column_map_from = arguments[1].column;
@ -330,7 +332,7 @@ public:
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_src.get())) if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_src.get()))
{ {
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
Impl::vector(col->getChars(), col->getOffsets(), map_from, map_to, col_res->getChars(), col_res->getOffsets()); Impl::vector(col->getChars(), col->getOffsets(), map_from, map_to, col_res->getChars(), col_res->getOffsets(), input_rows_count);
return col_res; return col_res;
} }
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get())) else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get()))

View File

@ -99,16 +99,16 @@ public:
return std::make_shared<DataTypeArray>(item_data_type); return std::make_shared<DataTypeArray>(item_data_type);
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
const IColumn * tuple_col = arguments[0].column.get(); const IColumn * tuple_col = arguments[0].column.get();
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(arguments[0].type.get()); const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(arguments[0].type.get());
const auto * tuple_col_concrete = assert_cast<const ColumnTuple*>(tuple_col); const auto * tuple_col_concrete = assert_cast<const ColumnTuple *>(tuple_col);
auto keys = ColumnString::create(); auto keys = ColumnString::create();
MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty(); MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty();
auto offsets = ColumnVector<UInt64>::create(); auto offsets = ColumnVector<UInt64>::create();
for (size_t row = 0; row < tuple_col_concrete->size(); ++row) for (size_t row = 0; row < input_rows_count; ++row)
{ {
for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col) for (size_t col = 0; col < tuple_col_concrete->tupleSize(); ++col)
{ {

View File

@ -492,48 +492,6 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk)
} }
} }
TEST_P(ArchiveReaderAndWriterTest, LargeFile)
{
/// Make an archive.
std::string_view contents = "The contents of a.txt\n";
int times = 10000000;
{
auto writer = createArchiveWriter(getPathToArchive());
{
auto out = writer->writeFile("a.txt", times * contents.size());
for (int i = 0; i < times; i++)
writeString(contents, *out);
out->finalize();
}
writer->finalize();
}
/// Read the archive.
auto reader = createArchiveReader(getPathToArchive());
ASSERT_TRUE(reader->fileExists("a.txt"));
auto file_info = reader->getFileInfo("a.txt");
EXPECT_EQ(file_info.uncompressed_size, contents.size() * times);
EXPECT_GT(file_info.compressed_size, 0);
{
auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true);
for (int i = 0; i < times; i++)
ASSERT_TRUE(checkString(String(contents), *in));
}
{
/// Use an enumerator.
auto enumerator = reader->firstFile();
ASSERT_NE(enumerator, nullptr);
EXPECT_EQ(enumerator->getFileName(), "a.txt");
EXPECT_EQ(enumerator->getFileInfo().uncompressed_size, contents.size() * times);
EXPECT_GT(enumerator->getFileInfo().compressed_size, 0);
EXPECT_FALSE(enumerator->nextFile());
}
}
TEST(TarArchiveReaderTest, FileExists) TEST(TarArchiveReaderTest, FileExists)
{ {
String archive_path = "archive.tar"; String archive_path = "archive.tar";

View File

@ -399,10 +399,9 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
if (query.if_empty) if (query.if_empty)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases");
if (database->hasReplicationThread()) if (!truncate && database->hasReplicationThread())
database->stopReplication(); database->stopReplication();
if (database->shouldBeEmptyOnDetach()) if (database->shouldBeEmptyOnDetach())
{ {
/// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish.

View File

@ -356,7 +356,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
for (size_t i = 0; i < src_arr_size; ++i) for (size_t i = 0; i < src_arr_size; ++i)
{ {
res[i] = convertFieldToType(src_arr[i], element_type); res[i] = convertFieldToType(src_arr[i], element_type);
if (res[i].isNull() && !element_type.isNullable()) if (res[i].isNull() && !canContainNull(element_type))
{ {
// See the comment for Tuples below. // See the comment for Tuples below.
have_unconvertible_element = true; have_unconvertible_element = true;
@ -384,25 +384,25 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
{ {
const auto & element_type = *(type_tuple->getElements()[i]); const auto & element_type = *(type_tuple->getElements()[i]);
res[i] = convertFieldToType(src_tuple[i], element_type); res[i] = convertFieldToType(src_tuple[i], element_type);
if (!res[i].isNull() || element_type.isNullable()) if (res[i].isNull() && !canContainNull(element_type))
continue; {
/*
/* * Either the source element was Null, or the conversion did not
* Either the source element was Null, or the conversion did not * succeed, because the source and the requested types of the
* succeed, because the source and the requested types of the * element are compatible, but the value is not convertible
* element are compatible, but the value is not convertible * (e.g. trying to convert -1 from Int8 to UInt8). In these
* (e.g. trying to convert -1 from Int8 to UInt8). In these * cases, consider the whole tuple also compatible but not
* cases, consider the whole tuple also compatible but not * convertible. According to the specification of this function,
* convertible. According to the specification of this function, * we must return Null in this case.
* we must return Null in this case. *
* * The following elements might be not even compatible, so it
* The following elements might be not even compatible, so it * makes sense to check them to detect user errors. Remember
* makes sense to check them to detect user errors. Remember * that there is an unconvertible element, and try to process
* that there is an unconvertible element, and try to process * the remaining ones. The convertFieldToType for each element
* the remaining ones. The convertFieldToType for each element * will throw if it detects incompatibility.
* will throw if it detects incompatibility. */
*/ have_unconvertible_element = true;
have_unconvertible_element = true; }
} }
return have_unconvertible_element ? Field(Null()) : Field(res); return have_unconvertible_element ? Field(Null()) : Field(res);
@ -433,11 +433,11 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
updated_entry[0] = convertFieldToType(key, key_type); updated_entry[0] = convertFieldToType(key, key_type);
if (updated_entry[0].isNull() && !key_type.isNullable()) if (updated_entry[0].isNull() && !canContainNull(key_type))
have_unconvertible_element = true; have_unconvertible_element = true;
updated_entry[1] = convertFieldToType(value, value_type); updated_entry[1] = convertFieldToType(value, value_type);
if (updated_entry[1].isNull() && !value_type.isNullable()) if (updated_entry[1].isNull() && !canContainNull(value_type))
have_unconvertible_element = true; have_unconvertible_element = true;
res[i] = updated_entry; res[i] = updated_entry;
@ -592,7 +592,7 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co
Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint)
{ {
bool is_null = from_value.isNull(); bool is_null = from_value.isNull();
if (is_null && !to_type.isNullable() && !to_type.isLowCardinalityNullable()) if (is_null && !canContainNull(to_type))
throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName());
Field converted = convertFieldToType(from_value, to_type, from_type_hint); Field converted = convertFieldToType(from_value, to_type, from_type_hint);

View File

@ -1207,6 +1207,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (!select_p.parse(pos, select, expected)) if (!select_p.parse(pos, select, expected))
return false; return false;
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>(); auto query = std::make_shared<ASTCreateQuery>();
node = query; node = query;
@ -1225,6 +1226,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
query->children.push_back(query->database); query->children.push_back(query->database);
if (query->table) if (query->table)
query->children.push_back(query->table); query->children.push_back(query->table);
if (comment)
query->set(query->comment, comment);
query->set(query->columns_list, columns_list); query->set(query->columns_list, columns_list);

View File

@ -418,8 +418,11 @@ namespace
} }
StorageKafka::StorageKafka( StorageKafka::StorageKafka(
const StorageID & table_id_, ContextPtr context_, const StorageID & table_id_,
const ColumnsDescription & columns_, std::unique_ptr<KafkaSettings> kafka_settings_, ContextPtr context_,
const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<KafkaSettings> kafka_settings_,
const String & collection_name_) const String & collection_name_)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_->getGlobalContext()) , WithContext(context_->getGlobalContext())
@ -451,6 +454,7 @@ StorageKafka::StorageKafka(
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode));
@ -1317,7 +1321,7 @@ void registerStorageKafka(StorageFactory & factory)
"See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration");
} }
return std::make_shared<StorageKafka>(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); return std::make_shared<StorageKafka>(args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name);
}; };
factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });

View File

@ -40,6 +40,7 @@ public:
const StorageID & table_id_, const StorageID & table_id_,
ContextPtr context_, ContextPtr context_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<KafkaSettings> kafka_settings_, std::unique_ptr<KafkaSettings> kafka_settings_,
const String & collection_name_); const String & collection_name_);

View File

@ -49,6 +49,7 @@ StorageNATS::StorageNATS(
const StorageID & table_id_, const StorageID & table_id_,
ContextPtr context_, ContextPtr context_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<NATSSettings> nats_settings_, std::unique_ptr<NATSSettings> nats_settings_,
LoadingStrictnessLevel mode) LoadingStrictnessLevel mode)
: IStorage(table_id_) : IStorage(table_id_)
@ -87,6 +88,7 @@ StorageNATS::StorageNATS(
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
setVirtuals(createVirtuals(nats_settings->nats_handle_error_mode)); setVirtuals(createVirtuals(nats_settings->nats_handle_error_mode));
@ -760,7 +762,7 @@ void registerStorageNATS(StorageFactory & factory)
if (!nats_settings->nats_subjects.changed) if (!nats_settings->nats_subjects.changed)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `nats_subjects` setting"); throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `nats_subjects` setting");
return std::make_shared<StorageNATS>(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.mode); return std::make_shared<StorageNATS>(args.table_id, args.getContext(), args.columns, args.comment, std::move(nats_settings), args.mode);
}; };
factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });

View File

@ -23,6 +23,7 @@ public:
const StorageID & table_id_, const StorageID & table_id_,
ContextPtr context_, ContextPtr context_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<NATSSettings> nats_settings_, std::unique_ptr<NATSSettings> nats_settings_,
LoadingStrictnessLevel mode); LoadingStrictnessLevel mode);

View File

@ -571,6 +571,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
StorageInMemoryMetadata metadata; StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns); metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints); metadata.setConstraints(args.constraints);
metadata.setComment(args.comment);
if (args.mode <= LoadingStrictnessLevel::CREATE if (args.mode <= LoadingStrictnessLevel::CREATE
&& !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table) && !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table)

View File

@ -70,6 +70,7 @@ StorageRabbitMQ::StorageRabbitMQ(
const StorageID & table_id_, const StorageID & table_id_,
ContextPtr context_, ContextPtr context_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_, std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
LoadingStrictnessLevel mode) LoadingStrictnessLevel mode)
: IStorage(table_id_) : IStorage(table_id_)
@ -145,6 +146,7 @@ StorageRabbitMQ::StorageRabbitMQ(
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode)); setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode));
@ -1288,7 +1290,7 @@ void registerStorageRabbitMQ(StorageFactory & factory)
if (!rabbitmq_settings->rabbitmq_format.changed) if (!rabbitmq_settings->rabbitmq_format.changed)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting"); throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting");
return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, std::move(rabbitmq_settings), args.mode); return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode);
}; };
factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });

View File

@ -26,6 +26,7 @@ public:
const StorageID & table_id_, const StorageID & table_id_,
ContextPtr context_, ContextPtr context_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_, std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
LoadingStrictnessLevel mode); LoadingStrictnessLevel mode);

View File

@ -691,6 +691,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
StorageInMemoryMetadata metadata; StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns); metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints); metadata.setConstraints(args.constraints);
metadata.setComment(args.comment);
if (!args.storage_def->primary_key) if (!args.storage_def->primary_key)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key");

View File

@ -77,7 +77,8 @@ StorageExecutable::StorageExecutable(
const ExecutableSettings & settings_, const ExecutableSettings & settings_,
const std::vector<ASTPtr> & input_queries_, const std::vector<ASTPtr> & input_queries_,
const ColumnsDescription & columns, const ColumnsDescription & columns,
const ConstraintsDescription & constraints) const ConstraintsDescription & constraints,
const String & comment)
: IStorage(table_id_) : IStorage(table_id_)
, settings(settings_) , settings(settings_)
, input_queries(input_queries_) , input_queries(input_queries_)
@ -86,6 +87,7 @@ StorageExecutable::StorageExecutable(
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns); storage_metadata.setColumns(columns);
storage_metadata.setConstraints(constraints); storage_metadata.setConstraints(constraints);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
ShellCommandSourceCoordinator::Configuration configuration ShellCommandSourceCoordinator::Configuration configuration
@ -237,7 +239,7 @@ void registerStorageExecutable(StorageFactory & factory)
settings.loadFromQuery(*args.storage_def); settings.loadFromQuery(*args.storage_def);
auto global_context = args.getContext()->getGlobalContext(); auto global_context = args.getContext()->getGlobalContext();
return std::make_shared<StorageExecutable>(args.table_id, format, settings, input_queries, columns, constraints); return std::make_shared<StorageExecutable>(args.table_id, format, settings, input_queries, columns, constraints, args.comment);
}; };
StorageFactory::StorageFeatures storage_features; StorageFactory::StorageFeatures storage_features;
@ -255,4 +257,3 @@ void registerStorageExecutable(StorageFactory & factory)
} }
} }

View File

@ -22,7 +22,8 @@ public:
const ExecutableSettings & settings, const ExecutableSettings & settings,
const std::vector<ASTPtr> & input_queries, const std::vector<ASTPtr> & input_queries,
const ColumnsDescription & columns, const ColumnsDescription & columns,
const ConstraintsDescription & constraints); const ConstraintsDescription & constraints,
const String & comment);
String getName() const override String getName() const override
{ {

View File

@ -1289,6 +1289,7 @@ StoragePtr create(const StorageFactory::Arguments & args)
StorageInMemoryMetadata metadata; StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns); metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints); metadata.setConstraints(args.constraints);
metadata.setComment(args.comment);
if (!args.storage_def->primary_key) if (!args.storage_def->primary_key)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageKeeperMap requires one column in primary key"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageKeeperMap requires one column in primary key");

View File

@ -50,6 +50,7 @@ StorageSQLite::StorageSQLite(
const String & remote_table_name_, const String & remote_table_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_) ContextPtr context_)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_->getGlobalContext()) , WithContext(context_->getGlobalContext())
@ -71,6 +72,7 @@ StorageSQLite::StorageSQLite(
storage_metadata.setConstraints(constraints_); storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
storage_metadata.setComment(comment);
} }
@ -211,7 +213,7 @@ void registerStorageSQLite(StorageFactory & factory)
auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */ args.mode <= LoadingStrictnessLevel::CREATE); auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */ args.mode <= LoadingStrictnessLevel::CREATE);
return std::make_shared<StorageSQLite>(args.table_id, sqlite_db, database_path, return std::make_shared<StorageSQLite>(args.table_id, sqlite_db, database_path,
table_name, args.columns, args.constraints, args.getContext()); table_name, args.columns, args.constraints, args.comment, args.getContext());
}, },
{ {
.supports_schema_inference = true, .supports_schema_inference = true,

View File

@ -27,6 +27,7 @@ public:
const String & remote_table_name_, const String & remote_table_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_); ContextPtr context_);
std::string getName() const override { return "SQLite"; } std::string getName() const override { return "SQLite"; }

View File

@ -1188,6 +1188,7 @@ StorageWindowView::StorageWindowView(
ContextPtr context_, ContextPtr context_,
const ASTCreateQuery & query, const ASTCreateQuery & query,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
LoadingStrictnessLevel mode) LoadingStrictnessLevel mode)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_->getGlobalContext()) , WithContext(context_->getGlobalContext())
@ -1206,6 +1207,7 @@ StorageWindowView::StorageWindowView(
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
/// If the target table is not set, use inner target table /// If the target table is not set, use inner target table
@ -1764,7 +1766,7 @@ void registerStorageWindowView(StorageFactory & factory)
"Experimental WINDOW VIEW feature " "Experimental WINDOW VIEW feature "
"is not enabled (the setting 'allow_experimental_window_view')"); "is not enabled (the setting 'allow_experimental_window_view')");
return std::make_shared<StorageWindowView>(args.table_id, args.getLocalContext(), args.query, args.columns, args.mode); return std::make_shared<StorageWindowView>(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment, args.mode);
}); });
} }

View File

@ -111,6 +111,7 @@ public:
ContextPtr context_, ContextPtr context_,
const ASTCreateQuery & query, const ASTCreateQuery & query,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const String & comment,
LoadingStrictnessLevel mode); LoadingStrictnessLevel mode);
String getName() const override { return "WindowView"; } String getName() const override { return "WindowView"; }

View File

@ -170,7 +170,14 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/,
if (settings_query != nullptr) if (settings_query != nullptr)
settings.applyChanges(settings_query->as<ASTSetQuery>()->changes); settings.applyChanges(settings_query->as<ASTSetQuery>()->changes);
auto storage = std::make_shared<StorageExecutable>(storage_id, format, settings, input_queries, getActualTableStructure(context, is_insert_query), ConstraintsDescription{}); auto storage = std::make_shared<StorageExecutable>(
storage_id,
format,
settings,
input_queries,
getActualTableStructure(context, is_insert_query),
ConstraintsDescription{},
/* comment = */ "");
storage->startup(); storage->startup();
return storage; return storage;
} }

View File

@ -57,7 +57,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/,
sqlite_db, sqlite_db,
database_path, database_path,
remote_table_name, remote_table_name,
cached_columns, ConstraintsDescription{}, context); cached_columns, ConstraintsDescription{}, /* comment = */ "", context);
storage->startup(); storage->startup();
return storage; return storage;

View File

@ -433,6 +433,7 @@ def test_broken_ignored_replicated(cluster):
check(node, table_name, 1) check(node, table_name, 1)
create_table(node, table_name2, 2, table_name) create_table(node, table_name2, 2, table_name)
node.query(f"system sync replica {table_name}")
check(node, table_name2, 1) check(node, table_name2, 1)
break_projection(node, table_name, "proj1", "all_0_0_0", "data") break_projection(node, table_name, "proj1", "all_0_0_0", "data")

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-random-settings, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -5,59 +5,72 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh . "$CUR_DIR"/../shell_config.sh
function elapsed_sec() set -e
function wait_until()
{ {
local expr=$1 && shift local expr=$1 && shift
local start end
start=$(date +%s.%N)
while ! eval "$expr"; do while ! eval "$expr"; do
sleep 0.5 sleep 0.5
done done
end=$(date +%s.%N) }
$CLICKHOUSE_LOCAL -q "select floor($end-$start)" function get_buffer_delay()
{
local buffer_insert_id=$1 && shift
$CLICKHOUSE_CLIENT -nm -q "
SYSTEM FLUSH LOGS;
WITH
(SELECT event_time_microseconds FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryStart' AND query_id = '$buffer_insert_id') AS begin_,
(SELECT max(event_time) FROM data_01256) AS end_
SELECT dateDiff('seconds', begin_, end_)::UInt64;
"
} }
$CLICKHOUSE_CLIENT -nm -q " $CLICKHOUSE_CLIENT -nm -q "
drop table if exists data_01256; drop table if exists data_01256;
drop table if exists buffer_01256; drop table if exists buffer_01256;
create table data_01256 as system.numbers Engine=Memory(); create table data_01256 (key UInt64, event_time DateTime(6) MATERIALIZED now64(6)) Engine=Memory();
" "
echo "min" echo "min"
$CLICKHOUSE_CLIENT -nm -q " $CLICKHOUSE_CLIENT -q "
create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1,
2, 100, /* time */ 2, 100, /* time */
4, 100, /* rows */ 4, 100, /* rows */
1, 1e6 /* bytes */ 1, 1e6 /* bytes */
); )
insert into buffer_01256 select * from system.numbers limit 5;
select count() from data_01256;
" "
sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 5 ]]') min_query_id=$(random_str 10)
$CLICKHOUSE_CLIENT --query_id="$min_query_id" -q "insert into buffer_01256 select * from system.numbers limit 5"
$CLICKHOUSE_CLIENT -q "select count() from data_01256"
wait_until '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 5 ]]'
sec=$(get_buffer_delay "$min_query_id")
[[ $sec -ge 2 ]] || echo "Buffer flushed too early, min_time=2, flushed after $sec sec" [[ $sec -ge 2 ]] || echo "Buffer flushed too early, min_time=2, flushed after $sec sec"
[[ $sec -lt 100 ]] || echo "Buffer flushed too late, max_time=100, flushed after $sec sec" [[ $sec -lt 100 ]] || echo "Buffer flushed too late, max_time=100, flushed after $sec sec"
$CLICKHOUSE_CLIENT -q "select count() from data_01256" $CLICKHOUSE_CLIENT -q "select count() from data_01256"
$CLICKHOUSE_CLIENT -q "drop table buffer_01256" $CLICKHOUSE_CLIENT -q "drop table buffer_01256"
echo "max" echo "max"
$CLICKHOUSE_CLIENT -nm -q " $CLICKHOUSE_CLIENT -q "
create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1,
100, 2, /* time */ 100, 2, /* time */
0, 100, /* rows */ 0, 100, /* rows */
0, 1e6 /* bytes */ 0, 1e6 /* bytes */
); );
insert into buffer_01256 select * from system.numbers limit 5;
select count() from data_01256;
" "
sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 10 ]]') max_query_id=$(random_str 10)
$CLICKHOUSE_CLIENT --query_id="$max_query_id" -q "insert into buffer_01256 select * from system.numbers limit 5"
$CLICKHOUSE_CLIENT -q "select count() from data_01256"
wait_until '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 10 ]]'
sec=$(get_buffer_delay "$max_query_id")
[[ $sec -ge 2 ]] || echo "Buffer flushed too early, max_time=2, flushed after $sec sec" [[ $sec -ge 2 ]] || echo "Buffer flushed too early, max_time=2, flushed after $sec sec"
$CLICKHOUSE_CLIENT -q "select count() from data_01256" $CLICKHOUSE_CLIENT -q "select count() from data_01256"
$CLICKHOUSE_CLIENT -q "drop table buffer_01256" $CLICKHOUSE_CLIENT -q "drop table buffer_01256"
echo "direct" echo "direct"
$CLICKHOUSE_CLIENT -nm -q " $CLICKHOUSE_CLIENT -nm -q "
create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, create table buffer_01256 (key UInt64) Engine=Buffer(currentDatabase(), data_01256, 1,
100, 100, /* time */ 100, 100, /* time */
0, 9, /* rows */ 0, 9, /* rows */
0, 1e6 /* bytes */ 0, 1e6 /* bytes */

View File

@ -1,4 +1,8 @@
t1 this is a temtorary table t1 this is a temporary table
t2 this is a MergeTree table t2 this is a MergeTree table
t3 this is a Log table t3 this is a Log table
CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temtorary table\' t4 this is a Kafka table
t5 this is a EmbeddedRocksDB table
t6 this is a Executable table
t7 this is a WindowView table
CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temporary table\'

View File

@ -1,4 +1,4 @@
-- Tags: no-parallel -- Tags: no-parallel, no-fasttest
DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2;
@ -9,7 +9,7 @@ CREATE TABLE t1
`n` Int8 `n` Int8
) )
ENGINE = Memory ENGINE = Memory
COMMENT 'this is a temtorary table'; COMMENT 'this is a temporary table';
CREATE TABLE t2 CREATE TABLE t2
( (
@ -26,14 +26,57 @@ CREATE TABLE t3
ENGINE = Log ENGINE = Log
COMMENT 'this is a Log table'; COMMENT 'this is a Log table';
CREATE TABLE t4
(
`n` Int8
)
ENGINE = Kafka
SETTINGS
kafka_broker_list = 'localhost:10000',
kafka_topic_list = 'test',
kafka_group_name = 'test',
kafka_format = 'JSONEachRow'
COMMENT 'this is a Kafka table';
CREATE TABLE t5
(
`n` Int8
)
ENGINE = EmbeddedRocksDB
PRIMARY KEY n
COMMENT 'this is a EmbeddedRocksDB table';
CREATE TABLE t6
(
`n` Int8
)
ENGINE = Executable('script.py', TabSeparated)
COMMENT 'this is a Executable table';
SET allow_experimental_window_view = 1;
-- New analyzer doesn't support WindowView tables
SET allow_experimental_analyzer = 0;
CREATE WINDOW VIEW t7
(
`n` Int8
)
ENGINE MergeTree
ORDER BY n
AS SELECT 1
GROUP BY tumble(now(), toIntervalDay('1'))
COMMENT 'this is a WindowView table';
SET allow_experimental_analyzer = 1;
SELECT SELECT
name, name,
comment comment
FROM system.tables FROM system.tables
WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name; WHERE name IN ('t1', 't2', 't3', 't4', 't5', 't6', 't7')
AND database = currentDatabase() order by name;
SHOW CREATE TABLE t1; SHOW CREATE TABLE t1;
DROP TABLE t1; DROP TABLE t1, t2, t3, t4, t5, t6;
DROP TABLE t2; DROP VIEW t7;
DROP TABLE t3;

View File

@ -7,7 +7,7 @@ create table rmt2 (n int, m int, k int) engine=ReplicatedMergeTree('/test/02446/
settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, old_parts_lifetime=0, cleanup_delay_period=0, max_cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0; settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, old_parts_lifetime=0, cleanup_delay_period=0, max_cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0;
-- FIXME zero-copy locks may remain in ZooKeeper forever if we failed to insert a part. -- FIXME zero-copy locks may remain in ZooKeeper forever if we failed to insert a part.
-- Probably that's why we have to replace repsistent lock with ephemeral sometimes. -- Probably that's why we have to replace persistent lock with ephemeral sometimes.
-- See also "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss" -- See also "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss"
-- in StorageReplicatedMergeTree::createZeroCopyLockNode -- in StorageReplicatedMergeTree::createZeroCopyLockNode
set insert_keeper_fault_injection_probability=0; set insert_keeper_fault_injection_probability=0;
@ -23,6 +23,10 @@ select sleepEachRow(0.5) as test_does_not_rely_on_this;
insert into rmt1 values(5, 5, 5); insert into rmt1 values(5, 5, 5);
alter table rmt2 update m = m * 10 where 1 settings mutations_sync=2; alter table rmt2 update m = m * 10 where 1 settings mutations_sync=2;
-- wait for parts to be merged
select throwIf(name = 'all_0_5_1_6') from system.parts where database=currentDatabase() and table like 'rmt%' and active
format Null; -- { retry 30 until serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
system sync replica rmt2; system sync replica rmt2;
set optimize_throw_if_noop=1; set optimize_throw_if_noop=1;
optimize table rmt2 final; optimize table rmt2 final;
@ -32,10 +36,10 @@ select 1, * from rmt1 order by n;
system sync replica rmt1; system sync replica rmt1;
select 2, * from rmt2 order by n; select 2, * from rmt2 order by n;
-- a funny way to wait for outdated parts to be removed -- wait for outdated parts to be removed (do not ignore _state column, so it will count Deleting parts as well)
select sleep(1), sleepEachRow(0.1) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( select throwIf(count() = 0), groupArray(_state) from (
'select *, _state from system.parts where database=''' || currentDatabase() || ''' and table like ''rmt%'' and active=0' select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0
), 'LineAsString', 's String') settings max_threads=1 format Null; ) format Null; -- { retry 30 until serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0; select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0;

View File

@ -1,4 +1,10 @@
CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog;
SELECT count() FROM mysql(mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', ''), '127.0.0.1:9004', currentDatabase(), 'foo', '', ''); -- { serverError UNKNOWN_FUNCTION } SELECT count() FROM mysql(
SELECT count() FROM mysql(mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', '', SETTINGS connection_pool_size = 1), '127.0.0.1:9004', currentDatabase(), 'foo', '', ''); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', ''),
'127.0.0.1:9004', currentDatabase(), 'foo', '', '',
SETTINGS connect_timeout = 100, connection_wait_timeout = 100, read_write_timeout = 300); -- { serverError UNKNOWN_FUNCTION }
SELECT count() FROM mysql(
mysql('127.0.0.1:9004', currentDatabase(), 'foo', 'default', '', SETTINGS connection_pool_size = 1),
'127.0.0.1:9004', currentDatabase(), 'foo', '', '',
SETTINGS connect_timeout = 100, connection_wait_timeout = 100, read_write_timeout = 300); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD }

View File

@ -20,3 +20,5 @@ source_table_stripe_log
source_table_tiny_log source_table_tiny_log
=== DICTIONARIES IN test_truncate_database === === DICTIONARIES IN test_truncate_database ===
dest_dictionary dest_dictionary
new tables
new_table

View File

@ -73,4 +73,8 @@ SELECT * FROM dest_dictionary; -- {serverError UNKNOWN_TABLE}
SHOW TABLES FROM test_truncate_database; SHOW TABLES FROM test_truncate_database;
SHOW DICTIONARIES FROM test_truncate_database; SHOW DICTIONARIES FROM test_truncate_database;
CREATE TABLE new_table (x UInt16) ENGINE = MergeTree ORDER BY x;
select 'new tables';
SHOW TABLES FROM test_truncate_database;
DROP DATABASE test_truncate_database; DROP DATABASE test_truncate_database;

View File

@ -1,10 +1,10 @@
Creating 300 tables Creating 50 tables
900 1 1 150 1 1
900 1 1 150 1 1
900 1 1 150 1 1
900 1 1 150 1 1
Making 200 requests to system.replicas Making 100 requests to system.replicas
Query system.replicas while waiting for other concurrent requests to finish Query system.replicas while waiting for other concurrent requests to finish
0 0
900 150
1 1

View File

@ -7,8 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
set -e set -e
NUM_TABLES=300 NUM_TABLES=50
CONCURRENCY=200 CONCURRENCY=100
echo "Creating $NUM_TABLES tables" echo "Creating $NUM_TABLES tables"
@ -46,10 +46,10 @@ wait;
# Check results with different max_block_size # Check results with different max_block_size
$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase()' $CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase()'
$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=1' $CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=1'
$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=77' $CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=77'
$CLICKHOUSE_CLIENT -q 'SELECT count(), sum(total_replicas) >= 2700, sum(active_replicas) >= 2700 FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=11111' $CLICKHOUSE_CLIENT -q 'SELECT count() as c, sum(total_replicas) >= 3*c, sum(active_replicas) >= 3*c FROM system.replicas WHERE database=currentDatabase() SETTINGS max_block_size=11111'
echo "Making $CONCURRENCY requests to system.replicas" echo "Making $CONCURRENCY requests to system.replicas"
@ -70,8 +70,8 @@ wait;
$CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq "
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
-- without optimisation there are ~350K zk requests -- Check that number of ZK request is less then a half of (total replicas * concurrency)
SELECT sum(ProfileEvents['ZooKeeperTransactions']) < 30000 SELECT sum(ProfileEvents['ZooKeeperTransactions']) < (${NUM_TABLES} * 3 * ${CONCURRENCY} / 2)
FROM system.query_log FROM system.query_log
WHERE current_database=currentDatabase() AND log_comment='02908_many_requests'; WHERE current_database=currentDatabase() AND log_comment='02908_many_requests';
" "

View File

@ -2,7 +2,7 @@
set allow_experimental_dynamic_type=1; set allow_experimental_dynamic_type=1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -2,7 +2,7 @@
set allow_experimental_dynamic_type=1; set allow_experimental_dynamic_type=1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -2,7 +2,7 @@
set allow_experimental_dynamic_type=1; set allow_experimental_dynamic_type=1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);
@ -13,7 +13,8 @@ insert into test select number, toDateTime(number) from numbers(50000);
insert into test select number, NULL from numbers(100000); insert into test select number, NULL from numbers(100000);
select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d);
system start merges test; optimize table test final;; system start merges test;
optimize table test final;
select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d);
system stop merges test; system stop merges test;

View File

@ -2,7 +2,7 @@
set allow_experimental_dynamic_type=1; set allow_experimental_dynamic_type=1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -3,7 +3,7 @@
set allow_experimental_dynamic_type = 1; set allow_experimental_dynamic_type = 1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(1000000); insert into test select number, number from numbers(1000000);
insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000);

View File

@ -3,7 +3,7 @@
set allow_experimental_dynamic_type = 1; set allow_experimental_dynamic_type = 1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(1000000); insert into test select number, number from numbers(1000000);
insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000);

View File

@ -3,7 +3,7 @@
set allow_experimental_dynamic_type = 1; set allow_experimental_dynamic_type = 1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(1000000); insert into test select number, number from numbers(1000000);
insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000);

View File

@ -3,7 +3,7 @@
set allow_experimental_dynamic_type = 1; set allow_experimental_dynamic_type = 1;
drop table if exists test; drop table if exists test;
create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(1000000); insert into test select number, number from numbers(1000000);
insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000); insert into test select number, 'str_' || toString(number) from numbers(1000000, 1000000);

View File

@ -6,7 +6,7 @@ set allow_experimental_dynamic_type = 1;
set enable_named_columns_in_function_tuple = 0; set enable_named_columns_in_function_tuple = 0;
drop table if exists test;; drop table if exists test;;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -6,7 +6,7 @@ set allow_experimental_dynamic_type = 1;
set enable_named_columns_in_function_tuple = 0; set enable_named_columns_in_function_tuple = 0;
drop table if exists test;; drop table if exists test;;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -6,7 +6,7 @@ set allow_experimental_dynamic_type = 1;
set enable_named_columns_in_function_tuple = 0; set enable_named_columns_in_function_tuple = 0;
drop table if exists test;; drop table if exists test;;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -6,7 +6,7 @@ set allow_experimental_dynamic_type = 1;
set enable_named_columns_in_function_tuple = 0; set enable_named_columns_in_function_tuple = 0;
drop table if exists test;; drop table if exists test;;
create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, lock_acquire_timeout_for_background_operations=600;
system stop merges test; system stop merges test;
insert into test select number, number from numbers(100000); insert into test select number, number from numbers(100000);

View File

@ -6,8 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -nm -q " $CLICKHOUSE_CLIENT -nm -q "
drop table if exists tp_1; drop table if exists tp_1;
create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100); create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100) settings max_parts_to_merge_at_once=1;
system stop merges tp_1;
insert into tp_1 select number, number from numbers(3); insert into tp_1 select number, number from numbers(3);
set mutations_sync = 2; set mutations_sync = 2;
@ -39,7 +38,6 @@ $CLICKHOUSE_CLIENT -nm -q "
set send_logs_level='fatal'; set send_logs_level='fatal';
drop table tp_1; drop table tp_1;
restore table tp_1 from Disk('backups', '$backup_id'); restore table tp_1 from Disk('backups', '$backup_id');
system stop merges tp_1;
" | grep -o "RESTORED" " | grep -o "RESTORED"
$CLICKHOUSE_CLIENT -q "select count() from tp_1;" $CLICKHOUSE_CLIENT -q "select count() from tp_1;"

View File

@ -0,0 +1,5 @@
set allow_experimental_variant_type=1;
set use_variant_as_common_type=1;
SELECT * FROM numbers([tuple(1, 2), NULL], 2); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}

View File

@ -0,0 +1,12 @@
0 1 1 1 0 0 0
1 3 3 3 2 2 2
2 5 5 5 4 4 4
3 7 7 7 6 6 6
4 9 9 9 8 8 8
5 11 11 11 10 10 10
6 13 13 13 12 12 12
7 15 15 15 14 14 14
8 17 17 17 16 16 16
9 19 19 19 18 18 18
0 107351244 107351244 107351244 107354520 107354520 107354520

View File

@ -0,0 +1,19 @@
SELECT
intDiv(number, 2) AS k,
sumArgMax(number, number % 20),
sumArgMax(number, leftPad(toString(number % 20), 5, '0')), -- Pad with 0 to preserve number ordering
sumArgMax(number, [number % 20, number % 20]),
sumArgMin(number, number % 20),
sumArgMin(number, leftPad(toString(number % 20), 5, '0')),
sumArgMin(number, [number % 20, number % 20]),
FROM
(
SELECT number
FROM system.numbers
LIMIT 65537
)
GROUP BY k
WITH TOTALS
ORDER BY k ASC
LIMIT 10
SETTINGS group_by_overflow_mode = 'any', totals_mode = 'before_having', max_rows_to_group_by = 100000;

View File

@ -1622,6 +1622,7 @@ filesystem
filesystemAvailable filesystemAvailable
filesystemCapacity filesystemCapacity
filesystemFree filesystemFree
filesystemUnreserved
filesystems filesystems
finalizeAggregation finalizeAggregation
fips fips