Merge branch 'fix-data-race-in-zookeeper' of github.com:ClickHouse/ClickHouse into fix-data-race-in-zookeeper

This commit is contained in:
Alexey Milovidov 2024-07-27 12:10:58 +02:00
commit 9d612a192e
129 changed files with 1601 additions and 1077 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

@ -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

@ -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;
@ -2241,6 +2240,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 +2322,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())
{
@ -2415,7 +2421,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;
}
}
@ -2840,168 +2847,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

@ -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

@ -1014,8 +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);
original_index.store(-1);
auto expire_session_if_not_expired = [&]
{
/// No new requests will appear in queue after finish()

View File

@ -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

@ -64,7 +64,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."},

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

@ -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

@ -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

@ -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

@ -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

@ -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

@ -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, });

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -60,6 +60,13 @@ def test_pkill(started_node):
def test_pkill_query_log(started_node):
if (
started_node.is_built_with_thread_sanitizer()
or started_node.is_built_with_address_sanitizer()
or started_node.is_built_with_memory_sanitizer()
):
pytest.skip("doesn't fit in timeouts for stacktrace generation")
for signal in ["SEGV", "4"]:
# force create query_log if it was not created
started_node.query("SYSTEM FLUSH LOGS")

View File

@ -3,10 +3,10 @@ DROP TABLE IF EXISTS mergetree_00673;
CREATE TABLE mergetree_00673 (x UInt64) ENGINE = MergeTree ORDER BY x;
INSERT INTO mergetree_00673 VALUES (1);
SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(10000000))))))))))));
SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(1000000))))))))))))))))))))));
SET force_primary_key = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(10000000))))))))))));
SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM (SELECT * FROM mergetree_00673 WHERE x IN (SELECT * FROM numbers(1000000))))))))))))))))))))));
DROP TABLE mergetree_00673;

View File

@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error -nm 2>/dev/null &
yes 'DROP TABLE table;' | head -n 1000 | $CLICKHOUSE_CLIENT --ignore-error -nm 2>/dev/null &
yes 'CREATE TABLE IF NOT EXISTS table (x UInt8) ENGINE = MergeTree ORDER BY tuple();' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery &
yes 'DROP TABLE IF EXISTS table;' | head -n 1000 | $CLICKHOUSE_CLIENT --multiquery &
wait
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: race
# Tags: race, no-parallel
# This is a monkey test used to trigger sanitizers.

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-ordinary-database, no-debug
# Tags: long, no-parallel, no-ordinary-database, no-debug
# Test is too heavy, avoid parallel run in Flaky Check
# shellcheck disable=SC2119

View File

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

View File

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

View File

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

View File

@ -1,12 +1,12 @@
10000000
10000000 1274991808
20000000
20000000 2549983616
30000000
30000000 3824991808
10000000
10000000 1274991808
20000000
20000000 2549983616
30000000
30000000 3824991808
10000000
10000000 1274991808
20000000
20000000 2549983616
30000000
30000000 3824991808

View File

@ -8,7 +8,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;
@ -23,7 +23,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;
@ -38,7 +38,7 @@ INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000);
INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000, 20000000);
select sum(length(arr)) from perf_lc_num;
select sum(length(arr)), sum(num) from perf_lc_num;

View File

@ -41,7 +41,7 @@ SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple
SELECT 'dictHas';
SELECT dictHas('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes ORDER BY ALL;
DROP DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
DROP TABLE 01753_dictionary_db.simple_key_simple_attributes_source_table;

View File

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

View File

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

File diff suppressed because it is too large Load Diff

View File

@ -45,59 +45,59 @@ SELECT 'JOIN {{ join_type }}';
SELECT 'JOIN ON without conditions';
SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value
FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id;
FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT t1.value, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER }
SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER }
SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError AMBIGUOUS_IDENTIFIER }
SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id ORDER BY ALL; -- { serverError AMBIGUOUS_IDENTIFIER }
SELECT 'JOIN ON with conditions';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0';
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0';
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0' ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0';
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0';
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0' ORDER BY ALL;
SELECT 'JOIN multiple clauses';
SELECT t1.id, t1.value, t2.id, t2.value
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id;
FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id ORDER BY ALL;
SELECT 'JOIN expression aliases';
SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id);
SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) ORDER BY ALL;
SELECT '--';
SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id;
SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id ORDER BY ALL;
{% endfor %}
@ -110,56 +110,56 @@ SELECT 'JOIN ON without conditions';
SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value
FROM test_table_join_1 {{ first_join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id
{{ second_join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id;
{{ second_join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id;
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL;
SELECT '--';
SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value,
t3.id, test_table_join_3.id, t3.value, test_table_join_3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id;
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL;
SELECT '--';
SELECT t1.value, t2.value, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id;
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id ORDER BY ALL;
SELECT 'JOIN ON with conditions';
SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0'
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0';
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' ORDER BY ALL;
SELECT '--';
SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0'
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0';
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0' ORDER BY ALL;
SELECT 'JOIN multiple clauses';
SELECT t1.value, t2.value, t3.value
FROM test_table_join_1 AS t1
{{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id;
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id ORDER BY ALL;
SELECT 'JOIN expression aliases';
SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id)
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id);
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id) ORDER BY ALL;
SELECT '--';
SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value
FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = t3_id;
{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = t3_id ORDER BY ALL;
{% endfor %}
{% endfor %}

View File

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

View File

@ -1,27 +1,27 @@
-- { echoOn }
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String
1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String
SELECT '--';
--
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String)
1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String)
2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String)
SELECT '--';
--
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String
1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String
\N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String
SELECT '--';
--
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String)
1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String)
2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String)
@ -30,14 +30,14 @@ SELECT '--';
--
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String
1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String
SELECT '--';
--
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
0 UInt64 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String)
1 UInt64 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String)
2 UInt64 2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String)
@ -45,7 +45,7 @@ SELECT '--';
--
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
0 UInt64 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String
1 UInt64 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String
3 UInt64 \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String
@ -53,7 +53,7 @@ SELECT '--';
--
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
0 Nullable(UInt64) 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String)
1 Nullable(UInt64) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String)
2 Nullable(UInt64) 2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String)

View File

@ -26,46 +26,46 @@ INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3');
-- { echoOn }
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id;
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY ALL;
SELECT '--';
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
SELECT '--';
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
SELECT '--';
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
SELECT '--';
SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value),
t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value)
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id);
FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY ALL;
-- { echoOff }

View File

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

View File

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

View File

@ -0,0 +1,4 @@
create table dist as system.one engine=Distributed(test_shard_localhost, system, one);
select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=1 format Null;
select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=1, use_hedged_requests=0 format Null;
select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=0 format Null;

View File

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

View File

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

View File

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

View File

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

View File

@ -39,11 +39,22 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N
2 2 2 2
3 3 3 33
\N \N \N \N
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND t2.x <> t1.x ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t2.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION }
-- aliases defined in the join condition are valid
-- FIXME(@vdimir) broken query formatting for the following queries:
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
1 42 \N \N \N 0
2 2 2 2 1 1
3 3 3 33 1 1
\N \N 4 42 \N 0
\N \N \N \N \N 1
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
1 42 \N \N \N 0
2 2 2 2 1 1
3 3 3 33 1 1
\N \N 4 42 \N 0
\N \N \N \N \N 0
\N \N \N \N \N 0
-- check for non-nullable columns for which `is null` is replaced with constant
SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
2 2 2 2

View File

@ -35,11 +35,12 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND t2.x <> t1.x ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( t2.x <> t1.x AND (t2.x IS NULL) AND (t2.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- { serverError INVALID_JOIN_ON_EXPRESSION }
-- aliases defined in the join condition are valid
-- FIXME(@vdimir) broken query formatting for the following queries:
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST;
-- check for non-nullable columns for which `is null` is replaced with constant
SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST;

View File

@ -8,12 +8,12 @@
[1,4,5]
[]
[]
1000000
999999
100000
99999
[9]
['a','c']
1000000
999999
50000
49999
['1']
[]
['2023-01-01 00:00:00']

View File

@ -39,15 +39,15 @@ DROP TABLE test_numbers;
DROP TABLE IF EXISTS test_big_numbers_sep;
CREATE TABLE test_big_numbers_sep (a Array(Int64)) engine=MergeTree ORDER BY a;
INSERT INTO test_big_numbers_sep SELECT array(number) FROM numbers_mt(1000000);
INSERT INTO test_big_numbers_sep SELECT array(number) FROM numbers_mt(100000);
SELECT groupArrayIntersect(*) FROM test_big_numbers_sep;
DROP TABLE test_big_numbers_sep;
DROP TABLE IF EXISTS test_big_numbers;
CREATE TABLE test_big_numbers (a Array(Int64)) engine=MergeTree ORDER BY a;
INSERT INTO test_big_numbers SELECT range(1000000);
INSERT INTO test_big_numbers SELECT range(100000);
SELECT length(groupArrayIntersect(*)) FROM test_big_numbers;
INSERT INTO test_big_numbers SELECT range(999999);
INSERT INTO test_big_numbers SELECT range(99999);
SELECT length(groupArrayIntersect(*)) FROM test_big_numbers;
INSERT INTO test_big_numbers VALUES ([9]);
SELECT groupArrayIntersect(*) FROM test_big_numbers;
@ -63,9 +63,9 @@ DROP TABLE test_string;
DROP TABLE IF EXISTS test_big_string;
CREATE TABLE test_big_string (a Array(String)) engine=MergeTree ORDER BY a;
INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(1000000);
INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(50000);
SELECT length(groupArrayIntersect(*)) FROM test_big_string;
INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(999999);
INSERT INTO test_big_string SELECT groupArray(toString(number)) FROM numbers_mt(49999);
SELECT length(groupArrayIntersect(*)) FROM test_big_string;
INSERT INTO test_big_string VALUES (['1']);
SELECT groupArrayIntersect(*) FROM test_big_string;

View File

@ -1,4 +1,8 @@
SYSTEM FLUSH LOGS;
SELECT 'Column ' || name || ' from table ' || concat(database, '.', table) || ' should have a comment'
FROM system.columns
WHERE (database = 'system') AND (comment = '') AND (table NOT ILIKE '%_log_%') AND (table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log')) AND (default_kind != 'ALIAS');
WHERE (database = 'system') AND
(comment = '') AND
(table NOT ILIKE '%log%') AND
(table NOT IN ('numbers', 'numbers_mt', 'one', 'generate_series', 'generateSeries', 'coverage_log', 'filesystem_read_prefetches_log')) AND
(default_kind != 'ALIAS');

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