Merge branch 'master' of github.com:ClickHouse/ClickHouse into zvonand-partmut

This commit is contained in:
Andrey Zvonov 2024-07-27 10:05:25 +00:00
commit 3c2d8023e1
77 changed files with 1227 additions and 789 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

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

@ -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;
@ -2848,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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1014,9 +1014,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
LOG_INFO(log, "Finalizing session {}. finalization_started: {}, queue_finished: {}, reason: '{}'",
session_id, already_started, requests_queue.isFinished(), reason);
/// Reset the original index.
original_index = -1;
auto expire_session_if_not_expired = [&]
{
/// No new requests will appear in queue after finish()

View File

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

View File

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

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

View File

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

View File

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

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

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

View File

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

View File

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

View File

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

View File

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

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

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

@ -10,18 +10,38 @@ DEBUG_LOG = os.path.join(
os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog",
)
STATE_MAP = {
-1: "process did not start",
0: "completion was found",
1: "process started and said ':)'",
2: "completion search was started",
3: "completion is missing",
}
def run_with_timeout(func, args, timeout):
process = multiprocessing.Process(target=func, args=args)
process.start()
process.join(timeout)
for _ in range(5):
state = multiprocessing.Value("i", -1)
process = multiprocessing.Process(
target=func, args=args, kwargs={"state": state}
)
process.start()
process.join(timeout)
if process.is_alive():
process.terminate()
print("Timeout")
if state.value in (0, 3):
return
if process.is_alive():
process.terminate()
if state.value == -1:
continue
print(f"Timeout, state: {STATE_MAP[state.value]}")
return
def test_completion(program, argv, comp_word):
def test_completion(program, argv, comp_word, state=None):
comp_begin = comp_word[:-3]
shell_pid, master = pty.fork()
@ -41,6 +61,8 @@ def test_completion(program, argv, comp_word):
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
state.value = 1
os.write(master, b"SET " + bytes(comp_begin.encode()))
output_b = os.read(master, 4096)
output = output_b.decode()
@ -55,23 +77,28 @@ def test_completion(program, argv, comp_word):
time.sleep(0.01)
os.write(master, b"\t")
state.value = 2
output_b = os.read(master, 4096)
output = output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
# fail fast if there is a bell character in the output,
# meaning no concise completion is found
if "\x07" in output:
print(f"{comp_word}: FAIL")
return
while not comp_word in output:
# fail fast if there is a bell character in the output,
# meaning no concise completion is found
if "\x07" in output:
print(f"{comp_word}: FAIL")
state.value = 3
return
output_b = os.read(master, 4096)
output += output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
print(f"{comp_word}: OK")
state.value = 0
finally:
os.close(master)
debug_log_fd.close()

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;

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

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

@ -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');

View File

@ -13,6 +13,6 @@ cp "$CURDIR"/data_json/key_ignore_case.json $USER_FILES_PATH/
$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl"
$CLICKHOUSE_CLIENT -q "create table test_tbl (id UInt16, reqid UInt32, name String) engine=MergeTree order by id"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_tbl SELECT * FROM file('key_ignore_case.json', 'JSONEachRow') SETTINGS input_format_json_ignore_key_case=true"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_tbl SELECT * FROM file('key_ignore_case.json', 'JSONEachRow') SETTINGS input_format_json_case_insensitive_column_matching=true"
$CLICKHOUSE_CLIENT -q "select * from test_tbl"
$CLICKHOUSE_CLIENT -q "drop table test_tbl"
$CLICKHOUSE_CLIENT -q "drop table test_tbl"

View File

@ -1,5 +1,6 @@
-- Tags: no-parallel
INSERT INTO FUNCTION file('data_03198_table_function_directory_path.csv', 'csv') SELECT '1.csv' SETTINGS engine_file_truncate_on_insert=1;
INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv' SETTINGS engine_file_truncate_on_insert=1;
INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv' SETTINGS engine_file_truncate_on_insert=1;
INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv' SETTINGS engine_file_truncate_on_insert=1;
@ -11,3 +12,4 @@ SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/');
SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/dir');
SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/*/dir', 'csv'); -- { serverError 74, 636 }
SELECT COUNT(*) FROM file('data_03198_table_function_directory_pat'); -- { serverError 400 }
SELECT COUNT(*) FROM file('data_03198_table_function_directory_path.csv');

View File

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

View File

@ -0,0 +1,4 @@
SELECT * FROM t1 INNER JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2)
SELECT * FROM t1 INNER JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2)
SELECT * FROM t1 INNER JOIN t2 ON (t1.x = t2.x) AND ((t1.x IS NULL) AS e2)
SELECT * FROM t1 INNER JOIN t2 ON t1.x = t2.x

View File

@ -0,0 +1,15 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# regression for the JOIN ON alias for the whole expression
phase1="$($CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON ((t1.x = t2.x) AND (t1.x IS NULL) AS e2)")"
echo "$phase1"
# phase 2
$CLICKHOUSE_FORMAT --oneline --query "$phase1"
# other test cases
$CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x) AND (t1.x IS NULL AS e2)"
$CLICKHOUSE_FORMAT --oneline --query "SELECT * FROM t1 JOIN t2 ON t1.x = t2.x"

View File

@ -0,0 +1,13 @@
010101 AggregateFunction(groupArrayIntersect, Array(UInt8))
[1]
1 [2,4,6,8,10]
2 [2,4,6,8,10]
3 [2,4,6,8,10]
5 [2,6,10]
6 [10]
7 []
a [(['2','4','6','8','10'])]
b [(['2','4','6','8','10'])]
c [(['2','4','6','8','10'])]
d []
e []

View File

@ -0,0 +1,43 @@
SELECT hex(groupArrayIntersectState([1]) AS a), toTypeName(a);
SELECT finalizeAggregation(CAST(unhex('010101'), 'AggregateFunction(groupArrayIntersect, Array(UInt8))'));
DROP TABLE IF EXISTS grouparray;
CREATE TABLE grouparray
(
`v` AggregateFunction(groupArrayIntersect, Array(UInt8))
)
ENGINE = Log;
INSERT INTO grouparray Select groupArrayIntersectState([2, 4, 6, 8, 10]::Array(UInt8));
SELECT '1', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
INSERT INTO grouparray Select groupArrayIntersectState([2, 4, 6, 8, 10]::Array(UInt8));
SELECT '2', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
INSERT INTO grouparray Select groupArrayIntersectState([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]::Array(UInt8));
SELECT '3', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
INSERT INTO grouparray Select groupArrayIntersectState([2, 6, 10]::Array(UInt8));
SELECT '5', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
INSERT INTO grouparray Select groupArrayIntersectState([10]::Array(UInt8));
SELECT '6', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
INSERT INTO grouparray Select groupArrayIntersectState([]::Array(UInt8));
SELECT '7', arraySort(groupArrayIntersectMerge(v)) FROM grouparray;
DROP TABLE IF EXISTS grouparray;
DROP TABLE IF EXISTS grouparray_string;
CREATE TABLE grouparray_string
(
`v` AggregateFunction(groupArrayIntersect, Array(Tuple(Array(String))))
)
ENGINE = Log;
INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10'])]);
SELECT 'a', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string;
INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10']), tuple(['2', '4', '6', '8', '10'])]);
SELECT 'b', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string;
INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10']), tuple(['2', '4', '6', '8', '10', '14'])]);
SELECT 'c', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string;
INSERT INTO grouparray_string Select groupArrayIntersectState([tuple(['2', '4', '6', '8', '10', '20']), tuple(['2', '4', '6', '8', '10', '14'])]);
SELECT 'd', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string;
INSERT INTO grouparray_string Select groupArrayIntersectState([]::Array(Tuple(Array(String))));
SELECT 'e', arraySort(groupArrayIntersectMerge(v)) FROM grouparray_string;

View File

@ -0,0 +1,3 @@
1 tx1 US
1 tx2 US
1 tx3 US

View File

@ -0,0 +1,33 @@
DROP TABLE IF EXISTS user_country;
DROP TABLE IF EXISTS user_transactions;
CREATE TABLE user_country (
user_id UInt64,
country String
)
ENGINE = ReplacingMergeTree
ORDER BY user_id;
CREATE TABLE user_transactions (
user_id UInt64,
transaction_id String
)
ENGINE = MergeTree
ORDER BY user_id;
INSERT INTO user_country (user_id, country) VALUES (1, 'US');
INSERT INTO user_transactions (user_id, transaction_id) VALUES (1, 'tx1'), (1, 'tx2'), (1, 'tx3'), (2, 'tx1');
-- Expected 3 rows, got only 1. Removing 'ANY' and adding 'FINAL' fixes
-- the issue (but it is not always possible). Moving filter by 'country' to
-- an outer query doesn't help. Query without filter by 'country' works
-- as expected (returns 3 rows).
SELECT * FROM user_transactions
ANY LEFT JOIN user_country USING (user_id)
WHERE
user_id = 1
AND country = 'US'
ORDER BY ALL;
DROP TABLE user_country;
DROP TABLE user_transactions;

View File

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

View File

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

View File

@ -0,0 +1,19 @@
DATA
┏━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━┓
┃ c0 ┃ c1 ┃ c2 ┃
┡━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━┩
1. │ 826636805 │ 0 │ │
├───────────┼───────────┼────┤
2. │ 0 │ 150808457 │ │
└───────────┴───────────┴────┘
NUMBER OF ROWS IN FIRST SHOULD BE EQUAL TO SECOND
FISRT
SECOND
1
TO DEBUG I TOOK JUST A SUBQUERY AND IT HAS 1 ROW
THIRD
1
AND I ADDED SINGLE CONDITION THAT CONDITION <>0 THAT IS 1 IN THIRD QUERY AND IT HAS NO RESULT!!!
FOURTH
1

View File

@ -0,0 +1,45 @@
DROP TABLE IF EXISTS t0;
CREATE TABLE t0 (c0 Int32, c1 Int32, c2 String) ENGINE = Log() ;
INSERT INTO t0(c0, c1, c2) VALUES (826636805,0, ''), (0, 150808457, '');
SELECT 'DATA';
SELECT * FROM t0 FORMAT PrettyMonoBlock;
SELECT 'NUMBER OF ROWS IN FIRST SHOULD BE EQUAL TO SECOND';
SELECT 'FISRT';
SELECT left.c2 FROM t0 AS left
LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1))
WHERE (abs ((- ((sign (right_0.c1))))));
SELECT 'SECOND';
SELECT SUM(check <> 0)
FROM
(
SELECT (abs ((- ((sign (right_0.c1)))))) AS `check`
FROM t0 AS left
LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1))
);
SELECT 'TO DEBUG I TOOK JUST A SUBQUERY AND IT HAS 1 ROW';
SELECT 'THIRD';
SELECT (abs ((- ((sign (right_0.c1)))))) AS `check`
FROM t0 AS left
LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1));
SELECT 'AND I ADDED SINGLE CONDITION THAT CONDITION <>0 THAT IS 1 IN THIRD QUERY AND IT HAS NO RESULT!!!';
SELECT 'FOURTH';
SELECT (abs ((- ((sign (right_0.c1)))))) AS `check`
FROM t0 AS left
LEFT ANTI JOIN t0 AS right_0 ON ((left.c0)=(right_0.c1))
WHERE check <> 0;
DROP TABLE t0;

View File

@ -17,11 +17,11 @@ $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE hits_s3_sampled FINAL"
$CLICKHOUSE_CLIENT -q "SYSTEM DROP FILESYSTEM CACHE"
# Warm up the cache
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000"
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000"
query_id=02906_read_from_cache_$RANDOM
$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null SETTINGS filesystem_cache_reserve_space_wait_lock_timeout_milliseconds=2000"
$CLICKHOUSE_CLIENT -nq "
SYSTEM FLUSH LOGS;

View File

@ -1866,6 +1866,7 @@ jdbc
jemalloc
jeprof
joinGet
joinGetOrNull
json
jsonMergePatch
jsonasstring