Merge branch 'fix_one_more_tiny_race' into in_memory_raft

This commit is contained in:
alesapin 2021-01-22 15:56:43 +03:00
commit 4a06bf1d79
62 changed files with 634 additions and 1364 deletions

View File

@ -0,0 +1,19 @@
---
name: Sanitizer alert
about: Potential issue has been found by special code instrumentation
title: ''
labels: testing
assignees: ''
---
(you don't have to strictly follow this form)
**Describe the bug**
A link to the report
**How to reproduce**
Try to reproduce the report and copy the tables and queries involved.
**Error message and/or stacktrace**
You can find additional information in server logs.

View File

@ -15,9 +15,13 @@ currently being supported with security updates:
| 20.4 | :x: |
| 20.5 | :x: |
| 20.6 | :x: |
| 20.7 | :white_check_mark: |
| 20.7 | :x: |
| 20.8 | :white_check_mark: |
| 20.9 | :white_check_mark: |
| 20.9 | :x: |
| 20.10 | :x: |
| 20.11 | :white_check_mark: |
| 20.12 | :white_check_mark: |
| 21.1 | :white_check_mark: |
## Reporting a Vulnerability

View File

@ -780,7 +780,7 @@ public:
return lut[index].date + time_offset;
}
inline time_t addWeeks(time_t t, Int64 delta) const
inline NO_SANITIZE_UNDEFINED time_t addWeeks(time_t t, Int64 delta) const
{
return addDays(t, delta * 7);
}
@ -812,7 +812,7 @@ public:
return lut[result_day].date + time_offset;
}
inline DayNum addMonths(DayNum d, Int64 delta) const
inline NO_SANITIZE_UNDEFINED DayNum addMonths(DayNum d, Int64 delta) const
{
const Values & values = lut[d];
@ -836,12 +836,12 @@ public:
}
}
inline time_t addQuarters(time_t t, Int64 delta) const
inline NO_SANITIZE_UNDEFINED time_t addQuarters(time_t t, Int64 delta) const
{
return addMonths(t, delta * 3);
}
inline DayNum addQuarters(DayNum d, Int64 delta) const
inline NO_SANITIZE_UNDEFINED DayNum addQuarters(DayNum d, Int64 delta) const
{
return addMonths(d, delta * 3);
}

View File

@ -332,7 +332,7 @@ function run_tests
01622_defaults_for_url_engine
)
time clickhouse-test -j 8 --order=random --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
time clickhouse-test -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
# substr is to remove semicolon after test name
readarray -t FAILED_TESTS < <(awk '/\[ FAIL|TIMEOUT|ERROR \]/ { print substr($3, 1, length($3)-1) }' "$FASTTEST_OUTPUT/test_log.txt" | tee "$FASTTEST_OUTPUT/failed-parallel-tests.txt")

View File

@ -75,7 +75,7 @@ function fuzz
{
# Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests.
cd ch
NEW_TESTS=$(git diff --name-only master "$SHA_TO_TEST" | grep -P 'tests/queries/0_stateless/.*\.sql' | sed -r -e 's!^!ch/!' | sort -R)
NEW_TESTS=$(git diff --name-only "$(git merge-base origin/master "$SHA_TO_TEST"~)" "$SHA_TO_TEST" | grep -P 'tests/queries/0_stateless/.*\.sql' | sed -r -e 's!^!ch/!' | sort -R)
cd ..
if [[ -n "$NEW_TESTS" ]]
then

View File

@ -120,5 +120,6 @@ toc_title: Adopters
| <a href="https://htc-cs.ru/" class="favicon">ЦВТ</a> | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) |
| <a href="https://mkb.ru/" class="favicon">МКБ</a> | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) |
| <a href="https://cft.ru/" class="favicon">ЦФТ</a> | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) |
| <a href="https://www.kakaocorp.com/" class="favicon">kakaocorp</a> | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) |
[Original article](https://clickhouse.tech/docs/en/introduction/adopters/) <!--hide-->

View File

@ -0,0 +1,24 @@
---
toc_priority: 65
toc_title: Caches
---
# Cache Types {#cache-types}
When performing queries, ClichHouse uses different caches.
Main cache types:
- `mark_cache` — Cache of marks used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family.
- `uncompressed_cache` — Cache of uncompressed data used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family.
Additional cache types:
- DNS cache
- [regexp](../interfaces/formats.md#data-format-regexp) cache
- compiled expressions cache
- [Avro format](../interfaces/formats.md#data-format-avro) schemas cache
- [dictionaries data cache](../sql-reference/dictionaries/index.md)
Indirectly used:
- OS page cache
[Original article](https://clickhouse.tech/docs/en/operations/caches/) <!--hide-->

View File

@ -2506,11 +2506,7 @@ Default value: 0.
Consider the following query with aggregate functions:
```sql
SELECT
SUM(-1),
MAX(0)
FROM system.one
WHERE 0
SELECT SUM(-1), MAX(0) FROM system.one WHERE 0;
```
With `aggregate_functions_null_for_empty = 0` it would produce:

View File

@ -0,0 +1,29 @@
---
toc_priority: 61
toc_title: Multiword Type Names
---
# Multiword Types {#multiword-types}
When creating tables, you can use data types with a name consisting of several words. This is implemented for better SQL compatibility.
## Multiword Types Support {#multiword-types-support}
| Multiword types | Simple types |
|----------------------------------|--------------------------------------------------------------|
| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) |
| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) |
| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| BINARY VARYING | [String](../../sql-reference/data-types/string.md) |
[Original article](https://clickhouse.tech/docs/en/sql-reference/data-types/multiword-types/) <!--hide-->

View File

@ -23,7 +23,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
```
Creates a table named `name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine.
The structure of the table is a list of column descriptions, secondary indexes and constraints . If primary key is supported by the engine, it will be indicated as parameter for the table engine.
The structure of the table is a list of column descriptions, secondary indexes and constraints . If [primary key](#primary-key) is supported by the engine, it will be indicated as parameter for the table engine.
A column description is `name type` in the simplest case. Example: `RegionID UInt32`.
@ -111,7 +111,7 @@ It is not possible to set default values for elements in nested data structures.
You can define a [primary key](../../../engines/table-engines/mergetree-family/mergetree.md#primary-keys-and-indexes-in-queries) when creating a table. Primary key can be specified in two ways:
- inside the column list
- Inside the column list
``` sql
CREATE TABLE db.table_name
@ -122,7 +122,7 @@ CREATE TABLE db.table_name
ENGINE = engine;
```
- outside the column list
- Outside the column list
``` sql
CREATE TABLE db.table_name
@ -133,7 +133,8 @@ ENGINE = engine
PRIMARY KEY(expr1[, expr2,...]);
```
You can't combine both ways in one query.
!!! warning "Warning"
You can't combine both ways in one query.
## Constraints {#constraints}

View File

@ -153,7 +153,7 @@ CREATE TABLE table_name
```xml
<default_replica_path>/clickhouse/tables/{shard}/{database}/{table}</default_replica_path>
<default_replica_name>{replica}</default_replica_path>
<default_replica_name>{replica}</default_replica_name>
```
В этом случае можно опустить аргументы при создании таблиц:

View File

@ -2375,11 +2375,7 @@ SELECT number FROM numbers(3) FORMAT JSONEachRow;
Рассмотрим запрос с агрегирующими функциями:
```sql
SELECT
SUM(-1),
MAX(0)
FROM system.one
WHERE 0
SELECT SUM(-1), MAX(0) FROM system.one WHERE 0;
```
Результат запроса с настройкой `aggregate_functions_null_for_empty = 0`:

View File

@ -0,0 +1,29 @@
---
toc_priority: 61
toc_title: Составные типы
---
# Составные типы {#multiword-types}
При создании таблиц вы можете использовать типы данных с названием, состоящим из нескольких слов. Такие названия поддерживаются для лучшей совместимости с SQL.
## Поддержка составных типов {#multiword-types-support}
| Составные типы | Обычные типы |
|-------------------------------------|-----------------------------------------------------------|
| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) |
| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) |
| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| BINARY VARYING | [String](../../sql-reference/data-types/string.md) |
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/data-types/multiword-types/) <!--hide-->

View File

@ -22,6 +22,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Описание столбца, это `name type`, в простейшем случае. Пример: `RegionID UInt32`.
Также могут быть указаны выражения для значений по умолчанию - смотрите ниже.
При необходимости можно указать [первичный ключ](#primary-key) с одним или несколькими ключевыми выражениями.
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine]
```
@ -88,6 +89,35 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ...
Отсутствует возможность задать значения по умолчанию для элементов вложенных структур данных.
## Первичный ключ {#primary-key}
Вы можете определить [первичный ключ](../../../engines/table-engines/mergetree-family/mergetree.md#primary-keys-and-indexes-in-queries) при создании таблицы. Первичный ключ может быть указан двумя способами:
- в списке столбцов:
``` sql
CREATE TABLE db.table_name
(
name1 type1, name2 type2, ...,
PRIMARY KEY(expr1[, expr2,...])]
)
ENGINE = engine;
```
- вне списка столбцов:
``` sql
CREATE TABLE db.table_name
(
name1 type1, name2 type2, ...
)
ENGINE = engine
PRIMARY KEY(expr1[, expr2,...]);
```
!!! warning "Предупреждение"
Вы не можете сочетать оба способа в одном запросе.
### Ограничения (constraints) {#constraints}
Наряду с объявлением столбцов можно объявить ограничения на значения в столбцах таблицы:

View File

@ -803,6 +803,9 @@ Packet Connection::receivePacket(std::function<void(Poco::Net::Socket &)> async_
}
catch (Exception & e)
{
/// This is to consider ATTEMPT_TO_READ_AFTER_EOF as a remote exception.
e.setRemoteException();
/// Add server address to exception message, if need.
if (e.code() != ErrorCodes::UNKNOWN_PACKET_FROM_SERVER)
e.addMessage("while receiving packet from " + getDescription());
@ -892,7 +895,7 @@ void Connection::setDescription()
std::unique_ptr<Exception> Connection::receiveException()
{
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription()));
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription(), true /* remote */));
}

View File

@ -50,8 +50,9 @@ void handle_error_code([[maybe_unused]] const std::string & msg, int code)
ErrorCodes::increment(code);
}
Exception::Exception(const std::string & msg, int code)
Exception::Exception(const std::string & msg, int code, bool remote_)
: Poco::Exception(msg, code)
, remote(remote_)
{
handle_error_code(msg, code);
}

View File

@ -25,7 +25,7 @@ class Exception : public Poco::Exception
{
public:
Exception() = default;
Exception(const std::string & msg, int code);
Exception(const std::string & msg, int code, bool remote_ = false);
Exception(const std::string & msg, const Exception & nested, int code);
Exception(int code, const std::string & message)
@ -61,12 +61,17 @@ public:
extendedMessage(message);
}
/// Used to distinguish local exceptions from the one that was received from remote node.
void setRemoteException(bool remote_ = true) { remote = remote_; }
bool isRemoteException() const { return remote; }
std::string getStackTraceString() const;
private:
#ifndef STD_EXCEPTION_HAS_STACK_TRACE
StackTrace trace;
#endif
bool remote = false;
const char * className() const throw() override { return "DB::Exception"; }
};

File diff suppressed because it is too large Load Diff

View File

@ -51,7 +51,13 @@ public:
bool isParametric() const override { return false; }
bool haveSubtypes() const override { return false; }
bool shouldAlignRightInPrettyFormats() const override { return true; }
bool shouldAlignRightInPrettyFormats() const override
{
/// Just a number, without customizations. Counterexample: IPv4.
return !custom_text_serialization;
}
bool textCanContainOnlyValidUTF8() const override { return true; }
bool isComparable() const override { return true; }
bool isValueRepresentedByNumber() const override { return true; }

View File

@ -497,7 +497,7 @@ public:
/// For all other substreams (like ArraySizes, NullMasks, etc.) we use only
/// generic compression codecs like LZ4.
static bool isSpecialCompressionAllowed(const SubstreamPath & path);
private:
protected:
friend class DataTypeFactory;
friend class AggregateFunctionSimpleState;
/// Customize this DataType

View File

@ -175,9 +175,9 @@ void DiskDecorator::truncateFile(const String & path, size_t size)
delegate->truncateFile(path, size);
}
int DiskDecorator::open(const String & path, mode_t mode) const
int DiskDecorator::open(const String & path, int flags) const
{
return delegate->open(path, mode);
return delegate->open(path, flags);
}
void DiskDecorator::close(int fd) const

View File

@ -48,7 +48,7 @@ public:
void setReadOnly(const String & path) override;
void createHardLink(const String & src_path, const String & dst_path) override;
void truncateFile(const String & path, size_t size) override;
int open(const String & path, mode_t mode) const override;
int open(const String & path, int flags) const override;
void close(int fd) const override;
void sync(int fd) const override;
const String getType() const override { return delegate->getType(); }

View File

@ -315,10 +315,10 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr<IDisk> & to
IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers.
}
int DiskLocal::open(const String & path, mode_t mode) const
int DiskLocal::open(const String & path, int flags) const
{
String full_path = disk_path + path;
int fd = ::open(full_path.c_str(), mode);
int fd = ::open(full_path.c_str(), flags);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + full_path, full_path,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);

View File

@ -98,7 +98,7 @@ public:
void createHardLink(const String & src_path, const String & dst_path) override;
int open(const String & path, mode_t mode) const override;
int open(const String & path, int flags) const override;
void close(int fd) const override;
void sync(int fd) const override;

View File

@ -436,7 +436,7 @@ void DiskMemory::setReadOnly(const String &)
throw Exception("Method setReadOnly is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED);
}
int DiskMemory::open(const String & /*path*/, mode_t /*mode*/) const
int DiskMemory::open(const String & /*path*/, int /*flags*/) const
{
throw Exception("Method open is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -89,7 +89,7 @@ public:
void createHardLink(const String & src_path, const String & dst_path) override;
int open(const String & path, mode_t mode) const override;
int open(const String & path, int flags) const override;
void close(int fd) const override;
void sync(int fd) const override;

View File

@ -175,7 +175,7 @@ public:
virtual void createHardLink(const String & src_path, const String & dst_path) = 0;
/// Wrapper for POSIX open
virtual int open(const String & path, mode_t mode) const = 0;
virtual int open(const String & path, int flags) const = 0;
/// Wrapper for POSIX close
virtual void close(int fd) const = 0;

View File

@ -878,7 +878,7 @@ void DiskS3::setReadOnly(const String & path)
metadata.save();
}
int DiskS3::open(const String & /*path*/, mode_t /*mode*/) const
int DiskS3::open(const String & /*path*/, int /*flags*/) const
{
throw Exception("Method open is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -105,7 +105,7 @@ public:
void setReadOnly(const String & path) override;
int open(const String & path, mode_t mode) const override;
int open(const String & path, int flags) const override;
void close(int fd) const override;
void sync(int fd) const override;

View File

@ -224,8 +224,9 @@ struct SubtractIntervalImpl : public Transform
using Transform::Transform;
template <typename T>
inline auto execute(T t, Int64 delta, const DateLUTImpl & time_zone) const
inline NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone) const
{
/// Signed integer overflow is Ok.
return Transform::execute(t, -delta, time_zone);
}
};

View File

@ -342,7 +342,7 @@ void NO_INLINE sliceDynamicOffsetUnbounded(Source && src, Sink && sink, const IC
if (offset > 0)
slice = src.getSliceFromLeft(offset - 1);
else
slice = src.getSliceFromRight(-offset);
slice = src.getSliceFromRight(-UInt64(offset));
writeSlice(slice, sink);
}
@ -374,7 +374,7 @@ void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const ICol
Int64 size = has_length ? length_nested_column->getInt(row_num) : static_cast<Int64>(src.getElementSize());
if (size < 0)
size += offset > 0 ? static_cast<Int64>(src.getElementSize()) - (offset - 1) : -offset;
size += offset > 0 ? static_cast<Int64>(src.getElementSize()) - (offset - 1) : -UInt64(offset);
if (offset != 0 && size > 0)
{
@ -383,7 +383,7 @@ void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const ICol
if (offset > 0)
slice = src.getSliceFromLeft(offset - 1, size);
else
slice = src.getSliceFromRight(-offset, size);
slice = src.getSliceFromRight(-UInt64(offset), size);
writeSlice(slice, sink);
}

View File

@ -5,6 +5,7 @@
#include <Columns/ColumnDecimal.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
#include <common/defines.h>
namespace DB
@ -121,7 +122,7 @@ struct ArrayAggregateImpl
}
template <typename Element>
static bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
static NO_SANITIZE_UNDEFINED bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
{
using Result = ArrayAggregateResult<Element, aggregate_operation>;
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;

View File

@ -231,7 +231,7 @@ struct ArrayElementNumImpl
if (builder)
builder.update(j);
}
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
else if (index < 0 && -static_cast<size_t>(index) <= array_size)
{
size_t j = offsets[i] + index;
result[i] = data[j];
@ -329,7 +329,7 @@ struct ArrayElementStringImpl
TIndex index = indices[i];
if (index > 0 && static_cast<size_t>(index) <= array_size)
adjusted_index = index - 1;
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
else if (index < 0 && -static_cast<size_t>(index) <= array_size)
adjusted_index = array_size + index;
else
adjusted_index = array_size; /// means no element should be taken
@ -427,7 +427,7 @@ struct ArrayElementGenericImpl
if (builder)
builder.update(j);
}
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
else if (index < 0 && -static_cast<size_t>(index) <= array_size)
{
size_t j = offsets[i] + index;
result.insertFrom(data, j);
@ -472,11 +472,24 @@ ColumnPtr FunctionArrayElement::executeNumberConst(
auto col_res = ColumnVector<DataType>::create();
if (index.getType() == Field::Types::UInt64)
{
ArrayElementNumImpl<DataType>::template vectorConst<false>(
col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData(), builder);
}
else if (index.getType() == Field::Types::Int64)
{
/// Cast to UInt64 before negation allows to avoid undefined behaviour for negation of the most negative number.
/// NOTE: this would be undefined behaviour in C++ sense, but nevertheless, compiler cannot see it on user provided data,
/// and generates the code that we want on supported CPU architectures (overflow in sense of two's complement arithmetic).
/// This is only needed to avoid UBSan report.
/// Negative array indices work this way:
/// arr[-1] is the element at offset 0 from the last
/// arr[-2] is the element at offset 1 from the last and so on.
ArrayElementNumImpl<DataType>::template vectorConst<true>(
col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData(), builder);
col_nested->getData(), col_array->getOffsets(), -(UInt64(safeGet<Int64>(index)) + 1), col_res->getData(), builder);
}
else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
@ -534,7 +547,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument
col_nested->getChars(),
col_array->getOffsets(),
col_nested->getOffsets(),
-safeGet<Int64>(index) - 1,
-(UInt64(safeGet<Int64>(index)) + 1),
col_res->getChars(),
col_res->getOffsets(),
builder);
@ -588,7 +601,7 @@ ColumnPtr FunctionArrayElement::executeGenericConst(
col_nested, col_array->getOffsets(), safeGet<UInt64>(index) - 1, *col_res, builder);
else if (index.getType() == Field::Types::Int64)
ArrayElementGenericImpl::vectorConst<true>(
col_nested, col_array->getOffsets(), -safeGet<Int64>(index) - 1, *col_res, builder);
col_nested, col_array->getOffsets(), -(UInt64(safeGet<Int64>(index) + 1)), *col_res, builder);
else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
@ -639,7 +652,7 @@ ColumnPtr FunctionArrayElement::executeConst(const ColumnsWithTypeAndName & argu
if (builder)
builder.update(j);
}
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
else if (index < 0 && -static_cast<size_t>(index) <= array_size)
{
size_t j = array_size + index;
res->insertFrom(array_elements, j);

View File

@ -1,20 +1,28 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/castColumn.h>
#include <memory>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include "array/arrayIndex.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
@ -57,7 +65,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() % 2 != 0)
throw Exception("Function " + getName() + " even number of arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception("Function " + getName() + " even number of arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypes keys, values;
for (size_t i = 0; i < arguments.size(); i += 2)
@ -130,11 +138,167 @@ public:
}
};
struct NameMapContains { static constexpr auto name = "mapContains"; };
class FunctionMapContains : public IFunction
{
public:
static constexpr auto name = NameMapContains::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMapContains>(); }
String getName() const override
{
return NameMapContains::name;
}
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
if (!map_type)
throw Exception{"First argument for function " + getName() + " must be a map",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto key_type = map_type->getKeyType();
if (!(isNumber(arguments[1].type) && isNumber(key_type))
&& key_type->getName() != arguments[1].type->getName())
throw Exception{"Second argument for function " + getName() + " must be a " + key_type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
if (!col_map)
return nullptr;
const auto & nested_column = col_map->getNestedColumn();
const auto & keys_data = col_map->getNestedData().getColumn(0);
/// Prepare arguments to call arrayIndex for check has the array element.
ColumnsWithTypeAndName new_arguments =
{
{
ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr()),
std::make_shared<DataTypeArray>(result_type),
""
},
arguments[1]
};
return FunctionArrayIndex<HasAction, NameMapContains>().executeImpl(new_arguments, result_type, input_rows_count);
}
};
class FunctionMapKeys : public IFunction
{
public:
static constexpr auto name = "mapKeys";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMapKeys>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
if (!map_type)
throw Exception{"First argument for function " + getName() + " must be a map",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto key_type = map_type->getKeyType();
return std::make_shared<DataTypeArray>(key_type);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
{
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
if (!col_map)
return nullptr;
const auto & nested_column = col_map->getNestedColumn();
const auto & keys_data = col_map->getNestedData().getColumn(0);
return ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr());
}
};
class FunctionMapValues : public IFunction
{
public:
static constexpr auto name = "mapValues";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMapValues>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
if (!map_type)
throw Exception{"First argument for function " + getName() + " must be a map",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto value_type = map_type->getValueType();
return std::make_shared<DataTypeArray>(value_type);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
{
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
if (!col_map)
return nullptr;
const auto & nested_column = col_map->getNestedColumn();
const auto & values_data = col_map->getNestedData().getColumn(1);
return ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr());
}
};
}
void registerFunctionsMap(FunctionFactory & factory)
{
factory.registerFunction<FunctionMap>();
factory.registerFunction<FunctionMapContains>();
factory.registerFunction<FunctionMapKeys>();
factory.registerFunction<FunctionMapValues>();
}
}

View File

@ -78,8 +78,8 @@ public:
Float64 seconds = applyVisitor(FieldVisitorConvertToNumber<Float64>(), assert_cast<const ColumnConst &>(*col).getField());
if (seconds < 0)
throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS);
if (seconds < 0 || !std::isfinite(seconds))
throw Exception("Cannot sleep infinite or negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS);
size_t size = col->size();

View File

@ -1014,7 +1014,7 @@ void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field)
}
Exception readException(ReadBuffer & buf, const String & additional_message)
Exception readException(ReadBuffer & buf, const String & additional_message, bool remote_exception)
{
int code = 0;
String name;
@ -1041,7 +1041,7 @@ Exception readException(ReadBuffer & buf, const String & additional_message)
if (!stack_trace.empty())
out << " Stack trace:\n\n" << stack_trace;
return Exception(out.str(), code);
return Exception(out.str(), code, remote_exception);
}
void readAndThrowException(ReadBuffer & buf, const String & additional_message)

View File

@ -1073,7 +1073,7 @@ void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field);
* (type is cut to base class, 'message' replaced by 'displayText', and stack trace is appended to 'message')
* Some additional message could be appended to exception (example: you could add information about from where it was received).
*/
Exception readException(ReadBuffer & buf, const String & additional_message = "");
Exception readException(ReadBuffer & buf, const String & additional_message = "", bool remote_exception = false);
void readAndThrowException(ReadBuffer & buf, const String & additional_message = "");

View File

@ -946,7 +946,7 @@ bool Context::hasScalar(const String & name) const
void Context::addQueryAccessInfo(const String & quoted_database_name, const String & full_quoted_table_name, const Names & column_names)
{
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
auto lock = getLock();
std::lock_guard<std::mutex> lock(query_access_info.mutex);
query_access_info.databases.emplace(quoted_database_name);
query_access_info.tables.emplace(full_quoted_table_name);
for (const auto & column_name : column_names)

View File

@ -194,9 +194,36 @@ private:
/// Record entities accessed by current query, and store this information in system.query_log.
struct QueryAccessInfo
{
std::set<std::string> databases;
std::set<std::string> tables;
std::set<std::string> columns;
QueryAccessInfo() = default;
QueryAccessInfo(const QueryAccessInfo & rhs)
{
std::lock_guard<std::mutex> lock(rhs.mutex);
databases = rhs.databases;
tables = rhs.tables;
columns = rhs.columns;
}
QueryAccessInfo(QueryAccessInfo && rhs) = delete;
QueryAccessInfo & operator=(QueryAccessInfo rhs)
{
swap(rhs);
return *this;
}
void swap(QueryAccessInfo & rhs)
{
std::swap(databases, rhs.databases);
std::swap(tables, rhs.tables);
std::swap(columns, rhs.columns);
}
/// To prevent a race between copy-constructor and other uses of this structure.
mutable std::mutex mutex{};
std::set<std::string> databases{};
std::set<std::string> tables{};
std::set<std::string> columns{};
};
QueryAccessInfo query_access_info;

View File

@ -28,14 +28,15 @@ namespace DB
namespace ErrorCodes
{
extern const int SYSTEM_ERROR;
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
}
struct PollResult
{
bool has_responses;
bool has_requests;
bool error;
size_t ready_responses_count{0};
bool has_requests{false};
bool error{false};
};
/// Queue with mutex. As simple as possible.
@ -191,10 +192,17 @@ struct SocketInterruptablePollWrapper
result.has_requests = true;
else
{
/// Skip all of them, we are not interested in exact
/// amount because responses ordered in responses queue.
response_in.ignore();
result.has_responses = true;
UInt8 dummy;
do
{
/// All ready responses stored in responses queue,
/// but we have to count amount of ready responses in pipe
/// and process them only. Otherwise states of response_in
/// and response queue will be inconsistent and race condition is possible.
readIntBinary(dummy, response_in);
result.ready_responses_count++;
}
while (response_in.available());
}
}
}
@ -349,23 +357,27 @@ void TestKeeperTCPHandler::runImpl()
while (in->available());
}
if (result.has_responses)
/// Process exact amount of responses from pipe
/// otherwise state of responses queue and signaling pipe
/// became inconsistent and race condition is possible.
while (result.ready_responses_count != 0)
{
Coordination::ZooKeeperResponsePtr response;
while (responses->tryPop(response))
{
if (response->xid == close_xid)
{
LOG_DEBUG(log, "Session #{} successfully closed", session_id);
return;
}
if (!responses->tryPop(response))
throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have at least {} ready responses, but queue is empty. It's a bug.", result.ready_responses_count);
if (response->error == Coordination::Error::ZOK)
response->write(*out);
else if (response->xid != Coordination::WATCH_XID)
response->write(*out);
/// skipping bad response for watch
if (response->xid == close_xid)
{
LOG_DEBUG(log, "Session #{} successfully closed", session_id);
return;
}
if (response->error == Coordination::Error::ZOK)
response->write(*out);
else if (response->xid != Coordination::WATCH_XID)
response->write(*out);
/// skipping bad response for watch
result.ready_responses_count--;
}
if (result.error)

View File

@ -155,6 +155,27 @@ namespace
return header;
}
/// remote_error argument is used to decide whether some errors should be
/// ignored or not, in particular:
///
/// - ATTEMPT_TO_READ_AFTER_EOF should not be ignored
/// if we receive it from remote (receiver), since:
/// - the sender will got ATTEMPT_TO_READ_AFTER_EOF when the client just go away,
/// i.e. server had been restarted
/// - since #18853 the file will be checked on the sender locally, and
/// if there is something wrong with the file itself, we will receive
/// ATTEMPT_TO_READ_AFTER_EOF not from the remote at first
/// and mark batch as broken.
bool isFileBrokenErrorCode(int code, bool remote_error)
{
return code == ErrorCodes::CHECKSUM_DOESNT_MATCH
|| code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|| code == ErrorCodes::UNKNOWN_CODEC
|| code == ErrorCodes::CANNOT_DECOMPRESS
|| (!remote_error && code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}
}
@ -571,7 +592,7 @@ struct StorageDistributedDirectoryMonitor::Batch
}
catch (const Exception & e)
{
if (isFileBrokenErrorCode(e.code()))
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
tryLogCurrentException(parent.log, "Failed to send batch due to");
batch_broken = true;
@ -801,16 +822,6 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
}
}
bool StorageDistributedDirectoryMonitor::isFileBrokenErrorCode(int code)
{
return code == ErrorCodes::CHECKSUM_DOESNT_MATCH
|| code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|| code == ErrorCodes::UNKNOWN_CODEC
|| code == ErrorCodes::CANNOT_DECOMPRESS
|| code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF;
}
void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_path) const
{
const auto last_path_separator_pos = file_path.rfind('/');
@ -837,7 +848,7 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p
bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & file_path, const Exception & e) const
{
/// mark file as broken if necessary
if (isFileBrokenErrorCode(e.code()))
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
markAsBroken(file_path);
return true;

View File

@ -70,7 +70,6 @@ private:
void processFile(const std::string & file_path);
void processFilesWithBatching(const std::map<UInt64, std::string> & files);
static bool isFileBrokenErrorCode(int code);
void markAsBroken(const std::string & file_path) const;
bool maybeMarkAsBroken(const std::string & file_path, const Exception & e) const;

View File

@ -40,6 +40,18 @@ MergeListElement::MergeListElement(const std::string & database_, const std::str
background_thread_memory_tracker = CurrentThread::getMemoryTracker();
if (background_thread_memory_tracker)
{
/// From the query context it will be ("for thread") memory tracker with VariableContext::Thread level,
/// which does not have any limits and sampling settings configured.
/// And parent for this memory tracker should be ("(for query)") with VariableContext::Process level,
/// that has limits and sampling configured.
MemoryTracker * parent;
if (background_thread_memory_tracker->level == VariableContext::Thread &&
(parent = background_thread_memory_tracker->getParent()) &&
parent != &total_memory_tracker)
{
background_thread_memory_tracker = parent;
}
background_thread_memory_tracker_prev_parent = background_thread_memory_tracker->getParent();
background_thread_memory_tracker->setParent(&memory_tracker);
}

View File

@ -182,27 +182,30 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc
index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty();
}
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
/// Write index. The index contains Primary Key value for each `index_granularity` row.
for (const auto & granule : granules_to_write)
{
if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start)
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
/// Write index. The index contains Primary Key value for each `index_granularity` row.
for (const auto & granule : granules_to_write)
{
for (size_t j = 0; j < primary_columns_num; ++j)
if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start)
{
const auto & primary_column = primary_index_block.getByPosition(j);
index_columns[j]->insertFrom(*primary_column.column, granule.start_row);
primary_column.type->serializeBinary(*primary_column.column, granule.start_row, *index_stream);
for (size_t j = 0; j < primary_columns_num; ++j)
{
const auto & primary_column = primary_index_block.getByPosition(j);
index_columns[j]->insertFrom(*primary_column.column, granule.start_row);
primary_column.type->serializeBinary(*primary_column.column, granule.start_row, *index_stream);
}
}
}
}
/// store last index row to write final mark at the end of column
for (size_t j = 0; j < primary_columns_num; ++j)
last_block_index_columns[j] = primary_index_block.getByPosition(j).column;

View File

@ -47,12 +47,18 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args
for (size_t i = 1; i < args.size(); ++i)
{
const auto & [value_field, value_type_ptr] = evaluateConstantExpression(args[i], context);
const DataTypes & value_types_tuple = typeid_cast<const DataTypeTuple *>(value_type_ptr.get())->getElements();
const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(value_type_ptr.get());
if (!type_tuple)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function VALUES requires all but first argument (rows specification) to be either tuples or single values");
const Tuple & value_tuple = value_field.safeGet<Tuple>();
if (value_tuple.size() != sample_block.columns())
throw Exception("Values size should match with number of columns", ErrorCodes::BAD_ARGUMENTS);
const DataTypes & value_types_tuple = type_tuple->getElements();
for (size_t j = 0; j < value_tuple.size(); ++j)
{
Field value = convertFieldToTypeOrThrow(value_tuple[j], *sample_block.getByPosition(j).type, value_types_tuple[j].get());
@ -64,8 +70,6 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args
void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/)
{
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)

View File

@ -0,0 +1,21 @@
drop table if exists data_01641;
create table data_01641 (key Int, value String) engine=MergeTree order by (key, repeat(value, 10)) settings old_parts_lifetime=0, min_bytes_for_wide_part=0;
-- peak memory usage is 170MiB
set max_memory_usage='200Mi';
system stop merges data_01641;
insert into data_01641 select number, toString(number) from numbers(toUInt64(120e6));
-- peak:
-- - is 21MiB if background merges already scheduled
-- - is ~60MiB otherwise
set max_memory_usage='80Mi';
system start merges data_01641;
optimize table data_01641 final;
-- definitely should fail
set max_memory_usage='1Mi';
optimize table data_01641 final; -- { serverError 241 }
drop table data_01641;

View File

@ -0,0 +1,19 @@
1
1
0
1
0
0
1
0
['name','age']
['name','gender']
1 0 0
1 0 1
1 0 0
[232]
[233]
[234]
[1000]
[1001]
[1002]

View File

@ -0,0 +1,22 @@
set allow_experimental_map_type = 1;
-- String type
drop table if exists table_map;
create table table_map (a Map(String, String), b String) engine = Memory;
insert into table_map values ({'name':'zhangsan', 'age':'10'}, 'name'), ({'name':'lisi', 'gender':'female'},'age');
select mapContains(a, 'name') from table_map;
select mapContains(a, 'gender') from table_map;
select mapContains(a, 'abc') from table_map;
select mapContains(a, b) from table_map;
select mapContains(a, 10) from table_map; -- { serverError 43 }
select mapKeys(a) from table_map;
drop table if exists table_map;
CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32) engine = MergeTree order by tuple();
insert into table_map select map(number, number), number, number from numbers(1000, 3);
select mapContains(a, b), mapContains(a, c), mapContains(a, 233) from table_map;
select mapContains(a, 'aaa') from table_map; -- { serverError 43 }
select mapContains(b, 'aaa') from table_map; -- { serverError 43 }
select mapKeys(a) from table_map;
select mapValues(a) from table_map;
drop table if exists table_map;

View File

@ -0,0 +1,2 @@
SELECT sleep(nan); -- { serverError 36 }
SELECT sleep(inf); -- { serverError 36 }

View File

@ -0,0 +1,4 @@
┌─x───────────────┬─y───────────────┬──────────z─┐
│ 1.1.1.1 │ 1.1.1.1 │ 16843009 │
│ 255.255.255.255 │ 255.255.255.255 │ 4294967295 │
└─────────────────┴─────────────────┴────────────┘

View File

@ -0,0 +1 @@
SELECT arrayJoin(['1.1.1.1', '255.255.255.255']) AS x, toIPv4(x) AS y, toUInt32(y) AS z FORMAT PrettyCompactNoEscapes;

View File

@ -0,0 +1,26 @@
0
0
0
0
---
0
0
0
---
0
0
0
0
0
0
0
1
---
0
0
0
0
0
0
0
1

View File

@ -0,0 +1,19 @@
SELECT [number][10000000000] FROM numbers(1);
SELECT [number][-10000000000] FROM numbers(1);
SELECT [number][-0x8000000000000000] FROM numbers(1);
SELECT [number][0xFFFFFFFFFFFFFFFF] FROM numbers(1);
SELECT '---';
SELECT [materialize(1)][0xFFFFFFFFFFFFFFFF];
SELECT [materialize(1)][materialize(18446744073709551615)];
SELECT [materialize(1)][-0x8000000000000000];
SELECT '---';
SELECT [number][arrayJoin([-0x8000000000000000, -10000000000, 0, -1])] FROM numbers(2);
SELECT '---';
SELECT [number][arrayJoin([0xFFFFFFFFFFFFFFFF, 10000000000, 0, 1])] FROM numbers(2);

View File

@ -0,0 +1,10 @@
/** NOTE: The behaviour of substring and substringUTF8 is inconsistent when negative offset is greater than string size:
* substring:
* hello
* ^-----^ - offset -10, length 7, result: "he"
* substringUTF8:
* hello
* ^-----^ - offset -10, length 7, result: "hello"
* This may be subject for change.
*/
SELECT substringUTF8('hello, пÑ<C2BF>ивеÑ<C2B5>', -9223372036854775808, number) FROM numbers(16) FORMAT Null;

View File

@ -0,0 +1 @@
SELECT * FROM VALUES('x UInt8, y UInt16', 1 + 2, 'Hello'); -- { serverError 36 }

View File

@ -0,0 +1 @@
446744073709551616

View File

@ -0,0 +1 @@
SELECT arraySum([-9000000000000000000, -9000000000000000000]);

View File

@ -16,7 +16,8 @@
"01474_executable_dictionary", /// informational stderr from sanitizer at start
"functions_bad_arguments", /// Too long for TSan
"01603_read_with_backoff_bug", /// Too long for TSan
"01646_system_restart_replicas_smoke" /// RESTART REPLICAS can acquire too much locks, while only 64 is possible from one thread under TSan
"01646_system_restart_replicas_smoke", /// RESTART REPLICAS can acquire too much locks, while only 64 is possible from one thread under TSan
"01641_memory_tracking_insert_optimize" /// INSERT lots of rows is too heavy for TSan
],
"address-sanitizer": [
"00877",
@ -62,7 +63,8 @@
"hyperscan",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01396_inactive_replica_cleanup_nodes"
"01396_inactive_replica_cleanup_nodes",
"01641_memory_tracking_insert_optimize" /// INSERT lots of rows is too heavy in debug build
],
"unbundled-build": [
"00429",