Merge branch 'master' into add-numactl

This commit is contained in:
Antonio Andelic 2024-07-28 11:29:26 +02:00
commit 6f2d25cc39
211 changed files with 2477 additions and 1669 deletions

View File

@ -64,7 +64,7 @@
* Print stacktrace and diagnostic info if `clickhouse-client` or `clickhouse-local` crashes. [#61109](https://github.com/ClickHouse/ClickHouse/pull/61109) ([Alexander Tokmakov](https://github.com/tavplubix)).
* The result of `SHOW INDEX | INDEXES | INDICES | KEYS` was previously sorted by the primary key column names. Since this was unintuitive, the result is now sorted by the position of the primary key columns within the primary key. [#61131](https://github.com/ClickHouse/ClickHouse/pull/61131) ([Robert Schulze](https://github.com/rschu1ze)).
* Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)).
* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_ignore_key_case`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)).
* Allow matching column names in a case insensitive manner when reading json files (`input_format_json_case_insensitive_column_matching`). [#61750](https://github.com/ClickHouse/ClickHouse/pull/61750) ([kevinyhzou](https://github.com/KevinyhZou)).
* Support reading partitioned data DeltaLake data. Infer DeltaLake schema by reading metadata instead of data. [#63201](https://github.com/ClickHouse/ClickHouse/pull/63201) ([Kseniia Sumarokova](https://github.com/kssenii)).
* In composable protocols TLS layer accepted only `certificateFile` and `privateKeyFile` parameters. https://clickhouse.com/docs/en/operations/settings/composable-protocols. [#63985](https://github.com/ClickHouse/ClickHouse/pull/63985) ([Anton Ivashkin](https://github.com/ianton-ru)).
* Added profile event `SelectQueriesWithPrimaryKeyUsage` which indicates how many SELECT queries use the primary key to evaluate the WHERE clause. [#64492](https://github.com/ClickHouse/ClickHouse/pull/64492) ([0x01f](https://github.com/0xfei)).

View File

@ -261,9 +261,12 @@ function timeout_with_logging() {
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout"
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code

View File

@ -251,9 +251,12 @@ function timeout_with_logging() {
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout"
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code

View File

@ -247,12 +247,22 @@ function run_tests()
try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')"
TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800))
START_TIME=${SECONDS}
set +e
timeout -k 60m -s TERM --preserve-status 140m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s \
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
set -e
DURATION=$((START_TIME - SECONDS))
echo "Elapsed ${DURATION} seconds."
if [[ $DURATION -ge $TIMEOUT ]]
then
echo "It looks like the command is terminated by the timeout, which is ${TIMEOUT} seconds."
fi
}
export -f run_tests
@ -264,7 +274,7 @@ if [ "$NUM_TRIES" -gt "1" ]; then
# We don't run tests with Ordinary database in PRs, only in master.
# So run new/changed tests with Ordinary at least once in flaky check.
timeout_with_logging "$TIMEOUT" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \
| sed 's/All tests have finished//' | sed 's/No tests were run//' ||:
| sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||:
fi
timeout_with_logging "$TIMEOUT" bash -c run_tests ||:

View File

@ -45,9 +45,12 @@ function timeout_with_logging() {
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout"
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code

View File

@ -12,6 +12,7 @@ UNKNOWN_SIGN = "[ UNKNOWN "
SKIPPED_SIGN = "[ SKIPPED "
HUNG_SIGN = "Found hung queries in processlist"
SERVER_DIED_SIGN = "Server died, terminating all processes"
SERVER_DIED_SIGN2 = "Server does not respond to health check"
DATABASE_SIGN = "Database: "
SUCCESS_FINISH_SIGNS = ["All tests have finished", "No tests were run"]
@ -43,7 +44,7 @@ def process_test_log(log_path, broken_tests):
if HUNG_SIGN in line:
hung = True
break
if SERVER_DIED_SIGN in line:
if SERVER_DIED_SIGN in line or SERVER_DIED_SIGN2 in line:
server_died = True
if RETRIES_SIGN in line:
retries = True
@ -111,12 +112,12 @@ def process_test_log(log_path, broken_tests):
# Python does not support TSV, so we have to escape '\t' and '\n' manually
# and hope that complex escape sequences will not break anything
test_results = [
(
[
test[0],
test[1],
test[2],
"".join(test[3])[:4096].replace("\t", "\\t").replace("\n", "\\n"),
)
]
for test in test_results
]
@ -170,18 +171,23 @@ def process_result(result_path, broken_tests):
if hung:
description = "Some queries hung, "
state = "failure"
test_results.append(("Some queries hung", "FAIL", "0", ""))
test_results.append(["Some queries hung", "FAIL", "0", ""])
elif server_died:
description = "Server died, "
state = "failure"
test_results.append(("Server died", "FAIL", "0", ""))
# When ClickHouse server crashes, some tests are still running
# and fail because they cannot connect to server
for result in test_results:
if result[1] == "FAIL":
result[1] = "SERVER_DIED"
test_results.append(["Server died", "FAIL", "0", ""])
elif not success_finish:
description = "Tests are not finished, "
state = "failure"
test_results.append(("Tests are not finished", "FAIL", "0", ""))
test_results.append(["Tests are not finished", "FAIL", "0", ""])
elif retries:
description = "Some tests restarted, "
test_results.append(("Some tests restarted", "SKIPPED", "0", ""))
test_results.append(["Some tests restarted", "SKIPPED", "0", ""])
else:
description = ""
@ -233,11 +239,12 @@ if __name__ == "__main__":
# sort by status then by check name
order = {
"FAIL": 0,
"Timeout": 1,
"NOT_FAILED": 2,
"BROKEN": 3,
"OK": 4,
"SKIPPED": 5,
"SERVER_DIED": 1,
"Timeout": 2,
"NOT_FAILED": 3,
"BROKEN": 4,
"OK": 5,
"SKIPPED": 6,
}
return order.get(item[1], 10), str(item[0]), item[1]

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**
```sql
filesystemFree()
filesystemUnreserved()
```
**Returned value**
@ -2121,7 +2121,7 @@ filesystemFree()
Query:
```sql
SELECT formatReadableSize(filesystemFree()) AS "Free space";
SELECT formatReadableSize(filesystemUnreserved()) AS "Free space";
```
Result:
@ -2449,11 +2449,11 @@ As you can see, `runningAccumulate` merges states for each group of rows separat
## joinGet
The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md).
Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key.
The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key.
:::note
Only supports tables created with the `ENGINE = Join(ANY, LEFT, <join_keys>)` statement.
:::
**Syntax**
@ -2463,26 +2463,32 @@ joinGet(join_storage_table_name, `value_column`, join_keys)
**Arguments**
- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. The identifier is searched in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example.
- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed.
- `value_column` — name of the column of the table that contains required data.
- `join_keys` — list of keys.
:::note
The identifier is searched for in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example.
:::
**Returned value**
Returns a list of values corresponded to list of keys.
If certain does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting.
- Returns a list of values corresponded to the list of keys.
:::note
If a certain key does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting during table creation.
More info about `join_use_nulls` in [Join operation](../../engines/table-engines/special/join.md).
:::
**Example**
Input table:
```sql
CREATE DATABASE db_test
CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1
INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13)
CREATE DATABASE db_test;
CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id);
INSERT INTO db_test.id_val VALUES (1, 11)(2, 12)(4, 13);
SELECT * FROM db_test.id_val;
```
```text
@ -2496,18 +2502,116 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13)
Query:
```sql
SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1
SELECT number, joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4);
```
Result:
```text
┌─joinGet(db_test.id_val, 'val', toUInt32(number))─┐
│ 0 │
│ 11 │
│ 12 │
│ 0 │
└──────────────────────────────────────────────────┘
┌─number─┬─joinGet('db_test.id_val', 'val', toUInt32(number))─┐
1. │ 0 │ 0 │
2. │ 1 │ 11 │
3. │ 2 │ 12 │
4. │ 3 │ 0 │
└────────┴────────────────────────────────────────────────────┘
```
Setting `join_use_nulls` can be used during table creation to change the behaviour of what gets returned if no key exists in the source table.
```sql
CREATE DATABASE db_test;
CREATE TABLE db_test.id_val_nulls(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls=1;
INSERT INTO db_test.id_val_nulls VALUES (1, 11)(2, 12)(4, 13);
SELECT * FROM db_test.id_val_nulls;
```
```text
┌─id─┬─val─┐
│ 4 │ 13 │
│ 2 │ 12 │
│ 1 │ 11 │
└────┴─────┘
```
Query:
```sql
SELECT number, joinGet(db_test.id_val_nulls, 'val', toUInt32(number)) from numbers(4);
```
Result:
```text
┌─number─┬─joinGet('db_test.id_val_nulls', 'val', toUInt32(number))─┐
1. │ 0 │ ᴺᵁᴸᴸ │
2. │ 1 │ 11 │
3. │ 2 │ 12 │
4. │ 3 │ ᴺᵁᴸᴸ │
└────────┴──────────────────────────────────────────────────────────┘
```
## joinGetOrNull
Like [joinGet](#joinget) but returns `NULL` when the key is missing instead of returning the default value.
**Syntax**
```sql
joinGetOrNull(join_storage_table_name, `value_column`, join_keys)
```
**Arguments**
- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed.
- `value_column` — name of the column of the table that contains required data.
- `join_keys` — list of keys.
:::note
The identifier is searched for in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example.
:::
**Returned value**
- Returns a list of values corresponded to the list of keys.
:::note
If a certain key does not exist in source table then `NULL` is returned for that key.
:::
**Example**
Input table:
```sql
CREATE DATABASE db_test;
CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id);
INSERT INTO db_test.id_val VALUES (1, 11)(2, 12)(4, 13);
SELECT * FROM db_test.id_val;
```
```text
┌─id─┬─val─┐
│ 4 │ 13 │
│ 2 │ 12 │
│ 1 │ 11 │
└────┴─────┘
```
Query:
```sql
SELECT number, joinGetOrNull(db_test.id_val, 'val', toUInt32(number)) from numbers(4);
```
Result:
```text
┌─number─┬─joinGetOrNull('db_test.id_val', 'val', toUInt32(number))─┐
1. │ 0 │ ᴺᵁᴸᴸ │
2. │ 1 │ 11 │
3. │ 2 │ 12 │
4. │ 3 │ ᴺᵁᴸᴸ │
└────────┴──────────────────────────────────────────────────────────┘
```
## catboostEvaluate

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],
...
) 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.
@ -626,11 +626,6 @@ SELECT * FROM base.t1;
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**
``` sql

View File

@ -16,6 +16,7 @@ Syntax:
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
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.
@ -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]
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
AS SELECT ...
[COMMENT 'comment']
```
:::tip
@ -161,6 +163,7 @@ RANDOMIZE FOR interval
DEPENDS ON [db.]name [, [db.]name [, ...]]
[TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY]
AS SELECT ...
[COMMENT 'comment']
```
where `interval` is a sequence of simple intervals:
```sql
@ -267,7 +270,10 @@ This is an experimental feature that may change in backwards-incompatible ways i
:::
``` 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.

View File

@ -209,8 +209,8 @@ std::vector<String> Client::loadWarningMessages()
{} /* query_parameters */,
"" /* query_id */,
QueryProcessingStage::Complete,
&global_context->getSettingsRef(),
&global_context->getClientInfo(), false, {});
&client_context->getSettingsRef(),
&client_context->getClientInfo(), false, {});
while (true)
{
Packet packet = connection->receivePacket();
@ -306,9 +306,6 @@ void Client::initialize(Poco::Util::Application & self)
if (env_password && !config().has("password"))
config().setString("password", env_password);
// global_context->setApplicationType(Context::ApplicationType::CLIENT);
global_context->setQueryParameters(query_parameters);
/// settings and limits could be specified in config file, but passed settings has higher priority
for (const auto & setting : global_context->getSettingsRef().allUnchanged())
{
@ -382,7 +379,7 @@ try
showWarnings();
/// Set user password complexity rules
auto & access_control = global_context->getAccessControl();
auto & access_control = client_context->getAccessControl();
access_control.setPasswordComplexityRules(connection->getPasswordComplexityRules());
if (is_interactive && !delayed_interactive)
@ -459,7 +456,7 @@ void Client::connect()
<< connection_parameters.host << ":" << connection_parameters.port
<< (!connection_parameters.user.empty() ? " as user " + connection_parameters.user : "") << "." << std::endl;
connection = Connection::createConnection(connection_parameters, global_context);
connection = Connection::createConnection(connection_parameters, client_context);
if (max_client_network_bandwidth)
{
@ -528,7 +525,7 @@ void Client::connect()
}
}
if (!global_context->getSettingsRef().use_client_time_zone)
if (!client_context->getSettingsRef().use_client_time_zone)
{
const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts);
if (!time_zone.empty())
@ -611,7 +608,7 @@ void Client::printChangedSettings() const
}
};
print_changes(global_context->getSettingsRef().changes(), "settings");
print_changes(client_context->getSettingsRef().changes(), "settings");
print_changes(cmd_merge_tree_settings.changes(), "MergeTree settings");
}
@ -709,7 +706,7 @@ bool Client::processWithFuzzing(const String & full_query)
{
const char * begin = full_query.data();
orig_ast = parseQuery(begin, begin + full_query.size(),
global_context->getSettingsRef(),
client_context->getSettingsRef(),
/*allow_multi_statements=*/ true);
}
catch (const Exception & e)
@ -733,7 +730,7 @@ bool Client::processWithFuzzing(const String & full_query)
}
// Kusto is not a subject for fuzzing (yet)
if (global_context->getSettingsRef().dialect == DB::Dialect::kusto)
if (client_context->getSettingsRef().dialect == DB::Dialect::kusto)
{
return true;
}
@ -1166,6 +1163,11 @@ void Client::processOptions(const OptionsDescription & options_description,
if (options.count("opentelemetry-tracestate"))
global_context->getClientTraceContext().tracestate = options["opentelemetry-tracestate"].as<std::string>();
/// In case of clickhouse-client the `client_context` can be just an alias for the `global_context`.
/// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.)
client_context = global_context;
initClientContext();
}
@ -1205,11 +1207,6 @@ void Client::processConfig()
pager = config().getString("pager", "");
setDefaultFormatsAndCompressionFromConfiguration();
global_context->setClientName(std::string(DEFAULT_CLIENT_NAME));
global_context->setQueryKindInitial();
global_context->setQuotaClientKey(config().getString("quota_key", ""));
global_context->setQueryKind(query_kind);
}

View File

@ -16,7 +16,6 @@ public:
int main(const std::vector<String> & /*args*/) override;
protected:
Poco::Util::LayeredConfiguration & getClientConfiguration() override;
bool processWithFuzzing(const String & full_query) override;

View File

@ -295,6 +295,8 @@ void LocalServer::cleanup()
if (suggest)
suggest.reset();
client_context.reset();
if (global_context)
{
global_context->shutdown();
@ -436,7 +438,7 @@ void LocalServer::connect()
in = input.get();
}
connection = LocalConnection::createConnection(
connection_parameters, global_context, in, need_render_progress, need_render_profile_events, server_display_name);
connection_parameters, client_context, in, need_render_progress, need_render_profile_events, server_display_name);
}
@ -497,8 +499,6 @@ try
initTTYBuffer(toProgressOption(getClientConfiguration().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
applyCmdSettings(global_context);
/// try to load user defined executable functions, throw on error and die
try
{
@ -510,6 +510,11 @@ try
throw;
}
/// Must be called after we stopped initializing the global context and changing its settings.
/// After this point the global context must be stayed almost unchanged till shutdown,
/// and all necessary changes must be made to the client context instead.
createClientContext();
if (is_interactive)
{
clearTerminal();
@ -735,6 +740,9 @@ void LocalServer::processConfig()
/// Load global settings from default_profile and system_profile.
global_context->setDefaultProfiles(getClientConfiguration());
/// Command-line parameters can override settings from the default profile.
applyCmdSettings(global_context);
/// We load temporary database first, because projections need it.
DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase();
@ -778,10 +786,6 @@ void LocalServer::processConfig()
server_display_name = getClientConfiguration().getString("display_name", "");
prompt_by_server_display_name = getClientConfiguration().getRawString("prompt_by_server_display_name.default", ":) ");
global_context->setQueryKindInitial();
global_context->setQueryKind(query_kind);
global_context->setQueryParameters(query_parameters);
}
@ -860,6 +864,16 @@ void LocalServer::applyCmdOptions(ContextMutablePtr context)
}
void LocalServer::createClientContext()
{
/// In case of clickhouse-local it's necessary to use a separate context for client-related purposes.
/// We can't just change the global context because it is used in background tasks (for example, in merges)
/// which don't expect that the global context can suddenly change.
client_context = Context::createCopy(global_context);
initClientContext();
}
void LocalServer::processOptions(const OptionsDescription &, const CommandLineOptions & options, const std::vector<Arguments> &, const std::vector<Arguments> &)
{
if (options.count("table"))

View File

@ -31,7 +31,6 @@ public:
int main(const std::vector<String> & /*args*/) override;
protected:
Poco::Util::LayeredConfiguration & getClientConfiguration() override;
void connect() override;
@ -50,7 +49,6 @@ protected:
void processConfig() override;
void readArguments(int argc, char ** argv, Arguments & common_arguments, std::vector<Arguments> &, std::vector<Arguments> &) override;
void updateLoggerLevel(const String & logs_level) override;
private:
@ -67,6 +65,8 @@ private:
void applyCmdOptions(ContextMutablePtr context);
void applyCmdSettings(ContextMutablePtr context);
void createClientContext();
ServerSettings server_settings;
std::optional<StatusFile> status;

View File

@ -1,12 +1,12 @@
#include <cassert>
#include <memory>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
@ -15,18 +15,14 @@
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/KeyHolderHelpers.h>
#include <Core/Field.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <memory>
namespace DB
@ -51,7 +47,7 @@ struct AggregateFunctionGroupArrayIntersectData
};
/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types.
/// Puts all values to the hash set. Returns an array of unique values present in all inputs. Implemented for numeric types.
template <typename T>
class AggregateFunctionGroupArrayIntersect
: public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectData<T>, AggregateFunctionGroupArrayIntersect<T>>
@ -69,7 +65,7 @@ public:
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectData<T>,
AggregateFunctionGroupArrayIntersect<T>>({argument_type}, parameters_, result_type_) {}
String getName() const override { return "GroupArrayIntersect"; }
String getName() const override { return "groupArrayIntersect"; }
bool allocatesMemoryInArena() const override { return false; }
@ -158,7 +154,7 @@ public:
set.reserve(size);
for (size_t i = 0; i < size; ++i)
{
int key;
T key;
readIntBinary(key, buf);
set.insert(key);
}
@ -213,7 +209,7 @@ public:
: IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData, AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>({input_data_type_}, parameters_, result_type_)
, input_data_type(result_type_) {}
String getName() const override { return "GroupArrayIntersect"; }
String getName() const override { return "groupArrayIntersect"; }
bool allocatesMemoryInArena() const override { return true; }
@ -240,7 +236,7 @@ public:
{
const char * begin = nullptr;
StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin);
assert(serialized.data != nullptr);
chassert(serialized.data != nullptr);
set.emplace(SerializedKeyHolder{serialized, *arena}, it, inserted);
}
}
@ -260,7 +256,7 @@ public:
{
const char * begin = nullptr;
StringRef serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin);
assert(serialized.data != nullptr);
chassert(serialized.data != nullptr);
it = set.find(serialized);
if (it != nullptr)

View File

@ -195,7 +195,7 @@ bool SingleValueDataFixed<T>::isEqualTo(const IColumn & column, size_t index) co
template <typename T>
bool SingleValueDataFixed<T>::isEqualTo(const SingleValueDataFixed<T> & to) const
{
return has() && to.value == value;
return has() && to.has() && to.value == value;
}
template <typename T>
@ -904,6 +904,7 @@ bool SingleValueDataNumeric<T>::isEqualTo(const DB::IColumn & column, size_t ind
template <typename T>
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);
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>
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);
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>
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);
return memory.get().setIfSmaller(other.memory.get(), arena);
}
@ -931,6 +934,7 @@ bool SingleValueDataNumeric<T>::setIfSmaller(const DB::SingleValueDataBase & to,
template <typename T>
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);
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
{
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)
@ -1291,7 +1295,7 @@ bool SingleValueDataGeneric::isEqualTo(const IColumn & column, size_t row_num) c
bool SingleValueDataGeneric::isEqualTo(const DB::SingleValueDataBase & other) const
{
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 *)

View File

@ -68,10 +68,13 @@ QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes)
return nullptr;
}
/// Checks if the node is combination of isNull and notEquals functions of two the same arguments
/// Checks if the node is combination of isNull and notEquals functions of two the same arguments:
/// [ (a <> b AND) ] (a IS NULL) AND (b IS NULL)
bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs, QueryTreeNodePtr & rhs)
{
QueryTreeNodePtrWithHashSet all_arguments;
QueryTreeNodePtrWithHashSet is_null_arguments;
for (const auto & node : nodes)
{
const auto * func_node = node->as<FunctionNode>();
@ -80,7 +83,11 @@ bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs,
const auto & arguments = func_node->getArguments().getNodes();
if (func_node->getFunctionName() == "isNull" && arguments.size() == 1)
{
all_arguments.insert(QueryTreeNodePtrWithHash(arguments[0]));
is_null_arguments.insert(QueryTreeNodePtrWithHash(arguments[0]));
}
else if (func_node->getFunctionName() == "notEquals" && arguments.size() == 2)
{
if (arguments[0]->isEqual(*arguments[1]))
@ -95,7 +102,7 @@ bool matchIsNullOfTwoArgs(const QueryTreeNodes & nodes, QueryTreeNodePtr & lhs,
return false;
}
if (all_arguments.size() != 2)
if (all_arguments.size() != 2 || is_null_arguments.size() != 2)
return false;
lhs = all_arguments.begin()->node;

View File

@ -1740,7 +1740,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu
const auto * tuple_data_type = typeid_cast<const DataTypeTuple *>(result_type.get());
if (!tuple_data_type)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Qualified matcher {} find non compound expression {} with type {}. Expected tuple or array of tuples. In scope {}",
"Qualified matcher {} found a non-compound expression {} with type {}. Expected a tuple or an array of tuples. In scope {}",
matcher_node->formatASTForErrorMessage(),
expression_query_tree_node->formatASTForErrorMessage(),
expression_query_tree_node->getResultType()->getName(),

View File

@ -226,6 +226,9 @@ add_object_library(clickhouse_storages_windowview Storages/WindowView)
add_object_library(clickhouse_storages_s3queue Storages/ObjectStorageQueue)
add_object_library(clickhouse_storages_materializedview Storages/MaterializedView)
add_object_library(clickhouse_client Client)
# Always compile this file with the highest possible level of optimizations, even in Debug builds.
# https://github.com/ClickHouse/ClickHouse/issues/65745
set_source_files_properties(Client/ClientBaseOptimizedParts.cpp PROPERTIES COMPILE_FLAGS "-O3")
add_object_library(clickhouse_bridge BridgeHelper)
add_object_library(clickhouse_server Server)
add_object_library(clickhouse_server_http Server/HTTP)

View File

@ -108,7 +108,6 @@ namespace ErrorCodes
extern const int UNEXPECTED_PACKET_FROM_SERVER;
extern const int INVALID_USAGE_OF_INPUT;
extern const int CANNOT_SET_SIGNAL_HANDLER;
extern const int UNRECOGNIZED_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int CANNOT_OPEN_FILE;
extern const int FILE_ALREADY_EXISTS;
@ -478,7 +477,7 @@ void ClientBase::sendExternalTables(ASTPtr parsed_query)
std::vector<ExternalTableDataPtr> data;
for (auto & table : external_tables)
data.emplace_back(table.getData(global_context));
data.emplace_back(table.getData(client_context));
connection->sendExternalTablesData(data);
}
@ -691,10 +690,10 @@ try
/// intermixed with data with parallel formatting.
/// It may increase code complexity significantly.
if (!extras_into_stdout || select_only_into_file)
output_format = global_context->getOutputFormatParallelIfPossible(
output_format = client_context->getOutputFormatParallelIfPossible(
current_format, out_file_buf ? *out_file_buf : *out_buf, block);
else
output_format = global_context->getOutputFormat(
output_format = client_context->getOutputFormat(
current_format, out_file_buf ? *out_file_buf : *out_buf, block);
output_format->setAutoFlush();
@ -773,6 +772,15 @@ void ClientBase::adjustSettings()
global_context->setSettings(settings);
}
void ClientBase::initClientContext()
{
client_context->setClientName(std::string(DEFAULT_CLIENT_NAME));
client_context->setQuotaClientKey(getClientConfiguration().getString("quota_key", ""));
client_context->setQueryKindInitial();
client_context->setQueryKind(query_kind);
client_context->setQueryParameters(query_parameters);
}
bool ClientBase::isRegularFile(int fd)
{
struct stat file_stat;
@ -963,7 +971,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
/// client-side. Thus we need to parse the query.
const char * begin = full_query.data();
auto parsed_query = parseQuery(begin, begin + full_query.size(),
global_context->getSettingsRef(),
client_context->getSettingsRef(),
/*allow_multi_statements=*/ false);
if (!parsed_query)
@ -986,7 +994,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
/// But for asynchronous inserts we don't extract data, because it's needed
/// to be done on server side in that case (for coalescing the data from multiple inserts on server side).
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && isSyncInsertWithData(*insert, global_context))
if (insert && isSyncInsertWithData(*insert, client_context))
query_to_execute = full_query.substr(0, insert->data - full_query.data());
else
query_to_execute = full_query;
@ -1104,7 +1112,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
}
}
const auto & settings = global_context->getSettingsRef();
const auto & settings = client_context->getSettingsRef();
const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1;
int retries_left = 10;
@ -1119,10 +1127,10 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
connection_parameters.timeouts,
query,
query_parameters,
global_context->getCurrentQueryId(),
client_context->getCurrentQueryId(),
query_processing_stage,
&global_context->getSettingsRef(),
&global_context->getClientInfo(),
&client_context->getSettingsRef(),
&client_context->getClientInfo(),
true,
[&](const Progress & progress) { onProgress(progress); });
@ -1309,7 +1317,7 @@ void ClientBase::onProgress(const Progress & value)
void ClientBase::onTimezoneUpdate(const String & tz)
{
global_context->setSetting("session_timezone", tz);
client_context->setSetting("session_timezone", tz);
}
@ -1505,13 +1513,13 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de
void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query)
{
if (!global_context->hasInsertionTable() && insert_query.table)
if (!client_context->hasInsertionTable() && insert_query.table)
{
String table = insert_query.table->as<ASTIdentifier &>().shortName();
if (!table.empty())
{
String database = insert_query.database ? insert_query.database->as<ASTIdentifier &>().shortName() : "";
global_context->setInsertionTable(StorageID(database, table));
client_context->setInsertionTable(StorageID(database, table));
}
}
}
@ -1562,7 +1570,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars
const auto & parsed_insert_query = parsed_query->as<ASTInsertQuery &>();
if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in))))
{
const auto & settings = global_context->getSettingsRef();
const auto & settings = client_context->getSettingsRef();
if (settings.throw_if_no_data_to_insert)
throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert");
else
@ -1576,10 +1584,10 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars
connection_parameters.timeouts,
query,
query_parameters,
global_context->getCurrentQueryId(),
client_context->getCurrentQueryId(),
query_processing_stage,
&global_context->getSettingsRef(),
&global_context->getClientInfo(),
&client_context->getSettingsRef(),
&client_context->getClientInfo(),
true,
[&](const Progress & progress) { onProgress(progress); });
@ -1627,7 +1635,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
/// Set callback to be called on file progress.
if (tty_buf)
progress_indication.setFileProgressCallback(global_context, *tty_buf);
progress_indication.setFileProgressCallback(client_context, *tty_buf);
}
/// If data fetched from file (maybe compressed file)
@ -1661,10 +1669,10 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
}
StorageFile::CommonArguments args{
WithContext(global_context),
WithContext(client_context),
parsed_insert_query->table_id,
current_format,
getFormatSettings(global_context),
getFormatSettings(client_context),
compression_method,
columns_for_storage_file,
ConstraintsDescription{},
@ -1672,7 +1680,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
{},
String{},
};
StoragePtr storage = std::make_shared<StorageFile>(in_file, global_context->getUserFilesPath(), args);
StoragePtr storage = std::make_shared<StorageFile>(in_file, client_context->getUserFilesPath(), args);
storage->startup();
SelectQueryInfo query_info;
@ -1683,16 +1691,16 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
storage->read(
plan,
sample.getNames(),
storage->getStorageSnapshot(metadata, global_context),
storage->getStorageSnapshot(metadata, client_context),
query_info,
global_context,
client_context,
{},
global_context->getSettingsRef().max_block_size,
client_context->getSettingsRef().max_block_size,
getNumberOfPhysicalCPUCores());
auto builder = plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(global_context),
BuildQueryPipelineSettings::fromContext(global_context));
QueryPlanOptimizationSettings::fromContext(client_context),
BuildQueryPipelineSettings::fromContext(client_context));
QueryPlanResourceHolder resources;
auto pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources);
@ -1753,14 +1761,14 @@ void ClientBase::sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDes
current_format = insert->format;
}
auto source = global_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size);
auto source = client_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size);
Pipe pipe(source);
if (columns_description.hasDefaults())
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *source, global_context);
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *source, client_context);
});
}
@ -1922,12 +1930,12 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (is_interactive)
{
global_context->setCurrentQueryId("");
client_context->setCurrentQueryId("");
// Generate a new query_id
for (const auto & query_id_format : query_id_formats)
{
writeString(query_id_format.first, std_out);
writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", global_context->getCurrentQueryId())), std_out);
writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", client_context->getCurrentQueryId())), std_out);
writeChar('\n', std_out);
std_out.next();
}
@ -1954,7 +1962,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
auto password = auth_data->getPassword();
if (password)
global_context->getAccessControl().checkPasswordComplexityRules(*password);
client_context->getAccessControl().checkPasswordComplexityRules(*password);
}
}
}
@ -1969,15 +1977,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
std::optional<Settings> old_settings;
SCOPE_EXIT_SAFE({
if (old_settings)
global_context->setSettings(*old_settings);
client_context->setSettings(*old_settings);
});
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)
old_settings.emplace(global_context->getSettingsRef());
global_context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
global_context->resetSettingsToDefaultValue(settings_ast.as<ASTSetQuery>()->default_settings);
old_settings.emplace(client_context->getSettingsRef());
client_context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
client_context->resetSettingsToDefaultValue(settings_ast.as<ASTSetQuery>()->default_settings);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
@ -2010,7 +2018,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (insert && insert->select)
insert->tryFindInputFunction(input_function);
bool is_async_insert_with_inlined_data = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData();
bool is_async_insert_with_inlined_data = client_context->getSettingsRef().async_insert && insert && insert->hasInlinedData();
if (is_async_insert_with_inlined_data)
{
@ -2045,9 +2053,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (change.name == "profile")
current_profile = change.value.safeGet<String>();
else
global_context->applySettingChange(change);
client_context->applySettingChange(change);
}
global_context->resetSettingsToDefaultValue(set_query->default_settings);
client_context->resetSettingsToDefaultValue(set_query->default_settings);
/// Query parameters inside SET queries should be also saved on the client side
/// to override their previous definitions set with --param_* arguments
@ -2055,7 +2063,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
for (const auto & [name, value] : set_query->query_parameters)
query_parameters.insert_or_assign(name, value);
global_context->addQueryParameters(NameToNameMap{set_query->query_parameters.begin(), set_query->query_parameters.end()});
client_context->addQueryParameters(NameToNameMap{set_query->query_parameters.begin(), set_query->query_parameters.end()});
}
if (const auto * use_query = parsed_query->as<ASTUseQuery>())
{
@ -2132,8 +2140,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
if (this_query_begin >= all_queries_end)
return MultiQueryProcessingStage::QUERIES_END;
unsigned max_parser_depth = static_cast<unsigned>(global_context->getSettingsRef().max_parser_depth);
unsigned max_parser_backtracks = static_cast<unsigned>(global_context->getSettingsRef().max_parser_backtracks);
unsigned max_parser_depth = static_cast<unsigned>(client_context->getSettingsRef().max_parser_depth);
unsigned max_parser_backtracks = static_cast<unsigned>(client_context->getSettingsRef().max_parser_backtracks);
// If there are only comments left until the end of file, we just
// stop. The parser can't handle this situation because it always
@ -2153,7 +2161,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
try
{
parsed_query = parseQuery(this_query_end, all_queries_end,
global_context->getSettingsRef(),
client_context->getSettingsRef(),
/*allow_multi_statements=*/ true);
}
catch (const Exception & e)
@ -2196,7 +2204,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
{
this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end);
insert_ast->end = this_query_end;
query_to_execute_end = isSyncInsertWithData(*insert_ast, global_context) ? insert_ast->data : this_query_end;
query_to_execute_end = isSyncInsertWithData(*insert_ast, client_context) ? insert_ast->data : this_query_end;
}
query_to_execute = all_queries_text.substr(this_query_begin - all_queries_text.data(), query_to_execute_end - this_query_begin);
@ -2241,6 +2249,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
ASTPtr parsed_query;
std::unique_ptr<Exception> current_exception;
size_t retries_count = 0;
while (true)
{
auto stage = analyzeMultiQueryText(this_query_begin, this_query_end, all_queries_end,
@ -2321,7 +2331,12 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
// Check whether the error (or its absence) matches the test hints
// (or their absence).
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())
{
@ -2398,13 +2413,13 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
// , where the inline data is delimited by semicolon and not by a
// newline.
auto * insert_ast = parsed_query->as<ASTInsertQuery>();
if (insert_ast && isSyncInsertWithData(*insert_ast, global_context))
if (insert_ast && isSyncInsertWithData(*insert_ast, client_context))
{
this_query_end = insert_ast->end;
adjustQueryEnd(
this_query_end, all_queries_end,
static_cast<unsigned>(global_context->getSettingsRef().max_parser_depth),
static_cast<unsigned>(global_context->getSettingsRef().max_parser_backtracks));
static_cast<unsigned>(client_context->getSettingsRef().max_parser_depth),
static_cast<unsigned>(client_context->getSettingsRef().max_parser_backtracks));
}
// Report error.
@ -2415,7 +2430,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
if (have_error && !ignore_error)
return is_interactive;
this_query_begin = this_query_end;
if (!need_retry)
this_query_begin = this_query_end;
break;
}
}
@ -2534,10 +2550,10 @@ void ClientBase::runInteractive()
if (load_suggestions)
{
/// Load suggestion data from the server.
if (global_context->getApplicationType() == Context::ApplicationType::CLIENT)
suggest->load<Connection>(global_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load);
else if (global_context->getApplicationType() == Context::ApplicationType::LOCAL)
suggest->load<LocalConnection>(global_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load);
if (client_context->getApplicationType() == Context::ApplicationType::CLIENT)
suggest->load<Connection>(client_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load);
else if (client_context->getApplicationType() == Context::ApplicationType::LOCAL)
suggest->load<LocalConnection>(client_context, connection_parameters, getClientConfiguration().getInt("suggestion_limit"), wait_for_suggestions_to_load);
}
if (home_path.empty())
@ -2675,7 +2691,7 @@ void ClientBase::runInteractive()
{
// If a separate connection loading suggestions failed to open a new session,
// use the main session to receive them.
suggest->load(*connection, connection_parameters.timeouts, getClientConfiguration().getInt("suggestion_limit"), global_context->getClientInfo());
suggest->load(*connection, connection_parameters.timeouts, getClientConfiguration().getInt("suggestion_limit"), client_context->getClientInfo());
}
try
@ -2724,10 +2740,10 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name)
if (!getClientConfiguration().has("log_comment"))
{
Settings settings = global_context->getSettings();
Settings settings = client_context->getSettings();
/// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]"
settings.log_comment = fs::absolute(fs::path(file_name));
global_context->setSettings(settings);
client_context->setSettings(settings);
}
return executeMultiQuery(queries_from_file);
@ -2840,168 +2856,6 @@ void ClientBase::showClientVersion()
output_stream << VERSION_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
}
namespace
{
/// Define transparent hash to we can use
/// std::string_view with the containers
struct TransparentStringHash
{
using is_transparent = void;
size_t operator()(std::string_view txt) const
{
return std::hash<std::string_view>{}(txt);
}
};
/*
* This functor is used to parse command line arguments and replace dashes with underscores,
* allowing options to be specified using either dashes or underscores.
*/
class OptionsAliasParser
{
public:
explicit OptionsAliasParser(const boost::program_options::options_description& options)
{
options_names.reserve(options.options().size());
for (const auto& option : options.options())
options_names.insert(option->long_name());
}
/*
* Parses arguments by replacing dashes with underscores, and matches the resulting name with known options
* Implements boost::program_options::ext_parser logic
*/
std::pair<std::string, std::string> operator()(const std::string & token) const
{
if (!token.starts_with("--"))
return {};
std::string arg = token.substr(2);
// divide token by '=' to separate key and value if options style=long_allow_adjacent
auto pos_eq = arg.find('=');
std::string key = arg.substr(0, pos_eq);
if (options_names.contains(key))
// option does not require any changes, because it is already correct
return {};
std::replace(key.begin(), key.end(), '-', '_');
if (!options_names.contains(key))
// after replacing '-' with '_' argument is still unknown
return {};
std::string value;
if (pos_eq != std::string::npos && pos_eq < arg.size())
value = arg.substr(pos_eq + 1);
return {key, value};
}
private:
std::unordered_set<std::string> options_names;
};
}
/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests
#if defined(__clang__)
#pragma clang optimize on
#endif
void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
{
if (allow_repeated_settings)
addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value());
else
addProgramOptions(cmd_settings, options_description.main_description.value());
if (allow_merge_tree_settings)
{
/// Add merge tree settings manually, because names of some settings
/// may clash. Query settings have higher priority and we just
/// skip ambiguous merge tree settings.
auto & main_options = options_description.main_description.value();
std::unordered_set<std::string, TransparentStringHash, std::equal_to<>> main_option_names;
for (const auto & option : main_options.options())
main_option_names.insert(option->long_name());
for (const auto & setting : cmd_merge_tree_settings.all())
{
const auto add_setting = [&](const std::string_view name)
{
if (auto it = main_option_names.find(name); it != main_option_names.end())
return;
if (allow_repeated_settings)
addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting);
else
addProgramOption(cmd_merge_tree_settings, main_options, name, setting);
};
const auto & setting_name = setting.getName();
add_setting(setting_name);
const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases();
if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end())
{
for (const auto alias : it->second)
{
add_setting(alias);
}
}
}
}
/// Parse main commandline options.
auto parser = po::command_line_parser(arguments)
.options(options_description.main_description.value())
.extra_parser(OptionsAliasParser(options_description.main_description.value()))
.allow_unregistered();
po::parsed_options parsed = parser.run();
/// Check unrecognized options without positional options.
auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional);
if (!unrecognized_options.empty())
{
auto hints = this->getHints(unrecognized_options[0]);
if (!hints.empty())
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}",
unrecognized_options[0], toString(hints));
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]);
}
/// Check positional options.
for (const auto & op : parsed.options)
{
if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--")
&& !op.original_tokens[0].empty() && !op.value.empty())
{
/// Two special cases for better usability:
/// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1"
/// These are relevant for interactive usage - user-friendly, but questionable in general.
/// In case of ambiguity or for scripts, prefer using proper options.
const auto & token = op.original_tokens[0];
po::variable_value value(boost::any(op.value), false);
const char * option;
if (token.contains(' '))
option = "query";
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token);
if (!options.emplace(option, value).second)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token);
}
}
po::store(parsed, options);
}
void ClientBase::init(int argc, char ** argv)
{
namespace po = boost::program_options;

View File

@ -206,6 +206,9 @@ protected:
/// Adjust some settings after command line options and config had been processed.
void adjustSettings();
/// Initializes the client context.
void initClientContext();
void setDefaultFormatsAndCompressionFromConfiguration();
void initTTYBuffer(ProgressOption progress);
@ -215,6 +218,9 @@ protected:
SharedContextHolder shared_context;
ContextMutablePtr global_context;
/// Client context is a context used only by the client to parse queries, process query parameters and to connect to clickhouse-server.
ContextMutablePtr client_context;
LoggerPtr fatal_log;
Poco::AutoPtr<Poco::SplitterChannel> fatal_channel_ptr;
Poco::AutoPtr<Poco::Channel> fatal_console_channel_ptr;

View File

@ -0,0 +1,176 @@
#include <Client/ClientBase.h>
#include <Core/BaseSettingsProgramOptions.h>
namespace DB
{
/**
* Program options parsing is very slow in debug builds and it affects .sh tests
* causing them to timeout sporadically.
* It seems impossible to enable optimizations for a single function (only to disable them), so
* instead we extract the code to a separate source file and compile it with different options.
*/
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int UNRECOGNIZED_ARGUMENTS;
}
namespace
{
/// Define transparent hash to we can use
/// std::string_view with the containers
struct TransparentStringHash
{
using is_transparent = void;
size_t operator()(std::string_view txt) const
{
return std::hash<std::string_view>{}(txt);
}
};
/*
* This functor is used to parse command line arguments and replace dashes with underscores,
* allowing options to be specified using either dashes or underscores.
*/
class OptionsAliasParser
{
public:
explicit OptionsAliasParser(const boost::program_options::options_description& options)
{
options_names.reserve(options.options().size());
for (const auto& option : options.options())
options_names.insert(option->long_name());
}
/*
* Parses arguments by replacing dashes with underscores, and matches the resulting name with known options
* Implements boost::program_options::ext_parser logic
*/
std::pair<std::string, std::string> operator()(const std::string & token) const
{
if (!token.starts_with("--"))
return {};
std::string arg = token.substr(2);
// divide token by '=' to separate key and value if options style=long_allow_adjacent
auto pos_eq = arg.find('=');
std::string key = arg.substr(0, pos_eq);
if (options_names.contains(key))
// option does not require any changes, because it is already correct
return {};
std::replace(key.begin(), key.end(), '-', '_');
if (!options_names.contains(key))
// after replacing '-' with '_' argument is still unknown
return {};
std::string value;
if (pos_eq != std::string::npos && pos_eq < arg.size())
value = arg.substr(pos_eq + 1);
return {key, value};
}
private:
std::unordered_set<std::string> options_names;
};
}
void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
{
if (allow_repeated_settings)
addProgramOptionsAsMultitokens(cmd_settings, options_description.main_description.value());
else
addProgramOptions(cmd_settings, options_description.main_description.value());
if (allow_merge_tree_settings)
{
/// Add merge tree settings manually, because names of some settings
/// may clash. Query settings have higher priority and we just
/// skip ambiguous merge tree settings.
auto & main_options = options_description.main_description.value();
std::unordered_set<std::string, TransparentStringHash, std::equal_to<>> main_option_names;
for (const auto & option : main_options.options())
main_option_names.insert(option->long_name());
for (const auto & setting : cmd_merge_tree_settings.all())
{
const auto add_setting = [&](const std::string_view name)
{
if (auto it = main_option_names.find(name); it != main_option_names.end())
return;
if (allow_repeated_settings)
addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting);
else
addProgramOption(cmd_merge_tree_settings, main_options, name, setting);
};
const auto & setting_name = setting.getName();
add_setting(setting_name);
const auto & settings_to_aliases = MergeTreeSettings::Traits::settingsToAliases();
if (auto it = settings_to_aliases.find(setting_name); it != settings_to_aliases.end())
{
for (const auto alias : it->second)
{
add_setting(alias);
}
}
}
}
/// Parse main commandline options.
auto parser = po::command_line_parser(arguments)
.options(options_description.main_description.value())
.extra_parser(OptionsAliasParser(options_description.main_description.value()))
.allow_unregistered();
po::parsed_options parsed = parser.run();
/// Check unrecognized options without positional options.
auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional);
if (!unrecognized_options.empty())
{
auto hints = this->getHints(unrecognized_options[0]);
if (!hints.empty())
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}",
unrecognized_options[0], toString(hints));
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]);
}
/// Check positional options.
for (const auto & op : parsed.options)
{
if (!op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--")
&& !op.original_tokens[0].empty() && !op.value.empty())
{
/// Two special cases for better usability:
/// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1"
/// These are relevant for interactive usage - user-friendly, but questionable in general.
/// In case of ambiguity or for scripts, prefer using proper options.
const auto & token = op.original_tokens[0];
po::variable_value value(boost::any(op.value), false);
const char * option;
if (token.contains(' '))
option = "query";
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token);
if (!options.emplace(option, value).second)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional option `{}` is not supported.", token);
}
}
po::store(parsed, options);
}
}

View File

@ -10,6 +10,7 @@
namespace DB::ErrorCodes
{
extern const int CANNOT_PARSE_TEXT;
extern const int OK;
}
namespace DB
@ -62,9 +63,28 @@ bool TestHint::hasExpectedServerError(int error)
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)
{
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{
"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())
{
if (token.type == TokenType::Whitespace)
continue;
String item = String(token.begin, token.end);
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);
if (item == "echoOff")
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))
{
@ -133,6 +180,9 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint)
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 <Core/Types.h>
#include <Common/Exception.h>
namespace DB
@ -65,12 +66,17 @@ public:
bool hasExpectedClientError(int error);
bool hasExpectedServerError(int error);
bool needRetry(const std::unique_ptr<Exception> & server_exception, size_t * retries_counter);
private:
const String & query;
ErrorVector server_errors{};
ErrorVector client_errors{};
std::optional<bool> echo;
size_t max_retries = 0;
bool retry_until = false;
void parse(Lexer & comment_lexer, bool is_leading_hint);
bool allErrorsExpected(int actual_server_error, int actual_client_error) const

View File

@ -8,6 +8,7 @@
#include <Common/ErrorCodes.h>
#include <Common/Exception.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <Common/Logger.h>
#include <Common/MemorySanitizer.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/config_version.h>
@ -100,7 +101,7 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
{
if (terminate_on_any_exception)
std::_Exit(terminate_status_code);
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
handle_error_code(msg_masked.msg, code, remote, getStackFramePointers());
}
@ -110,7 +111,7 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_)
{
if (terminate_on_any_exception)
std::_Exit(terminate_status_code);
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
handle_error_code(message(), code, remote, getStackFramePointers());
}
@ -119,7 +120,7 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
{
if (terminate_on_any_exception)
std::_Exit(terminate_status_code);
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
auto * stack_trace_frames = exc.get_stack_trace_frames();
auto stack_trace_size = exc.get_stack_trace_size();
@ -133,7 +134,7 @@ Exception::Exception(CreateFromSTDTag, const std::exception & exc)
{
if (terminate_on_any_exception)
std::_Exit(terminate_status_code);
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
auto * stack_trace_frames = exc.get_stack_trace_frames();
auto stack_trace_size = exc.get_stack_trace_size();
@ -223,10 +224,38 @@ Exception::FramePointers Exception::getStackFramePointers() const
}
thread_local bool Exception::enable_job_stack_trace = false;
thread_local std::vector<StackTrace::FramePointers> Exception::thread_frame_pointers = {};
thread_local bool Exception::can_use_thread_frame_pointers = false;
thread_local Exception::ThreadFramePointers Exception::thread_frame_pointers;
Exception::ThreadFramePointers::ThreadFramePointers()
{
can_use_thread_frame_pointers = true;
}
Exception::ThreadFramePointers::~ThreadFramePointers()
{
can_use_thread_frame_pointers = false;
}
Exception::ThreadFramePointersBase Exception::getThreadFramePointers()
{
if (can_use_thread_frame_pointers)
return thread_frame_pointers.frame_pointers;
return {};
}
void Exception::setThreadFramePointers(ThreadFramePointersBase frame_pointers)
{
if (can_use_thread_frame_pointers)
thread_frame_pointers.frame_pointers = std::move(frame_pointers);
}
static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message)
{
if (!isLoggingEnabled())
return;
try
{
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
@ -242,6 +271,9 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
{
if (!isLoggingEnabled())
return;
/// Under high memory pressure, new allocations throw a
/// MEMORY_LIMIT_EXCEEDED exception.
///

View File

@ -10,7 +10,6 @@
#include <cerrno>
#include <exception>
#include <memory>
#include <vector>
#include <fmt/core.h>
@ -49,14 +48,14 @@ public:
{
if (terminate_on_any_exception)
std::terminate();
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
}
Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code)
{
if (terminate_on_any_exception)
std::terminate();
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
message_format_string = msg.format_string;
message_format_string_args = msg.format_string_args;
}
@ -65,18 +64,36 @@ public:
{
if (terminate_on_any_exception)
std::terminate();
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
message_format_string = msg.format_string;
message_format_string_args = msg.format_string_args;
}
/// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution
static thread_local bool enable_job_stack_trace;
static thread_local std::vector<StackTrace::FramePointers> thread_frame_pointers;
static thread_local bool can_use_thread_frame_pointers;
/// Because of unknown order of static destructor calls,
/// thread_frame_pointers can already be uninitialized when a different destructor generates an exception.
/// To prevent such scenarios, a wrapper class is created and a function that will return empty vector
/// if its destructor is already called
using ThreadFramePointersBase = std::vector<StackTrace::FramePointers>;
struct ThreadFramePointers
{
ThreadFramePointers();
~ThreadFramePointers();
ThreadFramePointersBase frame_pointers;
};
static ThreadFramePointersBase getThreadFramePointers();
static void setThreadFramePointers(ThreadFramePointersBase frame_pointers);
/// Callback for any exception
static std::function<void(const std::string & msg, int code, bool remote, const Exception::FramePointers & trace)> callback;
protected:
static thread_local ThreadFramePointers thread_frame_pointers;
// used to remove the sensitive information from exceptions if query_masking_rules is configured
struct MessageMasked
{
@ -178,7 +195,7 @@ class ErrnoException : public Exception
public:
ErrnoException(std::string && msg, int code, int with_errno) : Exception(msg, code), saved_errno(with_errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
addMessage(", {}", errnoToString(saved_errno));
}
@ -187,7 +204,7 @@ public:
requires std::is_convertible_v<T, String>
ErrnoException(int code, T && message) : Exception(message, code), saved_errno(errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
capture_thread_frame_pointers = getThreadFramePointers();
addMessage(", {}", errnoToString(saved_errno));
}

View File

@ -25,3 +25,15 @@ bool hasLogger(const std::string & name)
{
return Poco::Logger::has(name);
}
static constinit std::atomic<bool> allow_logging{true};
bool isLoggingEnabled()
{
return allow_logging;
}
void disableLogging()
{
allow_logging = false;
}

View File

@ -64,3 +64,7 @@ LoggerRawPtr createRawLogger(const std::string & name, Poco::Channel * channel,
* Otherwise, returns false.
*/
bool hasLogger(const std::string & name);
void disableLogging();
bool isLoggingEnabled();

View File

@ -89,7 +89,7 @@ void signalHandler(int sig, siginfo_t * info, void * context)
writePODBinary(*info, out);
writePODBinary(signal_context, out);
writePODBinary(stack_trace, out);
writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector<StackTrace::FramePointers>{}, out);
writeVectorBinary(Exception::enable_job_stack_trace ? Exception::getThreadFramePointers() : std::vector<StackTrace::FramePointers>{}, out);
writeBinary(static_cast<UInt32>(getThreadId()), out);
writePODBinary(current_thread, out);

View File

@ -489,24 +489,25 @@ struct CacheEntry
using CacheEntryPtr = std::shared_ptr<CacheEntry>;
static constinit std::atomic<bool> can_use_cache = false;
static constinit bool can_use_cache = false;
using StackTraceCacheBase = std::map<StackTraceTriple, CacheEntryPtr, std::less<>>;
struct StackTraceCache : public StackTraceCacheBase
{
StackTraceCache()
: StackTraceCacheBase()
{
can_use_cache = true;
}
~StackTraceCache()
{
can_use_cache = false;
}
};
static StackTraceCache & cacheInstance()
{
static StackTraceCache cache;
can_use_cache = true;
return cache;
}
static StackTraceCache cache;
static DB::SharedMutex stacktrace_cache_mutex;
@ -524,7 +525,6 @@ String toStringCached(const StackTrace::FramePointers & pointers, size_t offset,
/// Calculation of stack trace text is extremely slow.
/// We use cache because otherwise the server could be overloaded by trash queries.
/// Note that this cache can grow unconditionally, but practically it should be small.
StackTraceCache & cache = cacheInstance();
CacheEntryPtr cache_entry;
// Optimistic try for cache hit to avoid any contention whatsoever, should be the main hot code route
@ -576,7 +576,7 @@ std::string StackTrace::toString(void * const * frame_pointers_raw, size_t offse
void StackTrace::dropCache()
{
std::lock_guard lock{stacktrace_cache_mutex};
cacheInstance().clear();
cache.clear();
}

View File

@ -51,7 +51,7 @@ public:
if (!capture_frame_pointers)
return;
/// Save all previous jobs call stacks and append with current
frame_pointers = DB::Exception::thread_frame_pointers;
frame_pointers = DB::Exception::getThreadFramePointers();
frame_pointers.push_back(StackTrace().getFramePointers());
}
@ -455,7 +455,7 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
try
{
if (DB::Exception::enable_job_stack_trace)
DB::Exception::thread_frame_pointers = std::move(job_data->frame_pointers);
DB::Exception::setThreadFramePointers(std::move(job_data->frame_pointers));
CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads);

View File

@ -548,7 +548,7 @@ public:
virtual bool isExpired() const = 0;
/// Get the current connected node idx.
virtual Int8 getConnectedNodeIdx() const = 0;
virtual std::optional<int8_t> getConnectedNodeIdx() const = 0;
/// Get the current connected host and port.
virtual String getConnectedHostPort() const = 0;

View File

@ -39,7 +39,7 @@ public:
~TestKeeper() override;
bool isExpired() const override { return expired; }
Int8 getConnectedNodeIdx() const override { return 0; }
std::optional<int8_t> getConnectedNodeIdx() const override { return 0; }
String getConnectedHostPort() const override { return "TestKeeper:0000"; }
int32_t getConnectionXid() const override { return 0; }
int64_t getSessionID() const override { return 0; }

View File

@ -128,16 +128,15 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr<Coordination::IKeeper>
ShuffleHosts shuffled_hosts = shuffleHosts();
impl = std::make_unique<Coordination::ZooKeeper>(shuffled_hosts, args, zk_log);
Int8 node_idx = impl->getConnectedNodeIdx();
auto node_idx = impl->getConnectedNodeIdx();
if (args.chroot.empty())
LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ","));
else
LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot);
/// If the balancing strategy has an optimal node then it will be the first in the list
bool connected_to_suboptimal_node = node_idx != shuffled_hosts[0].original_index;
bool connected_to_suboptimal_node = node_idx && static_cast<UInt8>(*node_idx) != shuffled_hosts[0].original_index;
bool respect_az = args.prefer_local_availability_zone && !args.client_availability_zone.empty();
bool may_benefit_from_reconnecting = respect_az || args.get_priority_load_balancing.hasOptimalNode();
if (connected_to_suboptimal_node && may_benefit_from_reconnecting)
@ -145,7 +144,7 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr<Coordination::IKeeper>
auto reconnect_timeout_sec = getSecondsUntilReconnect(args);
LOG_DEBUG(log, "Connected to a suboptimal ZooKeeper host ({}, index {})."
" To preserve balance in ZooKeeper usage, this ZooKeeper session will expire in {} seconds",
impl->getConnectedHostPort(), node_idx, reconnect_timeout_sec);
impl->getConnectedHostPort(), *node_idx, reconnect_timeout_sec);
auto reconnect_task_holder = DB::Context::getGlobalContextInstance()->getSchedulePool().createTask("ZKReconnect", [this, optimal_host = shuffled_hosts[0]]()
{
@ -154,13 +153,15 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr<Coordination::IKeeper>
LOG_DEBUG(log, "Trying to connect to a more optimal node {}", optimal_host.host);
ShuffleHosts node{optimal_host};
std::unique_ptr<Coordination::IKeeper> new_impl = std::make_unique<Coordination::ZooKeeper>(node, args, zk_log);
Int8 new_node_idx = new_impl->getConnectedNodeIdx();
auto new_node_idx = new_impl->getConnectedNodeIdx();
chassert(new_node_idx.has_value());
/// Maybe the node was unavailable when getting AZs first time, update just in case
if (args.availability_zone_autodetect && availability_zones[new_node_idx].empty())
if (args.availability_zone_autodetect && availability_zones[*new_node_idx].empty())
{
availability_zones[new_node_idx] = new_impl->tryGetAvailabilityZone();
LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[new_node_idx]);
availability_zones[*new_node_idx] = new_impl->tryGetAvailabilityZone();
LOG_DEBUG(log, "Got availability zone for {}: {}", optimal_host.host, availability_zones[*new_node_idx]);
}
optimal_impl = std::move(new_impl);
@ -1525,7 +1526,7 @@ void ZooKeeper::setServerCompletelyStarted()
zk->setServerCompletelyStarted();
}
Int8 ZooKeeper::getConnectedHostIdx() const
std::optional<int8_t> ZooKeeper::getConnectedHostIdx() const
{
return impl->getConnectedNodeIdx();
}
@ -1544,10 +1545,10 @@ String ZooKeeper::getConnectedHostAvailabilityZone() const
{
if (args.implementation != "zookeeper" || !impl)
return "";
Int8 idx = impl->getConnectedNodeIdx();
if (idx < 0)
std::optional<int8_t> idx = impl->getConnectedNodeIdx();
if (!idx)
return ""; /// session expired
return availability_zones.at(idx);
return availability_zones.at(*idx);
}
size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses)

View File

@ -620,7 +620,7 @@ public:
void setServerCompletelyStarted();
Int8 getConnectedHostIdx() const;
std::optional<int8_t> getConnectedHostIdx() const;
String getConnectedHostPort() const;
int32_t getConnectionXid() const;

View File

@ -536,7 +536,7 @@ void ZooKeeper::connect(
compressed_out.emplace(*out, CompressionCodecFactory::instance().get("LZ4", {}));
}
original_index = static_cast<Int8>(node.original_index);
original_index.store(node.original_index);
break;
}
catch (...)
@ -1014,9 +1014,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
LOG_INFO(log, "Finalizing session {}. finalization_started: {}, queue_finished: {}, reason: '{}'",
session_id, already_started, requests_queue.isFinished(), reason);
/// Reset the original index.
original_index = -1;
auto expire_session_if_not_expired = [&]
{
/// No new requests will appear in queue after finish()
@ -1534,6 +1531,30 @@ void ZooKeeper::close()
}
std::optional<int8_t> ZooKeeper::getConnectedNodeIdx() const
{
int8_t res = original_index.load();
if (res == -1)
return std::nullopt;
else
return res;
}
String ZooKeeper::getConnectedHostPort() const
{
auto idx = getConnectedNodeIdx();
if (idx)
return args.hosts[*idx];
else
return "";
}
int32_t ZooKeeper::getConnectionXid() const
{
return next_xid.load();
}
void ZooKeeper::setZooKeeperLog(std::shared_ptr<DB::ZooKeeperLog> zk_log_)
{
/// logOperationIfNeeded(...) uses zk_log and can be called from different threads, so we have to use atomic shared_ptr

View File

@ -114,13 +114,12 @@ public:
~ZooKeeper() override;
/// If expired, you can only destroy the object. All other methods will throw exception.
bool isExpired() const override { return requests_queue.isFinished(); }
Int8 getConnectedNodeIdx() const override { return original_index; }
String getConnectedHostPort() const override { return (original_index == -1) ? "" : args.hosts[original_index]; }
int32_t getConnectionXid() const override { return next_xid.load(); }
std::optional<int8_t> getConnectedNodeIdx() const override;
String getConnectedHostPort() const override;
int32_t getConnectionXid() const override;
String tryGetAvailabilityZone() override;
@ -219,7 +218,7 @@ private:
ACLs default_acls;
zkutil::ZooKeeperArgs args;
Int8 original_index = -1;
std::atomic<int8_t> original_index{-1};
/// Fault injection
void maybeInjectSendFault();

View File

@ -1,2 +1,2 @@
clickhouse_add_executable (mysqlxx_pool_test mysqlxx_pool_test.cpp)
target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx clickhouse_common_config)
target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx clickhouse_common_config loggers_no_text_log)

View File

@ -186,7 +186,7 @@ class IColumn;
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \
M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
@ -1125,7 +1125,7 @@ class IColumn;
M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \
M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \
M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \
M(Bool, input_format_json_ignore_key_case, false, "Ignore json key case while read json field from string", 0) \
M(Bool, input_format_json_case_insensitive_column_matching, false, "Ignore case when matching JSON keys with CH columns", 0) \
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \

View File

@ -57,6 +57,7 @@ String ClickHouseVersion::toString() const
/// Note: please check if the key already exists to prevent duplicate entries.
static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory::SettingsChanges>> settings_changes_history_initializer =
{
{"24.8", {{"compile_expressions", false, true, "We believe that the LLVM infrastructure behind the JIT compiler is stable enough to enable this setting by default."}}},
{"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."},
{"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"},
{"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"},
@ -64,7 +65,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"},
{"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"},
{"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."},
{"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."},
{"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."},
{"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."},
{"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."},
{"collect_hash_table_stats_during_joins", false, true, "New setting."},
@ -80,7 +81,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."},
{"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."},
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."},
}},
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},

View File

@ -158,7 +158,7 @@ BaseDaemon::~BaseDaemon()
tryLogCurrentException(&logger());
}
OwnSplitChannel::disableLogging();
disableLogging();
}

View File

@ -289,9 +289,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
tables.erase(it);
table_storage->is_detached = true;
if (!table_storage->isSystemStorage()
&& database_name != DatabaseCatalog::SYSTEM_DATABASE
&& database_name != DatabaseCatalog::TEMPORARY_DATABASE)
if (!table_storage->isSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name))
{
LOG_TEST(log, "Counting detached table {} to database {}", table_name, database_name);
CurrentMetrics::sub(getAttachedCounterForStorage(table_storage));
@ -339,9 +337,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
/// non-Atomic database the is_detached is set to true before RENAME.
table->is_detached = false;
if (!table->isSystemStorage()
&& database_name != DatabaseCatalog::SYSTEM_DATABASE
&& database_name != DatabaseCatalog::TEMPORARY_DATABASE)
if (!table->isSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name))
{
LOG_TEST(log, "Counting attached table {} to database {}", table_name, database_name);
CurrentMetrics::add(getAttachedCounterForStorage(table));

View File

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

View File

@ -34,7 +34,7 @@ public:
String getFileName() const override { return impl->getFileName(); }
size_t getFileSize() override { return impl->getFileSize(); }
std::optional<size_t> tryGetFileSize() override { return impl->tryGetFileSize(); }
String getInfoForLog() override { return impl->getInfoForLog(); }

View File

@ -253,16 +253,15 @@ void ReadBufferFromAzureBlobStorage::initialize()
initialized = true;
}
size_t ReadBufferFromAzureBlobStorage::getFileSize()
std::optional<size_t> ReadBufferFromAzureBlobStorage::tryGetFileSize()
{
if (!blob_client)
blob_client = std::make_unique<Azure::Storage::Blobs::BlobClient>(blob_container_client->GetBlobClient(path));
if (file_size.has_value())
return *file_size;
if (!file_size)
file_size = blob_client->GetProperties().Value.BlobSize;
file_size = blob_client->GetProperties().Value.BlobSize;
return *file_size;
return file_size;
}
size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & /*progress_callback*/) const

View File

@ -42,7 +42,7 @@ public:
bool supportsRightBoundedReads() const override { return true; }
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & progress_callback) const override;

View File

@ -41,7 +41,7 @@ public:
void setReadUntilEnd() override { setReadUntilPosition(getFileSize()); }
size_t getFileSize() override { return getTotalSize(blobs_to_read); }
std::optional<size_t> tryGetFileSize() override { return getTotalSize(blobs_to_read); }
size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; }

View File

@ -150,7 +150,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.json.try_infer_objects_as_tuples = settings.input_format_json_try_infer_named_tuples_from_objects;
format_settings.json.throw_on_bad_escape_sequence = settings.input_format_json_throw_on_bad_escape_sequence;
format_settings.json.ignore_unnecessary_fields = settings.input_format_json_ignore_unnecessary_fields;
format_settings.json.ignore_key_case = settings.input_format_json_ignore_key_case;
format_settings.json.case_insensitive_column_matching = settings.input_format_json_case_insensitive_column_matching;
format_settings.null_as_default = settings.input_format_null_as_default;
format_settings.force_null_for_omitted_fields = settings.input_format_force_null_for_omitted_fields;
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;

View File

@ -233,7 +233,7 @@ struct FormatSettings
bool infer_incomplete_types_as_strings = true;
bool throw_on_bad_escape_sequence = true;
bool ignore_unnecessary_fields = true;
bool ignore_key_case = false;
bool case_insensitive_column_matching = false;
} json{};
struct

View File

@ -84,7 +84,7 @@ public:
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.init(arguments, max_substrings_includes_remaining_string);
@ -107,18 +107,17 @@ public:
const ColumnString::Chars & src_chars = col_str->getChars();
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random.
res_offsets.reserve(input_rows_count);
res_strings_offsets.reserve(input_rows_count * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
size_t size = src_offsets.size();
ColumnString::Offset current_src_offset = 0;
ColumnArray::Offset current_dst_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]);
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>()));
}
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;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
@ -79,7 +79,7 @@ public:
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;
auto period_ptr = arguments[1].column->convertToFullColumnIfConst();

View File

@ -55,7 +55,7 @@ public:
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);
@ -71,12 +71,12 @@ public:
checkRepeatTime(times);
res_offsets.resize(col_times->size());
res_chars.resize(col_times->size() * (times + 1));
res_offsets.resize(input_rows_count);
res_chars.resize(input_rows_count * (times + 1));
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);
pos += times;
@ -92,20 +92,20 @@ public:
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());
if (!col_times)
return false;
res_offsets.resize(col_times->size());
res_chars.resize(col_times->size() * 10); /// heuristic
res_offsets.resize(input_rows_count);
res_chars.resize(input_rows_count * 10); /// heuristic
const PaddedPODArray<typename DataType::FieldType> & times_data = col_times->getData();
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];
@ -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;
@ -143,26 +143,26 @@ public:
if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get()))
{
if ((executeConstant<DataTypeUInt8>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt16>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt32>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt64>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt8>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt16>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt32>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt64>(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, input_rows_count))
|| (executeConstant<DataTypeUInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeUInt64>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeConstant<DataTypeInt64>(col_num, res_offsets, res_chars, input_rows_count)))
return col_res;
}
else
{
if ((executeVector<DataTypeUInt8>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt16>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt32>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt64>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt8>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt16>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt32>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt64>(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, input_rows_count))
|| (executeVector<DataTypeUInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeUInt64>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt8>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt16>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt32>(col_num, res_offsets, res_chars, input_rows_count))
|| (executeVector<DataTypeInt64>(col_num, res_offsets, res_chars, input_rows_count)))
return col_res;
}

View File

@ -32,7 +32,8 @@ struct StemImpl
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
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");
@ -45,7 +46,7 @@ struct StemImpl
res_offsets.assign(offsets);
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.
size_t original_size = offsets[i] - offsets[i - 1];
@ -101,7 +102,7 @@ public:
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 & strcolumn = arguments[1].column;
@ -119,7 +120,7 @@ public:
String language = lang_col->getValue<String>();
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;
}
};

View File

@ -40,7 +40,7 @@ public:
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);
@ -53,8 +53,7 @@ public:
const ColumnString::Chars & in_vec = col_str_in->getChars();
const ColumnString::Offsets & in_offsets = col_str_in->getOffsets();
size_t size = in_offsets.size();
out_offsets.resize(size);
out_offsets.resize(input_rows_count);
out_vec.resize(in_vec.size());
char * begin = reinterpret_cast<char *>(out_vec.data());
@ -62,7 +61,7 @@ public:
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]);
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);
@ -99,10 +98,8 @@ public:
const ColumnString::Chars & in_vec = col_fstr_in->getChars();
size_t size = col_fstr_in->size();
out_offsets.resize(size);
out_vec.resize(in_vec.size() + size);
out_offsets.resize(input_rows_count);
out_vec.resize(in_vec.size() + input_rows_count);
char * begin = reinterpret_cast<char *>(out_vec.data());
char * pos = begin;
@ -110,7 +107,7 @@ public:
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);
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();
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;
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",

View File

@ -68,7 +68,7 @@ namespace
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_delim = arguments[1].column;
@ -110,10 +110,10 @@ namespace
if (is_count_const)
{
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
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;
}
@ -124,18 +124,18 @@ namespace
const String & delim,
const IColumn * count_column,
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_offsets.reserve(rows);
res_offsets.reserve(input_rows_count);
bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !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);
Int64 count = count_column->getInt(i);
@ -157,18 +157,18 @@ namespace
const String & delim,
Int64 count,
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_offsets.reserve(rows);
res_offsets.reserve(input_rows_count);
bool all_ascii = isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !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);

View File

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

View File

@ -152,7 +152,7 @@ private:
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
static void vectorVector(
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)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = starts.size();
result_offsets.resize(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
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)
{
@ -66,18 +65,17 @@ struct TimeSlotsImpl
static void vectorConstant(
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)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = starts.size();
result_offsets.resize(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
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)
{
@ -91,18 +89,17 @@ struct TimeSlotsImpl
static void constantVector(
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)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time slot size cannot be zero");
size_t size = durations.size();
result_offsets.resize(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
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)
{
@ -120,12 +117,11 @@ struct TimeSlotsImpl
*/
static NO_SANITIZE_UNDEFINED void vectorVector(
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(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
/// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -139,7 +135,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0)
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)
{
@ -152,12 +148,11 @@ struct TimeSlotsImpl
static NO_SANITIZE_UNDEFINED void vectorConstant(
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(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
/// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -172,7 +167,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0)
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)
{
@ -185,12 +180,11 @@ struct TimeSlotsImpl
static NO_SANITIZE_UNDEFINED void constantVector(
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(size);
result_values.reserve(size);
result_offsets.resize(input_rows_count);
result_values.reserve(input_rows_count);
/// Modify all units to have same scale
UInt16 max_scale = std::max({dt_scale, duration_scale, time_slot_scale});
@ -205,7 +199,7 @@ struct TimeSlotsImpl
if (time_slot_size == 0)
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)
{
@ -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())
{
@ -308,17 +302,17 @@ public:
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;
}
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;
}
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;
}
}
@ -353,21 +347,21 @@ public:
if (starts && durations)
{
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;
}
else if (starts && const_durations)
{
TimeSlotsImpl::vectorConstant(
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;
}
else if (const_starts && durations)
{
TimeSlotsImpl::constantVector(
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;
}
}

View File

@ -54,9 +54,9 @@ private:
/// For operations with Integer/Float
template <typename FromVectorType>
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);
/// Buffer is used here and in functions below because resulting size cannot be precisely anticipated,
@ -74,9 +74,9 @@ private:
template <typename FirstArgVectorType>
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);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
@ -98,7 +98,8 @@ private:
/// For operations with Decimal
template <typename FirstArgVectorType>
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.
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);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
size_t input_rows_count = vec_from.size();
result_offsets.resize(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
@ -121,9 +121,9 @@ private:
template <typename FirstArgVectorType>
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);
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
@ -182,28 +182,28 @@ private:
}
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())
{
case TypeIndex::UInt8: return executeType<UInt8>(arguments);
case TypeIndex::UInt16: return executeType<UInt16>(arguments);
case TypeIndex::UInt32: return executeType<UInt32>(arguments);
case TypeIndex::UInt64: return executeType<UInt64>(arguments);
case TypeIndex::UInt128: return executeType<UInt128>(arguments);
case TypeIndex::UInt256: return executeType<UInt256>(arguments);
case TypeIndex::Int8: return executeType<Int8>(arguments);
case TypeIndex::Int16: return executeType<Int16>(arguments);
case TypeIndex::Int32: return executeType<Int32>(arguments);
case TypeIndex::Int64: return executeType<Int64>(arguments);
case TypeIndex::Int128: return executeType<Int128>(arguments);
case TypeIndex::Int256: return executeType<Int256>(arguments);
case TypeIndex::Float32: return executeType<Float32>(arguments);
case TypeIndex::Float64: return executeType<Float64>(arguments);
case TypeIndex::Decimal32: return executeType<Decimal32>(arguments);
case TypeIndex::Decimal64: return executeType<Decimal64>(arguments);
case TypeIndex::Decimal128: return executeType<Decimal128>(arguments);
case TypeIndex::Decimal256: return executeType<Decimal256>(arguments);
case TypeIndex::UInt8: return executeType<UInt8>(arguments, input_rows_count);
case TypeIndex::UInt16: return executeType<UInt16>(arguments, input_rows_count);
case TypeIndex::UInt32: return executeType<UInt32>(arguments, input_rows_count);
case TypeIndex::UInt64: return executeType<UInt64>(arguments, input_rows_count);
case TypeIndex::UInt128: return executeType<UInt128>(arguments, input_rows_count);
case TypeIndex::UInt256: return executeType<UInt256>(arguments, input_rows_count);
case TypeIndex::Int8: return executeType<Int8>(arguments, input_rows_count);
case TypeIndex::Int16: return executeType<Int16>(arguments, input_rows_count);
case TypeIndex::Int32: return executeType<Int32>(arguments, input_rows_count);
case TypeIndex::Int64: return executeType<Int64>(arguments, input_rows_count);
case TypeIndex::Int128: return executeType<Int128>(arguments, input_rows_count);
case TypeIndex::Int256: return executeType<Int256>(arguments, input_rows_count);
case TypeIndex::Float32: return executeType<Float32>(arguments, input_rows_count);
case TypeIndex::Float64: return executeType<Float64>(arguments, input_rows_count);
case TypeIndex::Decimal32: return executeType<Decimal32>(arguments, input_rows_count);
case TypeIndex::Decimal64: return executeType<Decimal64>(arguments, input_rows_count);
case TypeIndex::Decimal128: return executeType<Decimal128>(arguments, input_rows_count);
case TypeIndex::Decimal256: return executeType<Decimal256>(arguments, input_rows_count);
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
arguments[0].column->getName(), getName());
@ -212,7 +212,7 @@ public:
private:
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_const = checkAndGetColumnConst<ColumnVector<UInt8>>(arguments[1].column.get());
@ -230,9 +230,9 @@ private:
{
UInt8 from_scale = from_col->getScale();
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)
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
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 (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)
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
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();
}
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 & interval_column = arguments[1];
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;
}
private:
ColumnPtr dispatchForTimeColumn(
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_col = *time_column.column.get();
@ -170,19 +171,19 @@ private:
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
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))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col);
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))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(&time_column_col);
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());
}
@ -190,7 +191,7 @@ private:
template <typename TimeDataType, typename TimeColumnType>
ColumnPtr dispatchForIntervalColumn(
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());
if (!interval_type)
@ -207,27 +208,27 @@ private:
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
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:
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:
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:
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:
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:
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:
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:
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:
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:
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:
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();
@ -236,22 +237,21 @@ private:
template <typename TimeDataType, typename TimeColumnType, typename ResultDataType, IntervalKind::Kind unit>
ColumnPtr execute(
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 ResultFieldType = typename ResultDataType::FieldType;
const auto & time_data = time_column_type.getData();
size_t size = time_data.size();
auto result_col = result_type->createColumn();
auto * col_to = assert_cast<ResultColumnType *>(result_col.get());
auto & result_data = col_to->getData();
result_data.resize(size);
result_data.resize(input_rows_count);
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));
return result_col;

View File

@ -73,7 +73,7 @@ public:
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();
@ -90,9 +90,9 @@ public:
auto input_column = arguments[0].column;
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()))
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));
}
@ -105,9 +105,9 @@ public:
auto input_column = arguments[0].column;
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()))
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));
}
@ -120,15 +120,15 @@ private:
const ExtractorType & extractor,
StringColumnType & input_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;
auto & offsets_data = offsets_column.getData();
size_t column_size = input_data_column.size();
offsets_data.resize(column_size);
offsets_data.resize(input_rows_count);
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);

View File

@ -173,30 +173,30 @@ namespace
}
else if (cache.table_num_to_idx)
{
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const, *in_casted)
&& !executeNum<ColumnDecimal<Decimal64>>(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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
}
}
else if (cache.table_string_to_idx)
{
if (!executeString(in, *column_result, default_non_const, *in_casted))
executeContiguous(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, input_rows_count);
}
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
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);
}
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;
column_result.reserve(size);
for (size_t i = 0; i < size; ++i)
column_result.reserve(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
{
SipHash 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;
column_result.reserve(size);
for (size_t i = 0; i < size; ++i)
column_result.reserve(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
{
const auto * it = table.find(in->getDataAt(i));
if (it)
@ -259,7 +257,7 @@ namespace
}
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);
if (!in)
@ -269,24 +267,23 @@ namespace
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
in_scale = in->getScale();
if (!executeNumToString(pod, column_result, default_non_const)
&& !executeNumToNum<ColumnVector<UInt8>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<UInt16>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<UInt32>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<UInt64>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Int8>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Int16>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Int32>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Int64>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Float32>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnVector<Float64>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnDecimal<Decimal32>>(pod, column_result, default_non_const, in_scale)
&& !executeNumToNum<ColumnDecimal<Decimal64>>(pod, column_result, default_non_const, in_scale))
if (!executeNumToString(pod, column_result, default_non_const, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count)
&& !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, input_rows_count))
{
const size_t size = pod.size();
const auto & table = *cache.table_num_to_idx;
column_result.reserve(size);
for (size_t i = 0; i < size; ++i)
column_result.reserve(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
{
const auto * it = table.find(bit_cast<UInt64>(pod[i]));
if (it)
@ -303,14 +300,13 @@ namespace
}
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);
if (!out)
return false;
auto & out_offs = out->getOffsets();
const size_t size = pod.size();
out_offs.resize(size);
out_offs.resize(input_rows_count);
auto & out_chars = out->getChars();
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_data = def_chars.data();
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
{
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars();
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;
}
@ -347,10 +343,10 @@ namespace
const ColumnString::Offsets & to_offsets,
const DefData & def_data,
const DefOffs & def_offsets,
const size_t size) const
size_t input_rows_count) const
{
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;
size_t to_size = 0;
@ -382,14 +378,13 @@ namespace
template <typename T, typename U>
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);
if (!out)
return false;
auto & out_pod = out->getData();
const size_t size = pod.size();
out_pod.resize(size);
out_pod.resize(input_rows_count);
UInt32 out_scale = 0;
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
out_scale = out->getScale();
@ -399,15 +394,15 @@ namespace
if (cache.default_column)
{
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)
{
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
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;
}
@ -418,11 +413,11 @@ namespace
PaddedPODArray<Out> & out_pod,
const PaddedPODArray<Out> & to_pod,
const Def & def,
const size_t size,
const UInt32 out_scale,
const UInt32 def_scale) const
size_t input_rows_count,
UInt32 out_scale,
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]));
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);
if (!in)
@ -458,19 +453,19 @@ namespace
const auto & data = in->getChars();
const auto & offsets = in->getOffsets();
if (!executeStringToString(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<UInt8>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<UInt16>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<UInt32>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<UInt64>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Int8>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Int16>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Int32>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Int64>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Float32>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnVector<Float64>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnDecimal<Decimal32>>(data, offsets, column_result, default_non_const)
&& !executeStringToNum<ColumnDecimal<Decimal64>>(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, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt16>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<UInt64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int8>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int16>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Int64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Float32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnVector<Float64>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnDecimal<Decimal32>>(data, offsets, column_result, default_non_const, input_rows_count)
&& !executeStringToNum<ColumnDecimal<Decimal64>>(data, offsets, column_result, default_non_const, input_rows_count))
{
const size_t size = offsets.size();
const auto & table = *cache.table_string_to_idx;
@ -497,14 +492,14 @@ namespace
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
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);
if (!out)
return false;
auto & out_offs = out->getOffsets();
const size_t size = offsets.size();
out_offs.resize(size);
out_offs.resize(input_rows_count);
auto & out_chars = out->getChars();
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_data = def_chars.data();
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)
{
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars();
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
{
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;
}
@ -546,11 +541,11 @@ namespace
const ColumnString::Offsets & to_offsets,
const DefData & def_data,
const DefOffs & def_offsets,
const size_t size) const
size_t input_rows_count) const
{
ColumnString::Offset current_offset = 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;
size_t to_size = 0;
@ -587,26 +582,26 @@ namespace
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
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);
if (!out)
return false;
auto & out_pod = out->getData();
const size_t size = offsets.size();
out_pod.resize(size);
out_pod.resize(input_rows_count);
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
const auto & table = *cache.table_string_to_idx;
if (cache.default_column)
{
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
{
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;
}
@ -619,10 +614,10 @@ namespace
PaddedPODArray<Out> & out_pod,
const PaddedPODArray<Out> & to_pod,
const Def & def,
const size_t size) const
size_t input_rows_count) const
{
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};
current_offset = offsets[i];

View File

@ -52,7 +52,8 @@ struct TranslateImpl
const std::string & map_from,
const std::string & map_to,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{
Map map;
fillMapWithValues(map, map_from, map_to);
@ -62,7 +63,7 @@ struct TranslateImpl
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_end = data.data() + offsets[i] - 1;
@ -175,19 +176,20 @@ struct TranslateUTF8Impl
const std::string & map_from,
const std::string & map_to,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{
MapASCII map_ascii;
MapUTF8 map;
fillMapWithValues(map_ascii, map, map_from, map_to);
res_data.resize(data.size());
res_offsets.resize(offsets.size());
res_offsets.resize(input_rows_count);
UInt8 * dst = res_data.data();
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_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_map_from = arguments[1].column;
@ -330,7 +332,7 @@ public:
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_src.get()))
{
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;
}
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);
}
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 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();
MutableColumnPtr values = tuple_col_concrete->getColumn(0).cloneEmpty();
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)
{

View File

@ -321,7 +321,7 @@ public:
off_t getPosition() override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); }
String getFileName() const override { return handle.getFileName(); }
size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; }
std::optional<size_t> tryGetFileSize() override { return handle.getFileInfo().uncompressed_size; }
Handle releaseHandle() && { return std::move(handle); }

View File

@ -317,7 +317,7 @@ public:
String getFileName() const override { return handle.getFileName(); }
size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; }
std::optional<size_t> tryGetFileSize() override { return handle.getFileInfo().uncompressed_size; }
/// Releases owned handle to pass it to an enumerator.
HandleHolder releaseHandle() &&

View File

@ -244,7 +244,7 @@ void AsynchronousReadBufferFromFileDescriptor::rewind()
file_offset_of_buffer_end = 0;
}
size_t AsynchronousReadBufferFromFileDescriptor::getFileSize()
std::optional<size_t> AsynchronousReadBufferFromFileDescriptor::tryGetFileSize()
{
return getSizeFromFileDescriptor(fd, getFileName());
}

View File

@ -68,7 +68,7 @@ public:
/// Seek to the beginning, discarding already read data if any. Useful to reread file that changes on every read.
void rewind();
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; }

View File

@ -21,7 +21,7 @@ public:
off_t seek(off_t off, int whence) override;
off_t getPosition() override;
size_t getFileSize() override { return total_size; }
std::optional<size_t> tryGetFileSize() override { return total_size; }
private:
bool nextImpl() override;

View File

@ -87,7 +87,7 @@ off_t MMapReadBufferFromFileDescriptor::seek(off_t offset, int whence)
return new_pos;
}
size_t MMapReadBufferFromFileDescriptor::getFileSize()
std::optional<size_t> MMapReadBufferFromFileDescriptor::tryGetFileSize()
{
return getSizeFromFileDescriptor(getFD(), getFileName());
}

View File

@ -38,7 +38,7 @@ public:
int getFD() const;
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) const override;
bool supportsReadAt() override { return true; }

View File

@ -152,7 +152,7 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence)
return offset;
}
size_t ParallelReadBuffer::getFileSize()
std::optional<size_t> ParallelReadBuffer::tryGetFileSize()
{
return file_size;
}

View File

@ -33,7 +33,7 @@ public:
~ParallelReadBuffer() override { finishAndWait(); }
off_t seek(off_t off, int whence) override;
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
off_t getPosition() override;
const SeekableReadBuffer & getReadBuffer() const { return input; }

View File

@ -19,7 +19,8 @@ private:
std::string getFileName() const override { return "<empty>"; }
off_t seek(off_t /*off*/, int /*whence*/) override { return 0; }
off_t getPosition() override { return 0; }
size_t getFileSize() override { return 0; }
std::optional<size_t> tryGetFileSize() override { return 0; }
size_t getFileOffsetOfBufferEnd() const override { return 0; }
};
}

View File

@ -30,7 +30,7 @@ public:
void setReadUntilEnd() override { in->setReadUntilEnd(); }
size_t getFileSize() override { return in->getFileSize(); }
std::optional<size_t> tryGetFileSize() override { return in->tryGetFileSize(); }
private:
bool nextImpl() override;

View File

@ -5,11 +5,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FILE_SIZE;
}
ReadBufferFromFileBase::ReadBufferFromFileBase() : BufferWithOwnMemory<SeekableReadBuffer>(0)
{
}
@ -26,11 +21,9 @@ ReadBufferFromFileBase::ReadBufferFromFileBase(
ReadBufferFromFileBase::~ReadBufferFromFileBase() = default;
size_t ReadBufferFromFileBase::getFileSize()
std::optional<size_t> ReadBufferFromFileBase::tryGetFileSize()
{
if (file_size)
return *file_size;
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for read buffer");
return file_size;
}
void ReadBufferFromFileBase::setProgressCallback(ContextPtr context)

View File

@ -50,7 +50,7 @@ public:
clock_type = clock_type_;
}
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
void setProgressCallback(ContextPtr context);

View File

@ -52,9 +52,9 @@ bool ReadBufferFromFileDecorator::nextImpl()
return result;
}
size_t ReadBufferFromFileDecorator::getFileSize()
std::optional<size_t> ReadBufferFromFileDecorator::tryGetFileSize()
{
return getFileSizeFromReadBuffer(*impl);
return tryGetFileSizeFromReadBuffer(*impl);
}
}

View File

@ -27,7 +27,7 @@ public:
ReadBuffer & getWrappedReadBuffer() { return *impl; }
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
protected:
std::unique_ptr<SeekableReadBuffer> impl;

View File

@ -253,7 +253,7 @@ void ReadBufferFromFileDescriptor::rewind()
file_offset_of_buffer_end = 0;
}
size_t ReadBufferFromFileDescriptor::getFileSize()
std::optional<size_t> ReadBufferFromFileDescriptor::tryGetFileSize()
{
return getSizeFromFileDescriptor(fd, getFileName());
}

View File

@ -69,7 +69,7 @@ public:
/// Seek to the beginning, discarding already read data if any. Useful to reread file that changes on every read.
void rewind();
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
bool checkIfActuallySeekable() override;

View File

@ -311,15 +311,15 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence)
return offset;
}
size_t ReadBufferFromS3::getFileSize()
std::optional<size_t> ReadBufferFromS3::tryGetFileSize()
{
if (file_size)
return *file_size;
return file_size;
auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id);
file_size = object_size;
return *file_size;
return file_size;
}
off_t ReadBufferFromS3::getPosition()

View File

@ -63,7 +63,7 @@ public:
off_t getPosition() override;
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
void setReadUntilPosition(size_t position) override;
void setReadUntilEnd() override;

View File

@ -72,7 +72,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int SEEK_POSITION_OUT_OF_BOUND;
extern const int UNKNOWN_FILE_SIZE;
}
std::unique_ptr<ReadBuffer> ReadWriteBufferFromHTTP::CallResult::transformToReadBuffer(size_t buf_size) &&
@ -121,15 +120,33 @@ void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, s
credentials.authenticate(request);
}
size_t ReadWriteBufferFromHTTP::getFileSize()
std::optional<size_t> ReadWriteBufferFromHTTP::tryGetFileSize()
{
if (!file_info)
file_info = getFileInfo();
{
try
{
file_info = getFileInfo();
}
catch (const HTTPException &)
{
return std::nullopt;
}
catch (const NetException &)
{
return std::nullopt;
}
catch (const Poco::Net::NetException &)
{
return std::nullopt;
}
catch (const Poco::IOException &)
{
return std::nullopt;
}
}
if (file_info->file_size)
return *file_info->file_size;
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", initial_uri.toString());
return file_info->file_size;
}
bool ReadWriteBufferFromHTTP::supportsReadAt()
@ -311,12 +328,12 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
error_message = e.displayText();
exception = std::current_exception();
}
catch (DB::NetException & e)
catch (NetException & e)
{
error_message = e.displayText();
exception = std::current_exception();
}
catch (DB::HTTPException & e)
catch (HTTPException & e)
{
if (!isRetriableError(e.getHTTPStatus()))
is_retriable = false;
@ -324,7 +341,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
error_message = e.displayText();
exception = std::current_exception();
}
catch (DB::Exception & e)
catch (Exception & e)
{
is_retriable = false;
@ -683,7 +700,19 @@ std::optional<time_t> ReadWriteBufferFromHTTP::tryGetLastModificationTime()
{
file_info = getFileInfo();
}
catch (...)
catch (const HTTPException &)
{
return std::nullopt;
}
catch (const NetException &)
{
return std::nullopt;
}
catch (const Poco::Net::NetException &)
{
return std::nullopt;
}
catch (const Poco::IOException &)
{
return std::nullopt;
}
@ -704,7 +733,7 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo()
{
getHeadResponse(response);
}
catch (HTTPException & e)
catch (const HTTPException & e)
{
/// Maybe the web server doesn't support HEAD requests.
/// E.g. webhdfs reports status 400.

View File

@ -118,7 +118,7 @@ private:
std::unique_ptr<ReadBuffer> initialize();
size_t getFileSize() override;
std::optional<size_t> tryGetFileSize() override;
bool supportsReadAt() override;

View File

@ -13,41 +13,47 @@ namespace ErrorCodes
extern const int UNKNOWN_FILE_SIZE;
}
template <typename T>
static size_t getFileSize(T & in)
size_t WithFileSize::getFileSize()
{
if (auto * with_file_size = dynamic_cast<WithFileSize *>(&in))
{
return with_file_size->getFileSize();
}
if (auto maybe_size = tryGetFileSize())
return *maybe_size;
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size");
}
size_t getFileSizeFromReadBuffer(ReadBuffer & in)
template <typename T>
static std::optional<size_t> tryGetFileSize(T & in)
{
if (auto * delegate = dynamic_cast<ReadBufferFromFileDecorator *>(&in))
{
return getFileSize(delegate->getWrappedReadBuffer());
}
else if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
{
return getFileSize(compressed->getWrappedReadBuffer());
}
if (auto * with_file_size = dynamic_cast<WithFileSize *>(&in))
return with_file_size->tryGetFileSize();
return getFileSize(in);
return std::nullopt;
}
template <typename T>
static size_t getFileSize(T & in)
{
if (auto maybe_size = tryGetFileSize(in))
return *maybe_size;
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size");
}
std::optional<size_t> tryGetFileSizeFromReadBuffer(ReadBuffer & in)
{
try
{
return getFileSizeFromReadBuffer(in);
}
catch (...)
{
return std::nullopt;
}
if (auto * delegate = dynamic_cast<ReadBufferFromFileDecorator *>(&in))
return tryGetFileSize(delegate->getWrappedReadBuffer());
else if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
return tryGetFileSize(compressed->getWrappedReadBuffer());
return tryGetFileSize(in);
}
size_t getFileSizeFromReadBuffer(ReadBuffer & in)
{
if (auto maybe_size = tryGetFileSizeFromReadBuffer(in))
return *maybe_size;
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size");
}
bool isBufferWithFileSize(const ReadBuffer & in)

View File

@ -10,15 +10,16 @@ class ReadBuffer;
class WithFileSize
{
public:
virtual size_t getFileSize() = 0;
/// Returns nullopt if couldn't find out file size;
virtual std::optional<size_t> tryGetFileSize() = 0;
virtual ~WithFileSize() = default;
size_t getFileSize();
};
bool isBufferWithFileSize(const ReadBuffer & in);
size_t getFileSizeFromReadBuffer(ReadBuffer & in);
/// Return nullopt if couldn't find out file size;
std::optional<size_t> tryGetFileSizeFromReadBuffer(ReadBuffer & in);
size_t getDataOffsetMaybeCompressed(const ReadBuffer & in);

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)
{
String archive_path = "archive.tar";

View File

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

View File

@ -13,10 +13,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
@ -237,16 +233,8 @@ void SubstituteColumnOptimizer::perform()
const auto & compare_graph = metadata_snapshot->getConstraints().getGraph();
// Fill aliases
if (select_query->select())
{
auto * list = select_query->refSelect()->as<ASTExpressionList>();
if (!list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "List of selected columns must be ASTExpressionList");
for (ASTPtr & ast : list->children)
ast->setAlias(ast->getAliasOrColumnName());
}
if (compare_graph.getNumOfComponents() == 0)
return;
auto run_for_all = [&](const auto func)
{

View File

@ -15,7 +15,7 @@ struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
/// Optimizer that tries to replace columns to equal columns (according to constraints)
/// with lower size (according to compressed and uncomressed size).
/// with lower size (according to compressed and uncompressed sizes).
class SubstituteColumnOptimizer
{
public:

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)
{
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.
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]);
res[i] = convertFieldToType(src_tuple[i], element_type);
if (!res[i].isNull() || element_type.isNullable())
continue;
/*
* Either the source element was Null, or the conversion did not
* succeed, because the source and the requested types of the
* element are compatible, but the value is not convertible
* (e.g. trying to convert -1 from Int8 to UInt8). In these
* cases, consider the whole tuple also compatible but not
* convertible. According to the specification of this function,
* we must return Null in this case.
*
* The following elements might be not even compatible, so it
* makes sense to check them to detect user errors. Remember
* that there is an unconvertible element, and try to process
* the remaining ones. The convertFieldToType for each element
* will throw if it detects incompatibility.
*/
have_unconvertible_element = true;
if (res[i].isNull() && !canContainNull(element_type))
{
/*
* Either the source element was Null, or the conversion did not
* succeed, because the source and the requested types of the
* element are compatible, but the value is not convertible
* (e.g. trying to convert -1 from Int8 to UInt8). In these
* cases, consider the whole tuple also compatible but not
* convertible. According to the specification of this function,
* we must return Null in this case.
*
* The following elements might be not even compatible, so it
* makes sense to check them to detect user errors. Remember
* that there is an unconvertible element, and try to process
* the remaining ones. The convertFieldToType for each element
* will throw if it detects incompatibility.
*/
have_unconvertible_element = true;
}
}
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);
if (updated_entry[0].isNull() && !key_type.isNullable())
if (updated_entry[0].isNull() && !canContainNull(key_type))
have_unconvertible_element = true;
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;
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)
{
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());
Field converted = convertFieldToType(from_value, to_type, from_type_hint);

View File

@ -16,16 +16,9 @@
namespace DB
{
static constinit std::atomic<bool> allow_logging{true};
void OwnSplitChannel::disableLogging()
{
allow_logging = false;
}
void OwnSplitChannel::log(const Poco::Message & msg)
{
if (!allow_logging)
if (!isLoggingEnabled())
return;
#ifndef WITHOUT_TEXT_LOG

View File

@ -39,8 +39,6 @@ public:
void setLevel(const std::string & name, int level);
static void disableLogging();
private:
void logSplit(const Poco::Message & msg);
void tryLogSplit(const Poco::Message & msg);

View File

@ -235,7 +235,13 @@ void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatS
else if (on_expression)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "");
/// If there is an alias for the whole expression parens should be added, otherwise it will be invalid syntax
bool on_has_alias = !on_expression->tryGetAlias().empty();
if (on_has_alias)
settings.ostr << "(";
on_expression->formatImpl(settings, state, frame);
if (on_has_alias)
settings.ostr << ")";
}
}

View File

@ -66,7 +66,7 @@ public:
/** Set the alias. */
virtual void setAlias(const String & /*to*/)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't set alias of {}", getColumnName());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't set alias of {} of {}", getColumnName(), getID());
}
/** Get the text that identifies this element. */

View File

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

View File

@ -46,7 +46,7 @@ JSONEachRowRowInputFormat::JSONEachRowRowInputFormat(
{
const auto & header = getPort().getHeader();
name_map = header.getNamesToIndexesMap();
if (format_settings_.json.ignore_key_case)
if (format_settings_.json.case_insensitive_column_matching)
{
for (auto & it : name_map)
{
@ -181,7 +181,7 @@ void JSONEachRowRowInputFormat::readJSONObject(MutableColumns & columns)
continue;
}
size_t column_index = 0;
if (format_settings.json.ignore_key_case)
if (format_settings.json.case_insensitive_column_matching)
{
String lower_case_name = transformFieldNameToLowerCase(name_ref);
StringRef field_name_ref = lower_case_name_map[lower_case_name];

View File

@ -23,7 +23,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::
return 0;
const auto & table_join = join->getJoin()->getTableJoin();
if (table_join.strictness() == JoinStrictness::Asof)
/// Any JOIN issue https://github.com/ClickHouse/ClickHouse/issues/66447
/// Anti JOIN issue https://github.com/ClickHouse/ClickHouse/issues/67156
if (table_join.strictness() != JoinStrictness::All)
return 0;
/// TODO: Support join_use_nulls

View File

@ -53,7 +53,7 @@ public:
bool nextImpl() override;
off_t seek(off_t off, int whence) override;
off_t getPosition() override;
size_t getFileSize() override { return remote_file_size; }
std::optional<size_t> tryGetFileSize() override { return remote_file_size; }
private:
std::unique_ptr<LocalFileHolder> local_file_holder;

View File

@ -418,8 +418,11 @@ namespace
}
StorageKafka::StorageKafka(
const StorageID & table_id_, ContextPtr context_,
const ColumnsDescription & columns_, std::unique_ptr<KafkaSettings> kafka_settings_,
const StorageID & table_id_,
ContextPtr context_,
const ColumnsDescription & columns_,
const String & comment,
std::unique_ptr<KafkaSettings> kafka_settings_,
const String & collection_name_)
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
@ -451,6 +454,7 @@ StorageKafka::StorageKafka(
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
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");
}
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, });

Some files were not shown because too many files have changed in this diff Show More