Merge branch 'master' into add-retries-for-zk-create

This commit is contained in:
Antonio Andelic 2023-06-30 08:42:41 +02:00 committed by GitHub
commit ba1c50e803
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 325 additions and 361 deletions

View File

@ -21,7 +21,7 @@
* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)).
* The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Added option `--rename_files_after_processing <pattern>`. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)).
* Add support for `APPEND` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)).
* Add support for `TRUNCATE` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)).
* Add table engine `Redis` and table function `redis`. It allows querying external Redis servers. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)).
* Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)).
* Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)).
@ -40,12 +40,12 @@
* Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)).
* Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse-client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)).
* Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
#### Experimental Feature
* Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)).
* Add random sleep before large merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)).
* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)).
#### Improvement
* Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)).

View File

@ -0,0 +1,16 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.3.7.5-lts (bc683c11c92) FIXME as compared to v23.3.6.7-lts (7e3f0a271b7)
#### Build/Testing/Packaging Improvement
* Backported in [#51568](https://github.com/ClickHouse/ClickHouse/issues/51568): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)).

View File

@ -0,0 +1,27 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.4.5.22-stable (0ced5d6a8da) FIXME as compared to v23.4.4.16-stable (747ba4fc6a0)
#### Build/Testing/Packaging Improvement
* Backported in [#51530](https://github.com/ClickHouse/ClickHouse/issues/51530): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#51570](https://github.com/ClickHouse/ClickHouse/issues/51570): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)).
* Fix reconnecting of HTTPS session when target host IP was changed [#50240](https://github.com/ClickHouse/ClickHouse/pull/50240) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)).
* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)).
* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Increase max array size in group bitmap [#50620](https://github.com/ClickHouse/ClickHouse/pull/50620) ([Kruglov Pavel](https://github.com/Avogar)).

View File

@ -1322,7 +1322,7 @@ Connection pool size for PostgreSQL table engine and database engine.
Default value: 16
## postgresql_connection_pool_size {#postgresql-connection-pool-size}
## postgresql_connection_pool_wait_timeout {#postgresql-connection-pool-wait-timeout}
Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.

View File

@ -230,13 +230,15 @@ hasAll(set, subset)
**Arguments**
- `set` Array of any type with a set of elements.
- `subset` Array of any type with elements that should be tested to be a subset of `set`.
- `subset` Array of any type that shares a common supertype with `set` containing elements that should be tested to be a subset of `set`.
**Return values**
- `1`, if `set` contains all of the elements from `subset`.
- `0`, otherwise.
Raises an exception `NO_COMMON_TYPE` if the set and subset elements do not share a common supertype.
**Peculiar properties**
- An empty array is a subset of any array.
@ -253,7 +255,7 @@ hasAll(set, subset)
`SELECT hasAll(['a', 'b'], ['a'])` returns 1.
`SELECT hasAll([1], ['a'])` returns 0.
`SELECT hasAll([1], ['a'])` raises a `NO_COMMON_TYPE` exception.
`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` returns 0.
@ -268,13 +270,15 @@ hasAny(array1, array2)
**Arguments**
- `array1` Array of any type with a set of elements.
- `array2` Array of any type with a set of elements.
- `array2` Array of any type that shares a common supertype with `array1`.
**Return values**
- `1`, if `array1` and `array2` have one similar element at least.
- `0`, otherwise.
Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype.
**Peculiar properties**
- `Null` processed as a value.
@ -288,7 +292,7 @@ hasAny(array1, array2)
`SELECT hasAny([-128, 1., 512], [1])` returns `1`.
`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` returns `0`.
`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` raises a `NO_COMMON_TYPE` exception.
`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` returns `1`.
@ -318,6 +322,8 @@ For Example:
- `1`, if `array1` contains `array2`.
- `0`, otherwise.
Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype.
**Peculiar properties**
- The function will return `1` if `array2` is empty.
@ -339,6 +345,9 @@ For Example:
`SELECT hasSubstr(['a', 'b' , 'c'], ['a', 'c'])` returns 0.
`SELECT hasSubstr([[1, 2], [3, 4], [5, 6]], [[1, 2], [3, 4]])` returns 1.
i
`SELECT hasSubstr([1, 2, NULL, 3, 4], ['a'])` raises a `NO_COMMON_TYPE` exception.
## indexOf(arr, x)

View File

@ -22,14 +22,15 @@ tuple(x, y, …)
A function that allows getting a column from a tuple.
If the second argument is a number `n`, it is the column index, starting from 1. If the second argument is a string `s`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or element for such name does not exist, the default value returned instead of throw exception. The second and third arguments if provided are always must be constant. There is no cost to execute the function.
If the second argument is a number `index`, it is the column index, starting from 1. If the second argument is a string `name`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or no element exist for the name, the default value returned instead of throwing an exception. The second and third arguments, if provided, must be constants. There is no cost to execute the function.
The function implements the operator `x.n` and `x.s`.
The function implements operators `x.index` and `x.name`.
**Syntax**
``` sql
tupleElement(tuple, n/s [, default_value])
tupleElement(tuple, index, [, default_value])
tupleElement(tuple, name, [, default_value])
```
## untuple

View File

@ -362,6 +362,16 @@ ZooKeeper::ZooKeeper(
{
tryLogCurrentException(log, "Failed to connect to ZooKeeper");
try
{
requests_queue.finish();
socket.shutdown();
}
catch (...)
{
tryLogCurrentException(log);
}
send_thread.join();
receive_thread.join();

View File

@ -23,7 +23,7 @@ namespace DB
* The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful.
*/
template <typename To, typename From>
To assert_cast(From && from)
inline To assert_cast(From && from)
{
#ifndef NDEBUG
try

View File

@ -1293,6 +1293,16 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id,
assert(checkDigestValid(query_context));
}
bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const
{
/// ReplicatedMergeTree may call commitAlterTable from its background threads when executing ALTER_METADATA entries.
/// It may update the metadata digest (both locally and in ZooKeeper)
/// before DatabaseReplicatedDDLWorker::initializeReplication() has finished.
/// We should not update metadata until the database is initialized.
return ddl_worker && ddl_worker->isCurrentlyActive();
}
void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name)
{
auto txn = local_context->getZooKeeperMetadataTransaction();

View File

@ -48,6 +48,8 @@ public:
/// then it will be executed on all replicas.
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, bool internal) override;
bool canExecuteReplicatedMetadataAlter() const override;
bool hasReplicationThread() const override { return true; }
void stopReplication() override;

View File

@ -91,12 +91,12 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
if (zookeeper->tryGet(database->replica_path + "/digest", digest_str))
{
digest = parse<UInt64>(digest_str);
LOG_TRACE(log, "Metadata digest in ZooKeeper: {}", digest);
std::lock_guard lock{database->metadata_mutex};
local_digest = database->tables_metadata_digest;
}
else
{
LOG_WARNING(log, "Did not find digest in ZooKeeper, creating it");
/// Database was created by old ClickHouse versions, let's create the node
std::lock_guard lock{database->metadata_mutex};
digest = local_digest = database->tables_metadata_digest;
@ -104,6 +104,9 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
zookeeper->create(database->replica_path + "/digest", digest_str, zkutil::CreateMode::Persistent);
}
LOG_TRACE(log, "Trying to initialize replication: our_log_ptr={}, max_log_ptr={}, local_digest={}, zk_digest={}",
our_log_ptr, max_log_ptr, local_digest, digest);
bool is_new_replica = our_log_ptr == 0;
bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr;
bool lost_according_to_digest = database->db_settings.check_consistency && local_digest != digest;
@ -158,7 +161,7 @@ bool DatabaseReplicatedDDLWorker::waitForReplicaToProcessAllEntries(UInt64 timeo
LOG_TRACE(log, "Waiting for worker thread to process all entries before {}, current task is {}", max_log, current_task);
bool processed = wait_current_task_change.wait_for(lock, std::chrono::milliseconds(timeout_ms), [&]()
{
return zookeeper->expired() || current_task == max_log || stop_flag;
return zookeeper->expired() || current_task >= max_log || stop_flag;
});
if (!processed)

View File

@ -254,6 +254,9 @@ public:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: alterTable() is not supported", getEngineName());
}
/// Special method for ReplicatedMergeTree and DatabaseReplicated
virtual bool canExecuteReplicatedMetadataAlter() const { return true; }
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
virtual time_t getObjectMetadataModificationTime(const String & /*name*/) const
{

View File

@ -10,6 +10,7 @@
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/scope_guard_safe.h>
#include <Core/Defines.h>
@ -69,6 +70,11 @@ public:
shards_queues[shard].emplace(backlog);
pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()]
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
/// Do not account memory that was occupied by the dictionaries for the query/user context.
MemoryTrackerBlockerInThread memory_blocker;
@ -230,6 +236,11 @@ HashedDictionary<dictionary_key_type, sparse, sharded>::~HashedDictionary()
pool.trySchedule([&container, thread_group = CurrentThread::getGroup()]
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
/// Do not account memory that was occupied by the dictionaries for the query/user context.
MemoryTrackerBlockerInThread memory_blocker;

View File

@ -17,11 +17,8 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_INDEX;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int NUMBER_OF_DIMENSIONS_MISMATCHED;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
}
namespace
@ -34,32 +31,14 @@ class FunctionTupleElement : public IFunction
{
public:
static constexpr auto name = "tupleElement";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionTupleElement>();
}
String getName() const override
{
return name;
}
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTupleElement>(); }
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override
{
return 0;
}
bool useDefaultImplementationForConstants() const override
{
return true;
}
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
bool useDefaultImplementationForNulls() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
@ -72,193 +51,111 @@ public:
getName(), number_of_arguments);
size_t count_arrays = 0;
const IDataType * tuple_col = arguments[0].type.get();
while (const DataTypeArray * array = checkAndGetDataType<DataTypeArray>(tuple_col))
const IDataType * input_type = arguments[0].type.get();
while (const DataTypeArray * array = checkAndGetDataType<DataTypeArray>(input_type))
{
tuple_col = array->getNestedType().get();
input_type = array->getNestedType().get();
++count_arrays;
}
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(tuple_col);
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(input_type);
if (!tuple)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be tuple or array of tuple. Actual {}",
getName(),
arguments[0].type->getName());
auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments);
std::optional<size_t> index = getElementIndex(arguments[1].column, *tuple, number_of_arguments);
if (index.has_value())
{
DataTypePtr out_return_type = tuple->getElements()[index.value()];
DataTypePtr return_type = tuple->getElements()[index.value()];
for (; count_arrays; --count_arrays)
out_return_type = std::make_shared<DataTypeArray>(out_return_type);
return_type = std::make_shared<DataTypeArray>(return_type);
return out_return_type;
return return_type;
}
else
{
const IDataType * default_col = arguments[2].type.get();
size_t default_argument_count_arrays = 0;
if (const DataTypeArray * array = checkAndGetDataType<DataTypeArray>(default_col))
{
default_argument_count_arrays = array->getNumberOfDimensions();
}
if (count_arrays != default_argument_count_arrays)
{
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED,
"Dimension of types mismatched between first argument and third argument. "
"Dimension of 1st argument: {}. "
"Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays);
}
return arguments[2].type;
}
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
Columns array_offsets;
const auto & input_arg = arguments[0];
const IDataType * input_type = input_arg.type.get();
const IColumn * input_col = input_arg.column.get();
const auto & first_arg = arguments[0];
const IDataType * tuple_type = first_arg.type.get();
const IColumn * tuple_col = first_arg.column.get();
bool first_arg_is_const = false;
if (typeid_cast<const ColumnConst *>(tuple_col))
bool input_arg_is_const = false;
if (typeid_cast<const ColumnConst *>(input_col))
{
tuple_col = assert_cast<const ColumnConst *>(tuple_col)->getDataColumnPtr().get();
first_arg_is_const = true;
input_col = assert_cast<const ColumnConst *>(input_col)->getDataColumnPtr().get();
input_arg_is_const = true;
}
while (const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(tuple_type))
{
const ColumnArray * array_col = assert_cast<const ColumnArray *>(tuple_col);
tuple_type = array_type->getNestedType().get();
tuple_col = &array_col->getData();
Columns array_offsets;
while (const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(input_type))
{
const ColumnArray * array_col = assert_cast<const ColumnArray *>(input_col);
input_type = array_type->getNestedType().get();
input_col = &array_col->getData();
array_offsets.push_back(array_col->getOffsetsPtr());
}
const DataTypeTuple * tuple_type_concrete = checkAndGetDataType<DataTypeTuple>(tuple_type);
const ColumnTuple * tuple_col_concrete = checkAndGetColumn<ColumnTuple>(tuple_col);
if (!tuple_type_concrete || !tuple_col_concrete)
const DataTypeTuple * input_type_as_tuple = checkAndGetDataType<DataTypeTuple>(input_type);
const ColumnTuple * input_col_as_tuple = checkAndGetColumn<ColumnTuple>(input_col);
if (!input_type_as_tuple || !input_col_as_tuple)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be tuple or array of tuple. Actual {}",
getName(),
first_arg.type->getName());
"First argument for function {} must be tuple or array of tuple. Actual {}", getName(), input_arg.type->getName());
auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size());
std::optional<size_t> index = getElementIndex(arguments[1].column, *input_type_as_tuple, arguments.size());
if (!index.has_value())
{
if (!array_offsets.empty())
{
recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size());
}
return arguments[2].column;
}
ColumnPtr res = tuple_col_concrete->getColumns()[index.value()];
ColumnPtr res = input_col_as_tuple->getColumns()[index.value()];
/// Wrap into Arrays
for (auto it = array_offsets.rbegin(); it != array_offsets.rend(); ++it)
res = ColumnArray::create(res, *it);
if (first_arg_is_const)
{
if (input_arg_is_const)
res = ColumnConst::create(res, input_rows_count);
}
return res;
}
private:
void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const
{
for (size_t i = 1; i < depth; ++i)
{
checkArrayOffsets(col_x, col_y);
col_x = assert_cast<const ColumnArray *>(col_x.get())->getDataPtr();
col_y = assert_cast<const ColumnArray *>(col_y.get())->getDataPtr();
}
checkArrayOffsets(col_x, col_y);
}
void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const
{
if (isColumnConst(*col_x))
{
checkArrayOffsetsWithFirstArgConst(col_x, col_y);
}
else if (isColumnConst(*col_y))
{
checkArrayOffsetsWithFirstArgConst(col_y, col_x);
}
else
{
const auto & array_x = *assert_cast<const ColumnArray *>(col_x.get());
const auto & array_y = *assert_cast<const ColumnArray *>(col_y.get());
if (!array_x.hasEqualOffsets(array_y))
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument 3 of function {} have different array sizes", getName());
}
}
}
void checkArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const
{
col_x = assert_cast<const ColumnConst *>(col_x.get())->getDataColumnPtr();
col_y = col_y->convertToFullColumnIfConst();
const auto & array_x = *assert_cast<const ColumnArray *>(col_x.get());
const auto & array_y = *assert_cast<const ColumnArray *>(col_y.get());
const auto & offsets_x = array_x.getOffsets();
const auto & offsets_y = array_y.getOffsets();
ColumnArray::Offset prev_offset = 0;
size_t row_size = offsets_y.size();
for (size_t row = 0; row < row_size; ++row)
{
if (unlikely(offsets_x[0] != offsets_y[row] - prev_offset))
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument 3 of function {} have different array sizes", getName());
}
prev_offset = offsets_y[row];
}
}
std::optional<size_t> getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const
std::optional<size_t> getElementIndex(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t argument_size) const
{
if (checkAndGetColumnConst<ColumnUInt8>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt16>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt32>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt64>(index_column.get()))
{
size_t index = index_column->getUInt(0);
const size_t index = index_column->getUInt(0);
if (index == 0)
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based.");
if (index > 0 && index <= tuple.getElements().size())
return {index - 1};
else
{
if (argument_size == 2)
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with index '{}'", index);
return std::nullopt;
}
if (index > tuple.getElements().size())
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range.");
return std::optional<size_t>(index - 1);
}
else if (const auto * name_col = checkAndGetColumnConst<ColumnString>(index_column.get()))
{
auto index = tuple.tryGetPositionByName(name_col->getValue<String>());
if (index.has_value())
{
return index;
}
std::optional<size_t> index = tuple.tryGetPositionByName(name_col->getValue<String>());
if (argument_size == 2)
if (index.has_value())
return index;
else
{
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue<String>());
if (argument_size == 2)
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue<String>());
return std::nullopt;
}
return std::nullopt;
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,

View File

@ -31,7 +31,8 @@ public:
private:
std::optional<size_t> read_until_position;
size_t file_offset_of_buffer_end = 0;
/// atomic because can be used in log or exception messages while being updated.
std::atomic<size_t> file_offset_of_buffer_end = 0;
};
}

View File

@ -385,11 +385,23 @@ GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size)
void GraceHashJoin::addBucket(Buckets & destination)
{
auto & left_file = tmp_data->createStream(left_sample_block);
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block));
// There could be exceptions from createStream, In ci tests
// there is a certain probability of failure in allocating memory, see memory_tracker_fault_probability.
// It may terminate this thread and leave a broken hash_join, and another thread cores when it tries to
// use the broken hash_join. So we print an exception message here to help debug.
try
{
auto & left_file = tmp_data->createStream(left_sample_block);
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block));
BucketPtr new_bucket = std::make_shared<FileBucket>(destination.size(), left_file, right_file, log);
destination.emplace_back(std::move(new_bucket));
BucketPtr new_bucket = std::make_shared<FileBucket>(destination.size(), left_file, right_file, log);
destination.emplace_back(std::move(new_bucket));
}
catch (...)
{
LOG_ERROR(&Poco::Logger::get("GraceHashJoin"), "Can't create bucket. current buckets size: {}", destination.size());
throw;
}
}
void GraceHashJoin::checkTypesOfKeys(const Block & block) const
@ -626,7 +638,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block)
if (current_block.rows() > 0)
{
std::lock_guard lock(hash_join_mutex);
auto current_buckets = getCurrentBuckets();
if (!isPowerOf2(current_buckets.size())) [[unlikely]]
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Broken buckets. its size({}) is not power of 2", current_buckets.size());
}
if (!hash_join)
hash_join = makeInMemoryJoin();
@ -637,11 +653,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block)
current_block = {};
// Must use the latest buckets snapshot in case that it has been rehashed by other threads.
buckets_snapshot = rehashBuckets(current_buckets.size() * 2);
auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false);
hash_join = nullptr;
buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2);
{
Blocks current_blocks;
current_blocks.reserve(right_blocks.size());

View File

@ -477,7 +477,7 @@ TEST_F(FileCacheTest, get)
}
cv.notify_one();
file_segment2.wait(file_segment2.range().left);
file_segment2.wait(file_segment2.range().right);
file_segment2.complete();
ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED);
});

View File

@ -1448,6 +1448,15 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
LOG_TRACE(LogToStr(out_postpone_reason, log), fmt_string, entry.znode_name, entry.alter_version, head_alter);
return false;
}
auto database_name = storage.getStorageID().database_name;
auto database = DatabaseCatalog::instance().getDatabase(database_name);
if (!database->canExecuteReplicatedMetadataAlter())
{
LOG_TRACE(LogToStr(out_postpone_reason, log), "Cannot execute alter metadata {} with version {} "
"because database {} cannot process metadata alters now", entry.znode_name, entry.alter_version, database_name);
return false;
}
}
/// If this MUTATE_PART is part of alter modify/drop query, than we have to execute them one by one

View File

@ -38,6 +38,9 @@ def get_options(i, upgrade_check):
client_options.append("join_algorithm='partial_merge'")
if join_alg_num % 5 == 2:
client_options.append("join_algorithm='full_sorting_merge'")
if join_alg_num % 5 == 3 and not upgrade_check:
# Some crashes are not fixed in 23.2 yet, so ignore the setting in Upgrade check
client_options.append("join_algorithm='grace_hash'")
if join_alg_num % 5 == 4:
client_options.append("join_algorithm='auto'")
client_options.append("max_rows_in_join=1000")

View File

@ -45,6 +45,8 @@ def cluster():
f"CREATE TABLE data{i} (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'def', min_bytes_for_wide_part=1;"
)
node.query("SYSTEM STOP MERGES")
for _ in range(10):
node.query(
f"INSERT INTO data{i} SELECT number FROM numbers(500000 * {i+1})"

View File

@ -1,6 +1,6 @@
. {
hosts /example.com {
reload "200ms"
reload "20ms"
fallthrough
}
forward . 127.0.0.11

View File

@ -1,6 +1,6 @@
. {
hosts /example.com {
reload "200ms"
reload "20ms"
fallthrough
}
forward . 127.0.0.11

View File

@ -2,10 +2,11 @@ import pytest
from helpers.client import Client
from helpers.cluster import ClickHouseCluster
from helpers.ssl_context import WrapSSLContextWithSNI
import urllib.request, urllib.parse
import ssl
import os.path
from os import remove
import urllib3
import logging
# The test cluster is configured with certificate for that host name, see 'server-ext.cnf'.
@ -14,6 +15,7 @@ SSL_HOST = "integration-tests.clickhouse.com"
HTTPS_PORT = 8443
# It's important for the node to work at this IP because 'server-cert.pem' requires that (see server-ext.cnf).
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
MAX_RETRY = 5
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
@ -164,17 +166,19 @@ def get_ssl_context(cert_name):
def execute_query_https(
query, user, enable_ssl_auth=True, cert_name=None, password=None
):
url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}"
headers = {"X-ClickHouse-User": user}
url = (
f"https://{instance.ip_address}:{HTTPS_PORT}/?query={urllib.parse.quote(query)}"
)
request = urllib.request.Request(url)
request.add_header("X-ClickHouse-User", user)
if enable_ssl_auth:
headers["X-ClickHouse-SSL-Certificate-Auth"] = "on"
request.add_header("X-ClickHouse-SSL-Certificate-Auth", "on")
if password:
headers["X-ClickHouse-Key"] = password
http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name))
response = http_client.request("GET", url, headers=headers)
if response.status != 200:
raise Exception(response.status)
return response.data.decode("utf-8")
request.add_header("X-ClickHouse-Key", password)
response = urllib.request.urlopen(
request, context=get_ssl_context(cert_name)
).read()
return response.decode("utf-8")
def test_https():
@ -198,10 +202,18 @@ def test_https_wrong_cert():
execute_query_https("SELECT currentUser()", user="john", cert_name="client2")
assert "403" in str(err.value)
count = 0
# Wrong certificate: self-signed certificate.
with pytest.raises(Exception) as err:
execute_query_https("SELECT currentUser()", user="john", cert_name="wrong")
assert "unknown ca" in str(err.value)
while count <= MAX_RETRY:
with pytest.raises(Exception) as err:
execute_query_https("SELECT currentUser()", user="john", cert_name="wrong")
err_str = str(err.value)
if count < MAX_RETRY and "Broken pipe" in err_str:
count = count + 1
logging.warning(f"Failed attempt with wrong cert, err: {err_str}")
continue
assert "unknown ca" in err_str
break
# No certificate.
with pytest.raises(Exception) as err:
@ -291,24 +303,45 @@ def test_https_non_ssl_auth():
== "jane\n"
)
count = 0
# However if we send a certificate it must not be wrong.
with pytest.raises(Exception) as err:
execute_query_https(
"SELECT currentUser()",
user="peter",
enable_ssl_auth=False,
cert_name="wrong",
)
assert "unknown ca" in str(err.value)
with pytest.raises(Exception) as err:
execute_query_https(
"SELECT currentUser()",
user="jane",
enable_ssl_auth=False,
password="qwe123",
cert_name="wrong",
)
assert "unknown ca" in str(err.value)
while count <= MAX_RETRY:
with pytest.raises(Exception) as err:
execute_query_https(
"SELECT currentUser()",
user="peter",
enable_ssl_auth=False,
cert_name="wrong",
)
err_str = str(err.value)
if count < MAX_RETRY and "Broken pipe" in err_str:
count = count + 1
logging.warning(
f"Failed attempt with wrong cert, user: peter, err: {err_str}"
)
continue
assert "unknown ca" in err_str
break
count = 0
while count <= MAX_RETRY:
with pytest.raises(Exception) as err:
execute_query_https(
"SELECT currentUser()",
user="jane",
enable_ssl_auth=False,
password="qwe123",
cert_name="wrong",
)
err_str = str(err.value)
if count < MAX_RETRY and "Broken pipe" in err_str:
count = count + 1
logging.warning(
f"Failed attempt with wrong cert, user: jane, err: {err_str}"
)
continue
assert "unknown ca" in err_str
break
def test_create_user():

View File

@ -323,7 +323,7 @@ def test_concurrent_queries(started_cluster):
)
)
print(count)
assert count <= 18
assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat
busy_pool = Pool(30)
p = busy_pool.map_async(node_insert, range(30))
@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster):
)
)
print(count)
assert count <= 18
assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat
busy_pool = Pool(30)
p = busy_pool.map_async(node_insert_select, range(30))
@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster):
)
)
print(count)
assert count <= 18
assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat
node1.query("DROP TABLE test.test_table;")
node1.query("DROP TABLE test.stat;")

View File

@ -64,7 +64,7 @@ function alter_table()
if [ -z "$table" ]; then continue; fi
$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \
"alter table $table update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \
2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED
2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED | grep -Fv "Error while executing table function merge"
sleep 0.$RANDOM
done
}

View File

@ -1,5 +1,6 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tags: no-fasttest, long, no-debug, no-s3-storage
# This test is too slow with S3 storage and debug modes.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,64 +0,0 @@
#!/usr/bin/expect -f
# This is a test for system.warnings. Testing in interactive mode is necessary,
# as we want to see certain warnings from client
set basedir [file dirname $argv0]
set basename [file tail $argv0]
exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0
set history_file $env(CLICKHOUSE_TMP)/$basename.history
log_user 0
set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
-i $any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
-i $any_spawn_id timeout { exit 1 }
}
#
# Check that the query will fail in clickhouse-client
#
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file"
expect ":) "
send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r"
expect "Code: 241"
expect ":) "
# Exit.
send -- "\4"
expect eof
#
# Check that the query will fail in clickhouse-client
#
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file"
expect ":) "
send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r"
expect "Code: 241"
expect ":) "
# Exit.
send -- "\4"
expect eof
#
# Check that the query will not fail (due to max_untracked_memory)
#
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file"
expect ":) "
send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r"
expect "60000"
expect ":) "
# Exit.
send -- "\4"
expect eof

View File

@ -0,0 +1,8 @@
#!/usr/bin/bash -f
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }"
$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000"

View File

@ -14,12 +14,12 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 2) FROM t_tuple_element;
SELECT tupleElement(t1, 'a') FROM t_tuple_element;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element;
SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError 43 }
SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError 42 }
SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 10, 47 }
SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError 43 }
SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER }
SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX }
SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX }
SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT t2.1 FROM t_tuple_element;
EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element;
@ -27,11 +27,11 @@ EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element;
SELECT tupleElement(t2, 1) FROM t_tuple_element;
EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element;
SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 }
SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 10, 47 }
SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError 43 }
SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER }
SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX }
SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX }
SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
DROP TABLE t_tuple_element;

View File

@ -30,7 +30,7 @@ SELECT 'skipped';
{% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%}
SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}1M{% else %}0{% endif %}';
SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}16M{% else %}0{% endif %}';
SELECT '-- {{ join_algorithm }} --';
SET join_algorithm = '{{ join_algorithm }}';

View File

@ -12,8 +12,8 @@ SELECT * FROM t_tuple_numeric FORMAT JSONEachRow;
SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric;
SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric;
SELECT t.1.3 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX}
SELECT t.4 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX}
SELECT t.1.3 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK}
SELECT t.4 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK}
SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; -- {serverError UNKNOWN_IDENTIFIER}
DROP TABLE t_tuple_numeric;

View File

@ -1,26 +1,15 @@
z
SELECT tupleElement(t1, \'z\', \'z\')
FROM t_tuple_element_default
0
SELECT tupleElement(t1, \'z\', 0)
FROM t_tuple_element_default
z
SELECT tupleElement(t2, \'z\', \'z\')
FROM t_tuple_element_default
--------------------
[(3,4)]
SELECT tupleElement([(1, 2)], \'a\', [(3, 4)])
--------------------
SELECT tupleElement(t1, \'a\', [tuple(1)])
FROM t_tuple_element_default
--------------------
[(0)]
SELECT tupleElement(t1, \'a\', [tuple(0)])
FROM t_tuple_element_default
[0]
SELECT tupleElement(t1, \'a\', [0])
FROM t_tuple_element_default
[0]
[0]
SELECT tupleElement(t1, \'a\', [0])
FROM t_tuple_element_default
hello
world
default
default
[(['a'],1)]
[1,3]
[2,4]
default
--------
hello
world
default
default
[(['a'],1)]
[[1,2,3]]

View File

@ -1,50 +1,23 @@
DROP TABLE IF EXISTS t_tuple_element_default;
-- const tuple argument
CREATE TABLE t_tuple_element_default(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory;
INSERT INTO t_tuple_element_default VALUES ((1, 'a'), (2, 'b'));
SELECT tupleElement(('hello', 'world'), 1, 'default');
SELECT tupleElement(('hello', 'world'), 2, 'default');
SELECT tupleElement(('hello', 'world'), 3, 'default');
SELECT tupleElement(('hello', 'world'), 'xyz', 'default');
SELECT tupleElement(('hello', 'world'), 3, [([('a')], 1)]); -- arbitrary default value
SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default;
SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default;
SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default;
SELECT tupleElement([(1, 2), (3, 4)], 1, 'default');
SELECT tupleElement([(1, 2), (3, 4)], 2, 'default');
SELECT tupleElement([(1, 2), (3, 4)], 3, 'default');
SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError 127 }
SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError 127 }
SELECT '--------';
DROP TABLE t_tuple_element_default;
-- non-const tuple argument
SELECT '--------------------';
SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError 645 }
SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError 190 }
SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError 190 }
SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4)));
EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4)));
SELECT '--------------------';
CREATE TABLE t_tuple_element_default(t1 Array(Tuple(UInt32)), t2 UInt32) ENGINE = Memory;
SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default;
SELECT '--------------------';
INSERT INTO t_tuple_element_default VALUES ([(1)], 100);
SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default;
SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default;
INSERT INTO t_tuple_element_default VALUES ([(2)], 200);
SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default;
EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default;
DROP TABLE t_tuple_element_default;
SELECT tupleElement(materialize(('hello', 'world')), 1, 'default');
SELECT tupleElement(materialize(('hello', 'world')), 2, 'default');
SELECT tupleElement(materialize(('hello', 'world')), 3, 'default');
SELECT tupleElement(materialize(('hello', 'world')), 'xzy', 'default');
SELECT tupleElement(materialize(('hello', 'world')), 'xzy', [([('a')], 1)]); -- arbitrary default value
SELECT tupleElement([[(count('2147483646'), 1)]], 'aaaa', [[1, 2, 3]]) -- bug #51525

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv
echo -e 'key\nfoo\nbar' > $DATA_FILE
$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap"
$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:"
$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary"
$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap"

View File

@ -2264,6 +2264,7 @@ summap
summingmergetree
sumwithoverflow
superaggregates
supertype
supremum
symlink
symlinks

View File

@ -2,10 +2,12 @@ v23.5.4.25-stable 2023-06-29
v23.5.3.24-stable 2023-06-17
v23.5.2.7-stable 2023-06-10
v23.5.1.3174-stable 2023-06-09
v23.4.5.22-stable 2023-06-29
v23.4.4.16-stable 2023-06-17
v23.4.3.48-stable 2023-06-12
v23.4.2.11-stable 2023-05-02
v23.4.1.1943-stable 2023-04-27
v23.3.7.5-lts 2023-06-29
v23.3.6.7-lts 2023-06-28
v23.3.5.9-lts 2023-06-22
v23.3.4.17-lts 2023-06-17

1 v23.5.4.25-stable 2023-06-29
2 v23.5.3.24-stable 2023-06-17
3 v23.5.2.7-stable 2023-06-10
4 v23.5.1.3174-stable 2023-06-09
5 v23.4.5.22-stable 2023-06-29
6 v23.4.4.16-stable 2023-06-17
7 v23.4.3.48-stable 2023-06-12
8 v23.4.2.11-stable 2023-05-02
9 v23.4.1.1943-stable 2023-04-27
10 v23.3.7.5-lts 2023-06-29
11 v23.3.6.7-lts 2023-06-28
12 v23.3.5.9-lts 2023-06-22
13 v23.3.4.17-lts 2023-06-17