Skip to content

Commit

Permalink
Merge branch 'master' into s390x_transform_fix
Browse files Browse the repository at this point in the history
  • Loading branch information
SuzyWangIBMer committed Jul 19, 2023
2 parents 1947fa5 + 15f07e9 commit 4c1fd26
Show file tree
Hide file tree
Showing 88 changed files with 1,469 additions and 862 deletions.
2 changes: 1 addition & 1 deletion docs/en/operations/backup.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should
```
:::note ALL
`ALL` is only applicable to the `RESTORE` command prior to version 23.4 of Clickhouse.
Prior to version 23.4 of ClickHouse, `ALL` was only applicable to the `RESTORE` command.
:::
## Background
Expand Down
33 changes: 33 additions & 0 deletions docs/en/sql-reference/functions/string-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1267,3 +1267,36 @@ Like [initcap](#initcap), assuming that the string contains valid UTF-8 encoded
Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I).

If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point.

## firstLine

Returns the first line from a multi-line string.

**Syntax**

```sql
firstLine(val)
```

**Arguments**

- `val` - Input value. [String](../data-types/string.md)

**Returned value**

- The first line of the input value or the whole value if there is no line
separators. [String](../data-types/string.md)

**Example**

```sql
select firstLine('foo\nbar\nbaz');
```

Result:

```result
┌─firstLine('foo\nbar\nbaz')─┐
│ foo │
└────────────────────────────┘
```
22 changes: 21 additions & 1 deletion docs/en/sql-reference/statements/select/with.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,27 @@ sidebar_label: WITH

# WITH Clause

ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)), that is provides to use results of `WITH` clause in the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression.
ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression.

Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.

An example of such behavior is below
``` sql
with cte_numbers as
(
select
num
from generateRandom('num UInt64', NULL)
limit 1000000
)
select
count()
from cte_numbers
where num in (select num from cte_numbers)
```
If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`

However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on...

## Syntax

Expand Down
2 changes: 1 addition & 1 deletion docs/en/sql-reference/table-functions/file.md
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ Multiple path components can have globs. For being processed file must exist and

- `*` — Substitutes any number of any characters except `/` including empty string.
- `?` — Substitutes any single character.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`.
- `{N..M}` — Substitutes any number in range from N to M including both borders.
- `**` - Fetches all files inside the folder recursively.

Expand Down
36 changes: 36 additions & 0 deletions docs/ru/sql-reference/functions/string-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1124,3 +1124,39 @@ Do Nothing for 2 Minutes 2:00  
Не учитывает язык. То есть, для турецкого языка, результат может быть не совсем верным.
Если длина UTF-8 последовательности байтов различна для верхнего и нижнего регистра кодовой точки, то для этой кодовой точки результат работы может быть некорректным.
Если строка содержит набор байтов, не являющийся UTF-8, то поведение не определено.

## firstLine

Возвращает первую строку в многострочном тексте.

**Синтаксис**

```sql
firstLine(val)
```

**Аргументы**

- `val` - текст для обработки. [String](../data-types/string.md)

**Returned value**

- Первая строка текста или весь текст, если переносы строк отсутствуют.

Тип: [String](../data-types/string.md)

**Пример**

Запрос:

```sql
select firstLine('foo\nbar\nbaz');
```

Результат:

```result
┌─firstLine('foo\nbar\nbaz')─┐
│ foo │
└────────────────────────────┘
```
2 changes: 1 addition & 1 deletion docs/ru/sql-reference/table-functions/file.md
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U

- `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов.
- `?` — заменяет ровно один любой символ.
- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`.
- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`.
- `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули).

Конструкция с `{}` аналогична табличной функции [remote](remote.md).
Expand Down
2 changes: 2 additions & 0 deletions programs/server/Server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -887,6 +887,7 @@ try
#endif

global_context->setRemoteHostFilter(config());
global_context->setHTTPHeaderFilter(config());

std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
fs::path path = path_str;
Expand Down Expand Up @@ -1200,6 +1201,7 @@ try
}

global_context->setRemoteHostFilter(*config);
global_context->setHTTPHeaderFilter(*config);

global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
Expand Down
8 changes: 8 additions & 0 deletions programs/server/config.xml
Original file line number Diff line number Diff line change
Expand Up @@ -866,6 +866,14 @@
-->
<!--</remote_url_allow_hosts>-->

<!-- The list of HTTP headers forbidden to use in HTTP-related storage engines and table functions.
If this section is not present in configuration, all headers are allowed.
-->
<!-- <http_forbid_headers>
<header>exact_header</header>
<header_regexp>(?i)(case_insensitive_header)</header_regexp>
</http_forbid_headers> -->

<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
Values for substitutions are specified in /clickhouse/name_of_substitution elements in that file.
Expand Down
196 changes: 193 additions & 3 deletions src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp
Original file line number Diff line number Diff line change
@@ -1,10 +1,25 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionGroupArrayMoving.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>

#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>

#include <Columns/ColumnVector.h>
#include <Columns/ColumnArray.h>

#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>

#include <type_traits>

#define AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE 0xFFFFFF


namespace DB
Expand All @@ -13,11 +28,186 @@ struct Settings;

namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}

template <typename T>
struct MovingData
{
/// For easy serialization.
static_assert(std::has_unique_object_representations_v<T> || std::is_floating_point_v<T>);

using Accumulator = T;

/// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
using Allocator = MixedAlignedArenaAllocator<alignof(T), 4096>;
using Array = PODArray<T, 32, Allocator>;

Array value; /// Prefix sums.
T sum{};

void NO_SANITIZE_UNDEFINED add(T val, Arena * arena)
{
sum += val;
value.push_back(sum, arena);
}
};

template <typename T>
struct MovingSumData : public MovingData<T>
{
static constexpr auto name = "groupArrayMovingSum";

T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const
{
if (idx < window_size)
return this->value[idx];
else
return this->value[idx] - this->value[idx - window_size];
}
};

template <typename T>
struct MovingAvgData : public MovingData<T>
{
static constexpr auto name = "groupArrayMovingAvg";

T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const
{
if (idx < window_size)
return this->value[idx] / T(window_size);
else
return (this->value[idx] - this->value[idx - window_size]) / T(window_size);
}
};


template <typename T, typename LimitNumElements, typename Data>
class MovingImpl final
: public IAggregateFunctionDataHelper<Data, MovingImpl<T, LimitNumElements, Data>>
{
static constexpr bool limit_num_elems = LimitNumElements::value;
UInt64 window_size;

public:
using ResultT = typename Data::Accumulator;

using ColumnSource = ColumnVectorOrDecimal<T>;

/// Probably for overflow function in the future.
using ColumnResult = ColumnVectorOrDecimal<ResultT>;

explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<Data, MovingImpl<T, LimitNumElements, Data>>({data_type_}, {}, createResultType(data_type_))
, window_size(window_size_) {}

String getName() const override { return Data::name; }

static DataTypePtr createResultType(const DataTypePtr & argument)
{
return std::make_shared<DataTypeArray>(getReturnTypeElement(argument));
}

void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto value = static_cast<const ColumnSource &>(*columns[0]).getData()[row_num];
this->data(place).add(static_cast<ResultT>(value), arena);
}

void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_elems = this->data(place);
auto & rhs_elems = this->data(rhs);

size_t cur_size = cur_elems.value.size();

if (rhs_elems.value.size())
cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena);

for (size_t i = cur_size; i < cur_elems.value.size(); ++i)
{
cur_elems.value[i] += cur_elems.sum;
}

cur_elems.sum += rhs_elems.sum;
}

void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(value.data()), size * sizeof(value[0]));
}

void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
size_t size = 0;
readVarUInt(size, buf);

if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE);

if (size > 0)
{
auto & value = this->data(place).value;
value.resize(size, arena);
buf.readStrict(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
this->data(place).sum = value.back();
}
}

void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
const auto & data = this->data(place);
size_t size = data.value.size();

ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();

offsets_to.push_back(offsets_to.back() + size);

if (size)
{
typename ColumnResult::Container & data_to = assert_cast<ColumnResult &>(arr_to.getData()).getData();

for (size_t i = 0; i < size; ++i)
{
if (!limit_num_elems)
{
data_to.push_back(data.get(i, size));
}
else
{
data_to.push_back(data.get(i, window_size));
}
}
}
}

bool allocatesMemoryInArena() const override
{
return true;
}

private:
static auto getReturnTypeElement(const DataTypePtr & argument)
{
if constexpr (!is_decimal<ResultT>)
return std::make_shared<DataTypeNumber<ResultT>>();
else
{
using Res = DataTypeDecimal<ResultT>;
return std::make_shared<Res>(Res::maxPrecision(), getDecimalScale(*argument));
}
}
};


namespace
{

Expand Down Expand Up @@ -79,7 +269,7 @@ AggregateFunctionPtr createAggregateFunctionMoving(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);

if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);

Expand Down
Loading

0 comments on commit 4c1fd26

Please sign in to comment.