Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into HDFS_PowerPC

This commit is contained in:
MeenaRenganathan22 2023-01-16 08:22:37 -05:00
commit 456ab0f44e
96 changed files with 2023 additions and 628 deletions

View File

@ -683,3 +683,4 @@ jobs:
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py
python3 merge_pr.py

View File

@ -169,3 +169,4 @@ jobs:
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py
python3 merge_pr.py --check-approved

View File

@ -4388,3 +4388,4 @@ jobs:
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py
python3 merge_pr.py --check-approved

View File

@ -5,6 +5,7 @@ set -x
# core.COMM.PID-TID
sysctl kernel.core_pattern='core.%e.%p-%P'
dmesg --clear ||:
set -e
set -u
@ -368,6 +369,7 @@ if [ -f core.zst ]; then
fi
rg --text -F '<Fatal>' server.log > fatal.log ||:
dmesg -T > dmesg.log ||:
zstd --threads=0 server.log
@ -396,6 +398,7 @@ p.links a { padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-s
<a href="fuzzer.log">fuzzer.log</a>
<a href="server.log.zst">server.log.zst</a>
<a href="main.log">main.log</a>
<a href="dmesg.log">dmesg.log</a>
${CORE_LINK}
</p>
<table>

View File

@ -136,3 +136,7 @@ DESCRIBE TABLE test_database.test_table;
│ data │ Nullable(String) │
└────────┴───────────────────┘
```
## Related content
- Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres)

View File

@ -175,3 +175,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32)
- [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md)
- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql)
## Related content
- Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres)

View File

@ -120,5 +120,6 @@ Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec.
## Related Content
- [Extracting, converting, and querying data in local files using clickhouse-local](https://clickhouse.com/blog/extracting-converting-querying-local-files-with-sql-clickhouse-local)
- [Getting Data Into ClickHouse - Part 1](https://clickhouse.com/blog/getting-data-into-clickhouse-part-1)
- [Exploring massive, real-world data sets: 100+ Years of Weather Records in ClickHouse](https://clickhouse.com/blog/real-world-data-noaa-climate-data)

View File

@ -57,6 +57,7 @@ ClickHouse-specific aggregate functions:
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md)
- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md)
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md)
- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md)
- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md)
- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md)
- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md)
@ -79,4 +80,4 @@ ClickHouse-specific aggregate functions:
- [cramersVBiasCorrected](./cramersvbiascorrected.md)
- [theilsU](./theilsu.md)
- [maxIntersections](./maxintersections.md)
- [maxIntersectionsPosition](./maxintersectionsposition.md)
- [maxIntersectionsPosition](./maxintersectionsposition.md)

View File

@ -0,0 +1,68 @@
---
slug: /en/sql-reference/aggregate-functions/reference/quantileInterpolatedWeighted
sidebar_position: 203
---
# quantileInterpolatedWeighted
Computes [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using linear interpolation, taking into account the weight of each element.
To get the interpolated value, all the passed values are combined into an array, which are then sorted by their corresponding weights. Quantile interpolation is then performed using the [weighted percentile method](https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method) by building a cumulative distribution based on weights and then a linear interpolation is performed using the weights and the values to compute the quantiles.
When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function.
**Syntax**
``` sql
quantileInterpolatedWeighted(level)(expr, weight)
```
Alias: `medianInterpolatedWeighted`.
**Arguments**
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).
- `weight` — Column with weights of sequence members. Weight is a number of value occurrences.
**Returned value**
- Quantile of the specified level.
Type:
- [Float64](../../../sql-reference/data-types/float.md) for numeric data type input.
- [Date](../../../sql-reference/data-types/date.md) if input values have the `Date` type.
- [DateTime](../../../sql-reference/data-types/datetime.md) if input values have the `DateTime` type.
**Example**
Input table:
``` text
┌─n─┬─val─┐
│ 0 │ 3 │
│ 1 │ 2 │
│ 2 │ 1 │
│ 5 │ 4 │
└───┴─────┘
```
Query:
``` sql
SELECT quantileInterpolatedWeighted(n, val) FROM t
```
Result:
``` text
┌─quantileInterpolatedWeighted(n, val)─┐
│ 1 │
└──────────────────────────────────────┘
```
**See Also**
- [median](../../../sql-reference/aggregate-functions/reference/median.md#median)
- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles)

View File

@ -9,7 +9,7 @@ sidebar_position: 201
Syntax: `quantiles(level1, level2, …)(x)`
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantileInterpolatedWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
## quantilesExactExclusive

View File

@ -6,6 +6,10 @@ sidebar_label: JSON
# JSON
:::warning
This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/guides/developer/working-with-json/json-load-data.md) instead.
:::
Stores JavaScript Object Notation (JSON) documents in a single column.
`JSON` is an alias for `Object('json')`.

View File

@ -121,7 +121,7 @@ Accepts an empty array and returns a one-element array that is equal to the defa
## range(end), range(\[start, \] end \[, step\])
Returns an array of `UInt` numbers from `start` to `end - 1` by `step`.
Returns an array of numbers from `start` to `end - 1` by `step`. The supported types are [UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64](../data-types/int-uint.md).
**Syntax**
``` sql
@ -130,31 +130,30 @@ range([start, ] end [, step])
**Arguments**
- `start` — The first element of the array. Optional, required if `step` is used. Default value: 0. [UInt](../data-types/int-uint.md)
- `end` — The number before which the array is constructed. Required. [UInt](../data-types/int-uint.md)
- `step` — Determines the incremental step between each element in the array. Optional. Default value: 1. [UInt](../data-types/int-uint.md)
- `start` — The first element of the array. Optional, required if `step` is used. Default value: 0.
- `end` — The number before which the array is constructed. Required.
- `step` — Determines the incremental step between each element in the array. Optional. Default value: 1.
**Returned value**
- Array of `UInt` numbers from `start` to `end - 1` by `step`.
- Array of numbers from `start` to `end - 1` by `step`.
**Implementation details**
- All arguments must be positive values: `start`, `end`, `step` are `UInt` data types, as well as elements of the returned array.
- All arguments `start`, `end`, `step` must be below data types: `UInt8`, `UInt16`, `UInt32`, `UInt64`,`Int8`, `Int16`, `Int32`, `Int64`, as well as elements of the returned array, which's type is a super type of all arguments's.
- An exception is thrown if query results in arrays with a total length of more than number of elements specified by the [function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block) setting.
**Examples**
Query:
``` sql
SELECT range(5), range(1, 5), range(1, 5, 2);
SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2);
```
Result:
```txt
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┐
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │
└─────────────┴─────────────┴────────────────┘
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┬─range(-1, 5, 2)─
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │ [-1,1,3] │
└─────────────┴─────────────┴────────────────┴─────────────────
```
## array(x1, …), operator \[x1, …\]

View File

@ -39,3 +39,16 @@ SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(
│ [68] │ -67417.0770 │ ('2080-03-12 14:17:31.269','110425e5-413f-10a6-05ba-fa6b3e929f15') │
└──────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
```
```sql
CREATE TABLE random (a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) engine=Memory;
INSERT INTO random SELECT * FROM generateRandom() LIMIT 2;
SELECT * FROM random;
```
```text
┌─a────────────────────────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
│ [] │ 68091.8197 │ ('2037-10-02 12:44:23.368','039ecab7-81c2-45ee-208c-844e5c6c5652') │
│ [8,-83,0,-22,65,9,-30,28,64] │ -186233.4909 │ ('2062-01-11 00:06:04.124','69563ea1-5ad1-f870-16d8-67061da0df25') │
└──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
```

View File

@ -117,7 +117,7 @@ SELECT notEmpty([1,2]);
## range(end), range(\[start, \] end \[, step\]) {#range}
返回一个以`step`作为增量步长的从`start`到`end - 1`的`UInt`类型数字数组
返回一个以`step`作为增量步长的从`start`到`end - 1`的整形数字数组, 支持类型包括[`UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`](../data-types/int-uint.md)
**语法**
``` sql
@ -126,31 +126,30 @@ range([start, ] end [, step])
**参数**
- `start` — 数组的第一个元素。可选项,如果设置了`step`时同样需要`start`默认值为0,类型为[UInt](../data-types/int-uint.md)
- `end` — 计数到`end`结束,但不包括`end`,必填项,类型为[UInt](../data-types/int-uint.md)
- `step` — 确定数组中每个元素之间的增量步长。可选项默认值为1,类型为[UInt](../data-types/int-uint.md)
- `start` — 数组的第一个元素。可选项,如果设置了`step`时同样需要`start`默认值为0。
- `end` — 计数到`end`结束,但不包括`end`,必填项。
- `step` — 确定数组中每个元素之间的增量步长。可选项默认值为1。
**返回值**
- 以`step`作为增量步长的从`start`到`end - 1`的`UInt`类型数字数组。
- 以`step`作为增量步长的从`start`到`end - 1`的数字数组。
**注意事项**
- 所有参数必须是正值:`start`、`end`、`step`,类型均为`UInt`,结果数组的元素与此相同
- 所有参数`start`、`end`、`step`必须属于以下几种类型之一:[`UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`](../data-types/int-uint.md)。结果数组的元素数据类型为所有入参类型的最小超类,也必须属于以上几种类型之一
- 如果查询结果的数组总长度超过[function_range_max_elements_in_block](../../operations/settings/settings.md#settings-function_range_max_elements_in_block)指定的元素数,将会抛出异常。
**示例**
查询语句:
``` sql
SELECT range(5), range(1, 5), range(1, 5, 2);
SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2);
```
结果:
```txt
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┐
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │
└─────────────┴─────────────┴────────────────┘
┌─range(5)────┬─range(1, 5)─┬─range(1, 5, 2)─┬─range(-1, 5, 2)─
│ [0,1,2,3,4] │ [1,2,3,4] │ [1,3] │ [-1,1,3] │
└─────────────┴─────────────┴────────────────┴─────────────────
```
## array(x1, …), operator \[x1, …\] {#arrayx1-operator-x1}

View File

@ -140,6 +140,7 @@ namespace CurrentMetrics
namespace ProfileEvents
{
extern const Event MainConfigLoads;
extern const Event ServerStartupMilliseconds;
}
namespace fs = std::filesystem;
@ -652,6 +653,8 @@ static void sanityChecks(Server & server)
int Server::main(const std::vector<std::string> & /*args*/)
try
{
Stopwatch startup_watch;
Poco::Logger * log = &logger();
UseSSL use_ssl;
@ -1822,6 +1825,9 @@ try
LOG_INFO(log, "Ready for connections.");
}
startup_watch.stop();
ProfileEvents::increment(ProfileEvents::ServerStartupMilliseconds, startup_watch.elapsedMilliseconds());
try
{
global_context->startClusterDiscovery();

View File

@ -207,7 +207,7 @@ private:
{
// Fuse points if their text representations differ only in last digit
auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits<Mean>::epsilon();
if (points[left].mean + min_diff >= points[right].mean)
if (points[left].mean + std::fabs(min_diff) >= points[right].mean)
{
points[left] = points[left] + points[right];
}

View File

@ -232,6 +232,9 @@ struct NameQuantilesExactInclusive { static constexpr auto name = "quantilesExac
struct NameQuantileExactWeighted { static constexpr auto name = "quantileExactWeighted"; };
struct NameQuantilesExactWeighted { static constexpr auto name = "quantilesExactWeighted"; };
struct NameQuantileInterpolatedWeighted { static constexpr auto name = "quantileInterpolatedWeighted"; };
struct NameQuantilesInterpolatedWeighted { static constexpr auto name = "quantilesInterpolatedWeighted"; };
struct NameQuantileTiming { static constexpr auto name = "quantileTiming"; };
struct NameQuantileTimingWeighted { static constexpr auto name = "quantileTimingWeighted"; };
struct NameQuantilesTiming { static constexpr auto name = "quantilesTiming"; };

View File

@ -0,0 +1,70 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileInterpolatedWeighted.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Core/Field.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
template <typename Value, bool _> using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantileInterpolatedWeighted, true, void, false>;
template <typename Value, bool _> using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantilesInterpolatedWeighted, true, void, true>;
template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return std::make_shared<Function<TYPE, true>>(argument_types, params);
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
if (which.idx == TypeIndex::Int128) return std::make_shared<Function<Int128, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt128) return std::make_shared<Function<UInt128, true>>(argument_types, params);
if (which.idx == TypeIndex::Int256) return std::make_shared<Function<Int256, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt256) return std::make_shared<Function<UInt256, true>>(argument_types, params);
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory & factory)
{
/// For aggregate functions returning array we cannot return NULL on empty set.
AggregateFunctionProperties properties = { .returns_default_when_only_null = true };
factory.registerFunction(NameQuantileInterpolatedWeighted::name, createAggregateFunctionQuantile<FuncQuantileInterpolatedWeighted>);
factory.registerFunction(NameQuantilesInterpolatedWeighted::name, { createAggregateFunctionQuantile<FuncQuantilesInterpolatedWeighted>, properties });
/// 'median' is an alias for 'quantile'
factory.registerAlias("medianInterpolatedWeighted", NameQuantileInterpolatedWeighted::name);
}
}

View File

@ -0,0 +1,308 @@
#pragma once
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Approximates Quantile by:
* - sorting input values and weights
* - building a cumulative distribution based on weights
* - performing linear interpolation between the weights and values
*
*/
template <typename Value>
struct QuantileInterpolatedWeighted
{
struct Int128Hash
{
size_t operator()(Int128 x) const
{
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
}
};
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
Map map;
void add(const Value & x)
{
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, Weight weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileInterpolatedWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
Value get(Float64 level) const
{
return getImpl<Value>(level);
}
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
{
getManyImpl<Value>(levels, indices, size, result);
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64) const
{
throw Exception("Method getFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
}
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
{
throw Exception("Method getManyFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
}
private:
using Pair = typename std::pair<UnderlyingType, Float64>;
/// Get the value of the `level` quantile. The level must be between 0 and 1.
template <typename T>
T getImpl(Float64 level) const
{
size_t size = map.size();
if (0 == size)
return std::numeric_limits<Value>::quiet_NaN();
/// Maintain a vector of pair of values and weights for easier sorting and for building
/// a cumulative distribution using the provided weights.
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
/// Note: weight provided must be a 64-bit integer
/// Float64 is used as accumulator here to get approximate results.
/// But weight used in the internal array is stored as Float64 as we
/// do some quantile estimation operation which involves division and
/// require Float64 level of precision.
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.push_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
/// perform linear interpolation
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
return static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
template <typename T>
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
{
size_t size = map.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = Value();
return;
}
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.emplace_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
for (size_t level_index = 0; level_index < num_levels; ++level_index)
{
/// perform linear interpolation for every level
auto level = levels[indices[level_index]];
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
result[indices[level_index]] = static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
}
/// This ignores overflows or NaN's that might arise during add, sub and mul operations and doesn't aim to provide exact
/// results since `the quantileInterpolatedWeighted` function itself relies mainly on approximation.
UnderlyingType NO_SANITIZE_UNDEFINED interpolate(Float64 level, Float64 xl, Float64 xr, UnderlyingType yl, UnderlyingType yr) const
{
UnderlyingType dy = yr - yl;
Float64 dx = xr - xl;
dx = dx == 0 ? 1 : dx; /// to handle NaN behavior that might arise during integer division below.
/// yl + (dy / dx) * (level - xl)
return static_cast<UnderlyingType>(yl + (dy / dx) * (level - xl));
}
};
}

View File

@ -21,6 +21,7 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExactLow(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExactHigh(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExactInclusive(AggregateFunctionFactory &);
@ -106,6 +107,7 @@ void registerAggregateFunctions()
registerAggregateFunctionsQuantileDeterministic(factory);
registerAggregateFunctionsQuantileExact(factory);
registerAggregateFunctionsQuantileExactWeighted(factory);
registerAggregateFunctionsQuantileInterpolatedWeighted(factory);
registerAggregateFunctionsQuantileExactLow(factory);
registerAggregateFunctionsQuantileExactHigh(factory);
registerAggregateFunctionsQuantileExactInclusive(factory);

View File

@ -11,6 +11,7 @@
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTColumnsTransformers.h>
namespace DB
{
@ -206,19 +207,43 @@ QueryTreeNodePtr MatcherNode::cloneImpl() const
ASTPtr MatcherNode::toASTImpl() const
{
ASTPtr result;
ASTPtr transformers;
if (!children.empty())
{
transformers = std::make_shared<ASTColumnsTransformerList>();
for (const auto & child : children)
transformers->children.push_back(child->toAST());
}
if (matcher_type == MatcherNodeType::ASTERISK)
{
if (qualified_identifier.empty())
{
result = std::make_shared<ASTAsterisk>();
auto asterisk = std::make_shared<ASTAsterisk>();
if (transformers)
{
asterisk->transformers = std::move(transformers);
asterisk->children.push_back(asterisk->transformers);
}
result = asterisk;
}
else
{
auto qualified_asterisk = std::make_shared<ASTQualifiedAsterisk>();
auto identifier_parts = qualified_identifier.getParts();
qualified_asterisk->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
qualified_asterisk->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
qualified_asterisk->children.push_back(qualified_asterisk->qualifier);
if (transformers)
{
qualified_asterisk->transformers = std::move(transformers);
qualified_asterisk->children.push_back(qualified_asterisk->transformers);
}
result = qualified_asterisk;
}
@ -229,6 +254,13 @@ ASTPtr MatcherNode::toASTImpl() const
{
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
regexp_matcher->setPattern(columns_matcher->pattern());
if (transformers)
{
regexp_matcher->transformers = std::move(transformers);
regexp_matcher->children.push_back(regexp_matcher->transformers);
}
result = regexp_matcher;
}
else
@ -237,7 +269,14 @@ ASTPtr MatcherNode::toASTImpl() const
regexp_matcher->setPattern(columns_matcher->pattern());
auto identifier_parts = qualified_identifier.getParts();
regexp_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
regexp_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
regexp_matcher->children.push_back(regexp_matcher->qualifier);
if (transformers)
{
regexp_matcher->transformers = std::move(transformers);
regexp_matcher->children.push_back(regexp_matcher->transformers);
}
result = regexp_matcher;
}
@ -257,23 +296,36 @@ ASTPtr MatcherNode::toASTImpl() const
{
auto columns_list_matcher = std::make_shared<ASTColumnsListMatcher>();
columns_list_matcher->column_list = std::move(column_list);
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
if (transformers)
{
columns_list_matcher->transformers = std::move(transformers);
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
}
result = columns_list_matcher;
}
else
{
auto columns_list_matcher = std::make_shared<ASTQualifiedColumnsListMatcher>();
columns_list_matcher->column_list = std::move(column_list);
auto identifier_parts = qualified_identifier.getParts();
columns_list_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
columns_list_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
columns_list_matcher->column_list = std::move(column_list);
columns_list_matcher->children.push_back(columns_list_matcher->qualifier);
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
if (transformers)
{
columns_list_matcher->transformers = std::move(transformers);
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
}
result = columns_list_matcher;
}
}
for (const auto & child : children)
result->children.push_back(child->toAST());
return result;
}

View File

@ -111,7 +111,7 @@ private:
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const;
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
ASTPtr query;
QueryTreeNodePtr query_tree_node;
@ -439,13 +439,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
}
else if (const auto * asterisk = expression->as<ASTAsterisk>())
{
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
auto column_transformers = buildColumnTransformers(asterisk->transformers, context);
result = std::make_shared<MatcherNode>(std::move(column_transformers));
}
else if (const auto * qualified_asterisk = expression->as<ASTQualifiedAsterisk>())
{
auto & qualified_identifier = qualified_asterisk->children.at(0)->as<ASTTableIdentifier &>();
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
auto & qualified_identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
auto column_transformers = buildColumnTransformers(qualified_asterisk->transformers, context);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_transformers));
}
else if (const auto * ast_literal = expression->as<ASTLiteral>())
@ -543,7 +543,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
}
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
{
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context);
result = std::make_shared<MatcherNode>(columns_regexp_matcher->getMatcher(), std::move(column_transformers));
}
else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>())
@ -557,18 +557,18 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
}
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
auto column_transformers = buildColumnTransformers(columns_list_matcher->transformers, context);
result = std::make_shared<MatcherNode>(std::move(column_list_identifiers), std::move(column_transformers));
}
else if (const auto * qualified_columns_regexp_matcher = expression->as<ASTQualifiedColumnsRegexpMatcher>())
{
auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as<ASTTableIdentifier &>();
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>();
auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers));
}
else if (const auto * qualified_columns_list_matcher = expression->as<ASTQualifiedColumnsListMatcher>())
{
auto & qualified_identifier = qualified_columns_list_matcher->children.at(0)->as<ASTTableIdentifier &>();
auto & qualified_identifier = qualified_columns_list_matcher->qualifier->as<ASTIdentifier &>();
Identifiers column_list_identifiers;
column_list_identifiers.reserve(qualified_columns_list_matcher->column_list->children.size());
@ -579,7 +579,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
}
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
auto column_transformers = buildColumnTransformers(qualified_columns_list_matcher->transformers, context);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers));
}
else
@ -833,15 +833,15 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
}
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
{
ColumnTransformersNodes column_transformers;
size_t children_size = matcher_expression->children.size();
for (; start_child_index < children_size; ++start_child_index)
if (!matcher_expression)
return column_transformers;
for (const auto & child : matcher_expression->children)
{
const auto & child = matcher_expression->children[start_child_index];
if (auto * apply_transformer = child->as<ASTColumnsApplyTransformer>())
{
if (apply_transformer->lambda)

View File

@ -449,7 +449,8 @@ The server successfully detected this situation and will download merged part fr
M(OverflowBreak, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'break' and the result is incomplete.") \
M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \
M(OverflowAny, "Number of times approximate GROUP BY was in effect: when aggregation was performed only on top of first 'max_rows_to_group_by' unique keys and other keys were ignored due to 'group_by_overflow_mode' = 'any'.") \
\
M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\
namespace ProfileEvents
{

View File

@ -28,6 +28,7 @@ protected:
bool isCompression() const override { return false; }
bool isGenericCompression() const override { return false; }
bool isDelta() const override { return true; }
private:
UInt8 delta_bytes_size;

View File

@ -133,6 +133,7 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isDelta() const override { return true; }
private:
UInt8 data_bytes_size;

View File

@ -39,6 +39,7 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isFloatingPointTimeSeries() const override { return true; }
private:
static constexpr UInt32 HEADER_SIZE = 2;

View File

@ -123,6 +123,7 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isFloatingPointTimeSeries() const override { return true; }
private:
UInt8 data_bytes_size;
@ -444,14 +445,19 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc
void registerCodecGorilla(CompressionCodecFactory & factory)
{
UInt8 method_code = static_cast<UInt8>(CompressionMethodByte::Gorilla);
factory.registerCompressionCodecWithType("Gorilla", method_code,
[&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
{
if (arguments)
throw Exception("Codec Gorilla does not accept any arguments", ErrorCodes::BAD_ARGUMENTS);
if (column_type != nullptr)
if (!WhichDataType(*column_type).isFloat())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Gorilla codec is not applicable for {} because the data type is not float",
column_type->getName());
UInt8 data_bytes_size = column_type ? getDataBytesSize(column_type) : 0;
return std::make_shared<CompressionCodecGorilla>(data_bytes_size);
});
};
factory.registerCompressionCodecWithType("Gorilla", method_code, codec_builder);
}
}

View File

@ -59,15 +59,17 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool is_compression = false;
bool has_none = false;
bool with_compressing_codec = false;
bool with_none_codec = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> encryption_codecs;
std::optional<size_t> first_delta_codec_pos;
std::optional<size_t> last_floating_point_time_series_codec_pos;
std::set<size_t> encryption_codecs_pos;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
const ASTPtr & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
@ -83,8 +85,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
/// Replace "Default" codec by configured default codec which may depend on different settings and data properties at runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
@ -136,21 +137,27 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
with_compressing_codec |= result_codec->isCompression();
with_none_codec |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isEncryption())
encryption_codecs.insert(i);
encryption_codecs_pos.insert(i);
if (result_codec->isDelta() && !first_delta_codec_pos.has_value())
first_delta_codec_pos = i;
if (result_codec->isFloatingPointTimeSeries())
last_floating_point_time_series_codec_pos = i;
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && has_none)
if (codecs_descriptions->children.size() > 1 && with_none_codec)
throw Exception(
"It does not make sense to have codec NONE along with other compression codecs: " + codec_description
+ ". (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
@ -159,7 +166,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply encryption codecs solely without anything else.
if (!is_compression && !has_none && encryption_codecs.size() != codecs_descriptions->children.size())
if (!with_compressing_codec && !with_none_codec && encryption_codecs_pos.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
@ -171,8 +178,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any non-encryption codecs
/// after encryption one.
if (!encryption_codecs.empty() &&
*encryption_codecs.begin() != codecs_descriptions->children.size() - encryption_codecs.size())
if (!encryption_codecs_pos.empty() &&
*encryption_codecs_pos.begin() != codecs_descriptions->children.size() - encryption_codecs_pos.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
@ -181,11 +188,18 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs.size())
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs_pos.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// Floating point time series codecs usually have implicit delta compression (or something equivalent), so it does not make
/// sense to run delta compression manually. Another reason for blocking such combination is occasional data corruption (#45195).
if (first_delta_codec_pos.has_value() && last_floating_point_time_series_codec_pos.has_value()
&& (*first_delta_codec_pos < last_floating_point_time_series_codec_pos))
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply delta transformations before floating point time series codecs."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we

View File

@ -113,6 +113,12 @@ public:
/// If it does nothing.
virtual bool isNone() const { return false; }
/// If the only purpose of the codec is to delta (or double-delta) the data.
virtual bool isDelta() const { return false; }
/// If the codec is specialized for floating point time series.
virtual bool isFloatingPointTimeSeries() const { return false; }
protected:
/// This is used for fuzz testing
friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size);

View File

@ -529,6 +529,13 @@ public:
TEST_P(CodecTest, TranscodingWithDataType)
{
/// Gorilla can only be applied to floating point columns
bool codec_is_gorilla = std::get<0>(GetParam()).codec_statement.find("Gorilla") != std::string::npos;
WhichDataType which(std::get<1>(GetParam()).data_type.get());
bool data_is_float = which.isFloat();
if (codec_is_gorilla && !data_is_float)
GTEST_SKIP() << "Skipping Gorilla-compressed integer column";
const auto codec = makeCodec(CODEC_WITH_DATA_TYPE);
testTranscoding(*codec);
}
@ -1204,68 +1211,20 @@ auto DDperformanceTestSequence()
}
// prime numbers in ascending order with some random repetitions hit all the cases of Gorilla.
auto PrimesWithMultiplierGenerator = [](int multiplier = 1)
{
return [multiplier](auto i)
{
static const int vals[] = {
2, 3, 5, 7, 11, 11, 13, 17, 19, 23, 29, 29, 31, 37, 41, 43,
47, 47, 53, 59, 61, 61, 67, 71, 73, 79, 83, 89, 89, 97, 101, 103,
107, 107, 109, 113, 113, 127, 127, 127
};
static const size_t count = sizeof(vals)/sizeof(vals[0]);
return static_cast<UInt64>(vals[i % count]) * multiplier;
};
};
template <typename ValueType>
auto GCompatibilityTestSequence()
{
// Also multiply result by some factor to test large values on types that can hold those.
return generateSeq<ValueType>(G(PrimesWithMultiplierGenerator(intExp10(sizeof(ValueType)))), 0, 42);
}
INSTANTIATE_TEST_SUITE_P(Gorilla,
CodecTestCompatibility,
::testing::Combine(
::testing::Values(Codec("Gorilla")),
::testing::ValuesIn(std::initializer_list<std::tuple<CodecTestSequence, std::string>>{
{
GCompatibilityTestSequence<Int8>(),
BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a")
},
{
GCompatibilityTestSequence<UInt8>(),
BIN_STR("\x95\x35\x00\x00\x00\x2a\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x14\xe1\xdd\x25\xe5\x7b\x29\x86\xee\x2a\x16\x5a\xc5\x0b\x23\x75\x1b\x3c\xb1\x97\x8b\x5f\xcb\x43\xd9\xc5\x48\xab\x23\xaf\x62\x93\x71\x4a\x73\x0f\xc6\x0a")
},
{
GCompatibilityTestSequence<Int16>(),
BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4")
},
{
GCompatibilityTestSequence<UInt16>(),
BIN_STR("\x95\x52\x00\x00\x00\x54\x00\x00\x00\x02\x00\x2a\x00\x00\x00\xc8\x00\xdc\xfe\x66\xdb\x1f\x4e\xa7\xde\xdc\xd5\xec\x6e\xf7\x37\x3a\x23\xe7\x63\xf5\x6a\x8e\x99\x37\x34\xf9\xf8\x2e\x76\x35\x2d\x51\xbb\x3b\xc3\x6d\x13\xbf\x86\x53\x9e\x25\xe4\xaf\xaf\x63\xd5\x6a\x6e\x76\x35\x3a\x27\xd3\x0f\x91\xae\x6b\x33\x57\x6e\x64\xcc\x55\x81\xe4")
},
{
GCompatibilityTestSequence<Int32>(),
BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4")
},
{
GCompatibilityTestSequence<UInt32>(),
BIN_STR("\x95\x65\x00\x00\x00\xa8\x00\x00\x00\x04\x00\x2a\x00\x00\x00\x20\x4e\x00\x00\xe4\x57\x63\xc0\xbb\x67\xbc\xce\x91\x97\x99\x15\x9e\xe3\x36\x3f\x89\x5f\x8e\xf2\xec\x8e\xd3\xbf\x75\x43\x58\xc4\x7e\xcf\x93\x43\x38\xc6\x91\x36\x1f\xe7\xb6\x11\x6f\x02\x73\x46\xef\xe0\xec\x50\xfb\x79\xcb\x9c\x14\xfa\x13\xea\x8d\x66\x43\x48\xa0\xde\x3a\xcf\xff\x26\xe0\x5f\x93\xde\x5e\x7f\x6e\x36\x5e\xe6\xb4\x66\x5d\xb0\x0e\xc4")
},
{
GCompatibilityTestSequence<Int64>(),
BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10")
},
{
GCompatibilityTestSequence<UInt64>(),
BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10")
},
})
)
);
// auto PrimesWithMultiplierGenerator = [](int multiplier = 1)
// {
// return [multiplier](auto i)
// {
// static const int vals[] = {
// 2, 3, 5, 7, 11, 11, 13, 17, 19, 23, 29, 29, 31, 37, 41, 43,
// 47, 47, 53, 59, 61, 61, 67, 71, 73, 79, 83, 89, 89, 97, 101, 103,
// 107, 107, 109, 113, 113, 127, 127, 127
// };
// static const size_t count = sizeof(vals)/sizeof(vals[0]);
//
// return static_cast<UInt64>(vals[i % count]) * multiplier;
// };
// };
// These 'tests' try to measure performance of encoding and decoding and hence only make sense to be run locally,
// also they require pretty big data to run against and generating this data slows down startup of unit test process.

View File

@ -2,6 +2,7 @@
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Interpreters/misc.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/getClusterName.h>
@ -175,7 +176,7 @@ namespace
/// Finds dependencies of a function.
void visitFunction(const ASTFunction & function)
{
if (function.name == "joinGet" || function.name == "dictHas" || function.name == "dictIsIn" || function.name.starts_with("dictGet"))
if (functionIsJoinGet(function.name) || functionIsDictGet(function.name))
{
/// dictGet('dict_name', attr_names, id_expr)
/// dictHas('dict_name', id_expr)

View File

@ -1,6 +1,7 @@
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -52,23 +53,41 @@ bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child)
return true;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function)
ssize_t DDLMatcherBase::getPositionOfTableNameArgumentToEvaluate(const ASTFunction & function)
{
if (function.name == "joinGet" ||
function.name == "dictHas" ||
function.name == "dictIsIn" ||
function.name.starts_with("dictGet"))
if (functionIsJoinGet(function.name) || functionIsDictGet(function.name))
return 0;
if (Poco::toLower(function.name) == "in")
return -1;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgumentToVisit(const ASTFunction & function)
{
ssize_t maybe_res = getPositionOfTableNameArgumentToEvaluate(function);
if (0 <= maybe_res)
return maybe_res;
if (functionIsInOrGlobalInOperator(function.name))
{
if (function.children.empty())
return -1;
const auto * args = function.children[0]->as<ASTExpressionList>();
if (!args || args->children.size() != 2)
return -1;
if (args->children[1]->as<ASTFunction>())
return -1;
return 1;
}
return -1;
}
void DDLLoadingDependencyVisitor::visit(const ASTFunction & function, Data & data)
{
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
ssize_t table_name_arg_idx = getPositionOfTableNameArgumentToVisit(function);
if (table_name_arg_idx < 0)
return;
extractTableNameFromArgument(function, data, table_name_arg_idx);

View File

@ -23,7 +23,8 @@ class DDLMatcherBase
{
public:
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
static ssize_t getPositionOfTableNameArgument(const ASTFunction & function);
static ssize_t getPositionOfTableNameArgumentToVisit(const ASTFunction & function);
static ssize_t getPositionOfTableNameArgumentToEvaluate(const ASTFunction & function);
};
/// Visits ASTCreateQuery and extracts the names of all tables which should be loaded before a specified table.

View File

@ -23,7 +23,7 @@ void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & d
{
/// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))"
/// with "dictGet('db_name.dict', 'value', toUInt32(1))"
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
ssize_t table_name_arg_idx = getPositionOfTableNameArgumentToEvaluate(function);
if (table_name_arg_idx < 0)
return;

View File

@ -118,6 +118,16 @@ struct MatchImpl
if (haystack_offsets.empty())
return;
/// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack:
/// - col [not] [i]like '%' / '%%'
/// - match(col, '.*')
if ((is_like && (needle == "%" or needle == "%%")) || (!is_like && (needle == ".*" || needle == ".*?")))
{
for (auto & x : res)
x = !negate;
return;
}
/// Special case that the [I]LIKE expression reduces to finding a substring in a string
String strstr_pattern;
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
@ -267,6 +277,16 @@ struct MatchImpl
if (haystack.empty())
return;
/// Shortcut for the silly but practical case that the pattern matches everything/nothing independently of the haystack:
/// - col [not] [i]like '%' / '%%'
/// - match(col, '.*')
if ((is_like && (needle == "%" or needle == "%%")) || (!is_like && (needle == ".*" || needle == ".*?")))
{
for (auto & x : res)
x = !negate;
return;
}
/// Special case that the [I]LIKE expression reduces to finding a substring in a string
String strstr_pattern;
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))

View File

@ -28,13 +28,29 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier &
database = current_database;
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
{
alias = identifier.tryGetAlias();
if (identifier.name_parts.size() == 2)
std::tie(database, table) = std::tie(identifier.name_parts[0], identifier.name_parts[1]);
else if (identifier.name_parts.size() == 1)
table = identifier.name_parts[0];
else
throw Exception("Logical error: invalid identifier", ErrorCodes::LOGICAL_ERROR);
if (database.empty())
database = current_database;
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
{
const auto * identifier = node->as<ASTTableIdentifier>();
if (!identifier)
throw Exception("Logical error: table identifier expected", ErrorCodes::LOGICAL_ERROR);
*this = DatabaseAndTableWithAlias(*identifier, current_database);
if (const auto * table_identifier = node->as<ASTTableIdentifier>())
*this = DatabaseAndTableWithAlias(*table_identifier, current_database);
else if (const auto * identifier = node->as<ASTIdentifier>())
*this = DatabaseAndTableWithAlias(*identifier, current_database);
else
throw Exception("Logical error: identifier or table identifier expected", ErrorCodes::LOGICAL_ERROR);
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database)

View File

@ -14,6 +14,7 @@ namespace DB
{
class ASTSelectQuery;
class ASTIdentifier;
class ASTTableIdentifier;
struct ASTTableExpression;
@ -28,6 +29,7 @@ struct DatabaseAndTableWithAlias
DatabaseAndTableWithAlias() = default;
explicit DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
explicit DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
explicit DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database = "");
explicit DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");

View File

@ -25,6 +25,7 @@ static const std::unordered_map<String, String> quantile_fuse_name_mapping = {
{NameQuantileExactInclusive::name, NameQuantilesExactInclusive::name},
{NameQuantileExactLow::name, NameQuantilesExactLow::name},
{NameQuantileExactWeighted::name, NameQuantilesExactWeighted::name},
{NameQuantileInterpolatedWeighted::name, NameQuantilesInterpolatedWeighted::name},
{NameQuantileTDigest::name, NameQuantilesTDigest::name},
{NameQuantileTDigestWeighted::name, NameQuantilesTDigestWeighted::name},
{NameQuantileTiming::name, NameQuantilesTiming::name},
@ -61,9 +62,11 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr
const auto & arguments = func->arguments->children;
bool need_two_args = func->name == NameQuantileDeterministic::name || func->name == NameQuantileExactWeighted::name
|| func->name == NameQuantileTimingWeighted::name || func->name == NameQuantileTDigestWeighted::name
|| func->name == NameQuantileBFloat16Weighted::name;
|| func->name == NameQuantileInterpolatedWeighted::name || func->name == NameQuantileTimingWeighted::name
|| func->name == NameQuantileTDigestWeighted::name || func->name == NameQuantileBFloat16Weighted::name;
if (arguments.size() != (need_two_args ? 2 : 1))
return;

View File

@ -288,6 +288,20 @@ struct ExplainSettings : public Settings
}
};
struct QuerySyntaxSettings
{
bool oneline = false;
constexpr static char name[] = "SYNTAX";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"oneline", oneline},
};
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
};
template <typename Settings>
ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
{
@ -362,13 +376,12 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
}
case ASTExplainQuery::AnalyzedSyntax:
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
auto settings = checkAndGetSettings<QuerySyntaxSettings>(ast.getSettings());
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, settings.oneline));
break;
}
case ASTExplainQuery::QueryTree:

View File

@ -49,7 +49,8 @@ ASTPtr makeSubqueryTemplate()
ASTPtr makeSubqueryQualifiedAsterisk()
{
auto asterisk = std::make_shared<ASTQualifiedAsterisk>();
asterisk->children.emplace_back(std::make_shared<ASTTableIdentifier>("--.s"));
asterisk->qualifier = std::make_shared<ASTIdentifier>("--.s");
asterisk->children.push_back(asterisk->qualifier);
return asterisk;
}
@ -153,24 +154,34 @@ private:
for (auto & table_name : data.tables_order)
data.addTableColumns(table_name, columns);
for (const auto & transformer : asterisk->children)
IASTColumnsTransformer::transform(transformer, columns);
if (asterisk->transformers)
{
for (const auto & transformer : asterisk->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
{
has_asterisks = true;
auto & identifier = child->children[0]->as<ASTTableIdentifier &>();
if (!qualified_asterisk->qualifier)
throw Exception("Logical error: qualified asterisk must have a qualifier", ErrorCodes::LOGICAL_ERROR);
auto & identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
data.addTableColumns(identifier.name(), columns);
// QualifiedAsterisk's transformers start to appear at child 1
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
if (qualified_asterisk->transformers)
{
if (it->get()->as<ASTColumnsApplyTransformer>() || it->get()->as<ASTColumnsExceptTransformer>() || it->get()->as<ASTColumnsReplaceTransformer>())
IASTColumnsTransformer::transform(*it, columns);
else
throw Exception("Logical error: qualified asterisk must only have children of IASTColumnsTransformer type", ErrorCodes::LOGICAL_ERROR);
for (const auto & transformer : qualified_asterisk->transformers->children)
{
if (transformer->as<ASTColumnsApplyTransformer>() ||
transformer->as<ASTColumnsExceptTransformer>() ||
transformer->as<ASTColumnsReplaceTransformer>())
IASTColumnsTransformer::transform(transformer, columns);
else
throw Exception("Logical error: qualified asterisk must only have children of IASTColumnsTransformer type", ErrorCodes::LOGICAL_ERROR);
}
}
}
else if (const auto * columns_list_matcher = child->as<ASTColumnsListMatcher>())
@ -180,8 +191,11 @@ private:
for (const auto & ident : columns_list_matcher->column_list->children)
columns.emplace_back(ident->clone());
for (const auto & transformer : columns_list_matcher->children)
IASTColumnsTransformer::transform(transformer, columns);
if (columns_list_matcher->transformers)
{
for (const auto & transformer : columns_list_matcher->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else if (const auto * columns_regexp_matcher = child->as<ASTColumnsRegexpMatcher>())
{
@ -193,8 +207,11 @@ private:
columns,
[&](const String & column_name) { return columns_regexp_matcher->isColumnMatching(column_name); });
for (const auto & transformer : columns_regexp_matcher->children)
IASTColumnsTransformer::transform(transformer, columns);
if (columns_regexp_matcher->transformers)
{
for (const auto & transformer : columns_regexp_matcher->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else
data.new_select_expression_list->children.push_back(child);
@ -425,6 +442,7 @@ private:
{
if (data.expression_list->children.empty())
data.expression_list->children.emplace_back(std::make_shared<ASTAsterisk>());
select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list));
}
data.done = true;

View File

@ -154,7 +154,7 @@ private:
static void visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
{
auto & identifier = node.children[0]->as<ASTTableIdentifier &>();
auto & identifier = node.qualifier->as<ASTIdentifier &>();
bool rewritten = false;
for (const auto & table : data)
{

View File

@ -156,21 +156,19 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D
func_arguments->children.clear();
}
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk &, const ASTPtr & ast, Data & data)
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
{
if (ast->children.empty())
throw Exception("Logical error: qualified asterisk must have children", ErrorCodes::LOGICAL_ERROR);
auto & ident = ast->children[0];
if (!node.qualifier)
throw Exception("Logical error: qualified asterisk must have a qualifier", ErrorCodes::LOGICAL_ERROR);
/// @note it could contain table alias as table name.
DatabaseAndTableWithAlias db_and_table(ident);
DatabaseAndTableWithAlias db_and_table(node.qualifier);
for (const auto & known_table : data.tables)
if (db_and_table.satisfies(known_table.table, true))
return;
throw Exception("Unknown qualified identifier: " + ident->getAliasOrColumnName(), ErrorCodes::UNKNOWN_IDENTIFIER);
throw Exception("Unknown qualified identifier: " + node.qualifier->getAliasOrColumnName(), ErrorCodes::UNKNOWN_IDENTIFIER);
}
void TranslateQualifiedNamesMatcher::visit(ASTTableJoin & join, const ASTPtr & , Data & data)
@ -266,16 +264,22 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
first_table = false;
}
for (const auto & transformer : asterisk->children)
IASTColumnsTransformer::transform(transformer, columns);
if (asterisk->transformers)
{
for (const auto & transformer : asterisk->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else if (auto * asterisk_column_list = child->as<ASTColumnsListMatcher>())
{
for (const auto & ident : asterisk_column_list->column_list->children)
columns.emplace_back(ident->clone());
for (const auto & transformer : asterisk_column_list->children)
IASTColumnsTransformer::transform(transformer, columns);
if (asterisk_column_list->transformers)
{
for (const auto & transformer : asterisk_column_list->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else if (const auto * asterisk_regexp_pattern = child->as<ASTColumnsRegexpMatcher>())
{
@ -292,12 +296,15 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
first_table = false;
}
for (const auto & transformer : asterisk_regexp_pattern->children)
IASTColumnsTransformer::transform(transformer, columns);
if (asterisk_regexp_pattern->transformers)
{
for (const auto & transformer : asterisk_regexp_pattern->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
{
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->qualifier);
for (const auto & table : tables_with_columns)
{
@ -309,10 +316,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
}
}
// QualifiedAsterisk's transformers start to appear at child 1
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
if (qualified_asterisk->transformers)
{
IASTColumnsTransformer::transform(*it, columns);
for (const auto & transformer : qualified_asterisk->transformers->children)
IASTColumnsTransformer::transform(transformer, columns);
}
}
else

View File

@ -8,21 +8,37 @@ namespace DB
ASTPtr ASTAsterisk::clone() const
{
auto clone = std::make_shared<ASTAsterisk>(*this);
clone->cloneChildren();
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
return clone;
}
void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const { ostr.write('*'); }
void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const
{
if (expression)
{
expression->appendColumnName(ostr);
writeCString(".", ostr);
}
ostr.write('*');
}
void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
}
settings.ostr << "*";
/// Format column transformers
for (const auto & child : children)
if (transformers)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}

View File

@ -16,6 +16,8 @@ public:
ASTPtr clone() const override;
void appendColumnName(WriteBuffer & ostr) const override;
ASTPtr expression;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};

View File

@ -18,12 +18,20 @@ namespace ErrorCodes
ASTPtr ASTColumnsRegexpMatcher::clone() const
{
auto clone = std::make_shared<ASTColumnsRegexpMatcher>(*this);
clone->cloneChildren();
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
return clone;
}
void ASTColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
{
if (expression)
{
expression->appendColumnName(ostr);
writeCString(".", ostr);
}
writeCString("COLUMNS(", ostr);
writeQuotedString(original_pattern, ostr);
writeChar(')', ostr);
@ -38,15 +46,21 @@ void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const
void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
}
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(original_pattern);
settings.ostr << ")";
/// Format column transformers
for (const auto & child : children)
if (transformers)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}
@ -60,6 +74,11 @@ void ASTColumnsRegexpMatcher::setPattern(String pattern)
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
const String & ASTColumnsRegexpMatcher::getPattern() const
{
return original_pattern;
}
const std::shared_ptr<re2::RE2> & ASTColumnsRegexpMatcher::getMatcher() const
{
return column_matcher;
@ -73,19 +92,23 @@ bool ASTColumnsRegexpMatcher::isColumnMatching(const String & column_name) const
ASTPtr ASTColumnsListMatcher::clone() const
{
auto clone = std::make_shared<ASTColumnsListMatcher>(*this);
clone->column_list = column_list->clone();
clone->cloneChildren();
return clone;
}
void ASTColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const
{
column_list->updateTreeHash(hash_state);
IAST::updateTreeHashImpl(hash_state);
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
clone->column_list = column_list->clone();
clone->children.push_back(clone->column_list);
return clone;
}
void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
{
if (expression)
{
expression->appendColumnName(ostr);
writeCString(".", ostr);
}
writeCString("COLUMNS(", ostr);
for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
@ -99,7 +122,15 @@ void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (expression)
{
expression->formatImpl(settings, state, frame);
settings.ostr << ".";
}
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
@ -111,33 +142,39 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt
}
settings.ostr << ")";
/// Format column transformers
for (const auto & child : children)
if (transformers)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}
ASTPtr ASTQualifiedColumnsRegexpMatcher::clone() const
{
auto clone = std::make_shared<ASTQualifiedColumnsRegexpMatcher>(*this);
clone->cloneChildren();
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
clone->qualifier = qualifier->clone();
clone->children.push_back(clone->qualifier);
return clone;
}
void ASTQualifiedColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
{
const auto & qualifier = children.at(0);
qualifier->appendColumnName(ostr);
writeCString(".COLUMNS(", ostr);
writeQuotedString(original_pattern, ostr);
writeChar(')', ostr);
}
void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern)
void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern, bool set_matcher)
{
original_pattern = std::move(pattern);
if (!set_matcher)
return;
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
if (!column_matcher->ok())
throw DB::Exception(
@ -166,35 +203,35 @@ void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & setting
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(original_pattern);
settings.ostr << ")";
/// Format column transformers
size_t children_size = children.size();
for (size_t i = 1; i < children_size; ++i)
if (transformers)
{
const auto & child = children[i];
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}
ASTPtr ASTQualifiedColumnsListMatcher::clone() const
{
auto clone = std::make_shared<ASTQualifiedColumnsListMatcher>(*this);
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
clone->qualifier = qualifier->clone();
clone->column_list = column_list->clone();
clone->cloneChildren();
clone->children.push_back(clone->qualifier);
clone->children.push_back(clone->column_list);
return clone;
}
void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
{
const auto & qualifier = children.at(0);
qualifier->appendColumnName(ostr);
writeCString(".COLUMNS(", ostr);
@ -208,19 +245,10 @@ void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
writeChar(')', ostr);
}
void ASTQualifiedColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const
{
column_list->updateTreeHash(hash_state);
IAST::updateTreeHashImpl(hash_state);
}
void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "");
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
@ -232,14 +260,9 @@ void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings,
}
settings.ostr << ")";
/// Format column transformers
size_t children_size = children.size();
for (size_t i = 1; i < children_size; ++i)
if (transformers)
{
const auto & child = children[i];
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}

View File

@ -24,10 +24,13 @@ public:
void appendColumnName(WriteBuffer & ostr) const override;
void setPattern(String pattern);
const String & getPattern() const;
const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const;
void updateTreeHashImpl(SipHash & hash_state) const override;
ASTPtr expression;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
@ -43,9 +46,10 @@ public:
String getID(char) const override { return "ColumnsListMatcher"; }
ASTPtr clone() const override;
void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override;
ASTPtr expression;
ASTPtr column_list;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
@ -59,10 +63,12 @@ public:
void appendColumnName(WriteBuffer & ostr) const override;
const std::shared_ptr<re2::RE2> & getMatcher() const;
void setPattern(String pattern);
void setPattern(String pattern, bool set_matcher = true);
void setMatcher(std::shared_ptr<re2::RE2> matcher);
void updateTreeHashImpl(SipHash & hash_state) const override;
ASTPtr qualifier;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
@ -78,9 +84,10 @@ public:
String getID(char) const override { return "QualifiedColumnsListMatcher"; }
ASTPtr clone() const override;
void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override;
ASTPtr qualifier;
ASTPtr column_list;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};

View File

@ -19,6 +19,15 @@ namespace ErrorCodes
extern const int CANNOT_COMPILE_REGEXP;
}
void ASTColumnsTransformerList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
for (const auto & child : children)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
}
}
void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes)
{
if (const auto * apply = transformer->as<ASTColumnsApplyTransformer>())

View File

@ -9,6 +9,23 @@ namespace re2
namespace DB
{
/// A list of column transformers
class ASTColumnsTransformerList : public IAST
{
public:
String getID(char) const override { return "ColumnsTransformerList"; }
ASTPtr clone() const override
{
auto clone = std::make_shared<ASTColumnsTransformerList>(*this);
clone->cloneChildren();
return clone;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
class IASTColumnsTransformer : public IAST
{
public:

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/// AST, EXPLAIN or other query with meaning of explanation query instead of execution
class ASTExplainQuery : public ASTQueryWithOutput
@ -23,6 +27,45 @@ public:
CurrentTransaction, /// 'EXPLAIN CURRENT TRANSACTION'
};
static String toString(ExplainKind kind)
{
switch (kind)
{
case ParsedAST: return "EXPLAIN AST";
case AnalyzedSyntax: return "EXPLAIN SYNTAX";
case QueryTree: return "EXPLAIN QUERY TREE";
case QueryPlan: return "EXPLAIN";
case QueryPipeline: return "EXPLAIN PIPELINE";
case QueryEstimates: return "EXPLAIN ESTIMATE";
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
}
UNREACHABLE();
}
static ExplainKind fromString(const String & str)
{
if (str == "EXPLAIN AST")
return ParsedAST;
if (str == "EXPLAIN SYNTAX")
return AnalyzedSyntax;
if (str == "EXPLAIN QUERY TREE")
return QueryTree;
if (str == "EXPLAIN" || str == "EXPLAIN PLAN")
return QueryPlan;
if (str == "EXPLAIN PIPELINE")
return QueryPipeline;
if (str == "EXPLAIN ESTIMATE")
return QueryEstimates;
if (str == "EXPLAIN TABLE OVERRIDE")
return TableOverride;
if (str == "EXPLAIN CURRENT TRANSACTION")
return CurrentTransaction;
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown explain kind '{}'", str);
}
explicit ASTExplainQuery(ExplainKind kind_) : kind(kind_) {}
String getID(char delim) const override { return "Explain" + (delim + toString(kind)); }
@ -103,23 +146,6 @@ private:
/// Used by EXPLAIN TABLE OVERRIDE
ASTPtr table_function;
ASTPtr table_override;
static String toString(ExplainKind kind)
{
switch (kind)
{
case ParsedAST: return "EXPLAIN AST";
case AnalyzedSyntax: return "EXPLAIN SYNTAX";
case QueryTree: return "EXPLAIN QUERY TREE";
case QueryPlan: return "EXPLAIN";
case QueryPipeline: return "EXPLAIN PIPELINE";
case QueryEstimates: return "EXPLAIN ESTIMATE";
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
}
UNREACHABLE();
}
};
}

View File

@ -7,22 +7,18 @@ namespace DB
void ASTQualifiedAsterisk::appendColumnName(WriteBuffer & ostr) const
{
const auto & qualifier = children.at(0);
qualifier->appendColumnName(ostr);
writeCString(".*", ostr);
}
void ASTQualifiedAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".*";
/// Format column transformers
for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it)
if (transformers)
{
settings.ostr << ' ';
(*it)->formatImpl(settings, state, frame);
transformers->formatImpl(settings, state, frame);
}
}

View File

@ -17,11 +17,18 @@ public:
ASTPtr clone() const override
{
auto clone = std::make_shared<ASTQualifiedAsterisk>(*this);
clone->cloneChildren();
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
clone->qualifier = qualifier->clone();
clone->children.push_back(clone->qualifier);
return clone;
}
void appendColumnName(WriteBuffer & ostr) const override;
ASTPtr qualifier;
ASTPtr transformers;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};

View File

@ -28,6 +28,8 @@
#include <Parsers/ASTWindowDefinition.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
@ -116,8 +118,40 @@ bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else if (ASTPtr explain_node; explain.parse(pos, explain_node, expected))
{
/// Replace SELECT * FROM (EXPLAIN SELECT ...) with SELECT * FROM viewExplain(EXPLAIN SELECT ...)
result_node = buildSelectFromTableFunction(makeASTFunction("viewExplain", explain_node));
const auto & explain_query = explain_node->as<const ASTExplainQuery &>();
if (explain_query.getTableFunction() || explain_query.getTableOverride())
throw Exception("EXPLAIN in a subquery cannot have a table function or table override", ErrorCodes::BAD_ARGUMENTS);
/// Replace subquery `(EXPLAIN <kind> <explain_settings> SELECT ...)`
/// with `(SELECT * FROM viewExplain("<kind>", "<explain_settings>", SELECT ...))`
String kind_str = ASTExplainQuery::toString(explain_query.getKind());
String settings_str;
if (ASTPtr settings_ast = explain_query.getSettings())
{
if (!settings_ast->as<ASTSetQuery>())
throw Exception("EXPLAIN settings must be a SET query", ErrorCodes::BAD_ARGUMENTS);
settings_str = queryToString(settings_ast);
}
const ASTPtr & explained_ast = explain_query.getExplainedQuery();
if (explained_ast)
{
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str),
explained_ast);
result_node = buildSelectFromTableFunction(view_explain);
}
else
{
auto view_explain = makeASTFunction("viewExplain",
std::make_shared<ASTLiteral>(kind_str),
std::make_shared<ASTLiteral>(settings_str));
result_node = buildSelectFromTableFunction(view_explain);
}
}
else
{
@ -1623,13 +1657,21 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
++pos;
auto asterisk = std::make_shared<ASTAsterisk>();
auto transformers = std::make_shared<ASTColumnsTransformerList>();
ParserColumnsTransformers transformers_p(allowed_transformers);
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
asterisk->children.push_back(transformer);
transformers->children.push_back(transformer);
}
node = asterisk;
if (!transformers->children.empty())
{
asterisk->transformers = std::move(transformers);
asterisk->children.push_back(asterisk->transformers);
}
node = std::move(asterisk);
return true;
}
return false;
@ -1638,7 +1680,7 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected))
return false;
if (pos->type != TokenType::Dot)
@ -1650,13 +1692,23 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
++pos;
auto res = std::make_shared<ASTQualifiedAsterisk>();
res->children.push_back(node);
auto transformers = std::make_shared<ASTColumnsTransformerList>();
ParserColumnsTransformers transformers_p;
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
res->children.push_back(transformer);
transformers->children.push_back(transformer);
}
res->qualifier = std::move(node);
res->children.push_back(res->qualifier);
if (!transformers->children.empty())
{
res->transformers = std::move(transformers);
res->children.push_back(res->transformers);
}
node = std::move(res);
return true;
}
@ -1680,28 +1732,44 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected
return false;
++pos;
auto transformers = std::make_shared<ASTColumnsTransformerList>();
ParserColumnsTransformers transformers_p(allowed_transformers);
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
transformers->children.push_back(transformer);
}
ASTPtr res;
if (column_list)
{
auto list_matcher = std::make_shared<ASTColumnsListMatcher>();
list_matcher->column_list = column_list;
res = list_matcher;
list_matcher->column_list = std::move(column_list);
list_matcher->children.push_back(list_matcher->column_list);
if (!transformers->children.empty())
{
list_matcher->transformers = std::move(transformers);
list_matcher->children.push_back(list_matcher->transformers);
}
node = std::move(list_matcher);
}
else
{
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
regexp_matcher->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
res = regexp_matcher;
if (!transformers->children.empty())
{
regexp_matcher->transformers = std::move(transformers);
regexp_matcher->children.push_back(regexp_matcher->transformers);
}
node = std::move(regexp_matcher);
}
ParserColumnsTransformers transformers_p(allowed_transformers);
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
res->children.push_back(transformer);
}
node = std::move(res);
return true;
}
@ -1717,29 +1785,19 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected))
return false;
auto identifier_node = node;
const auto & identifier_node_typed = identifier_node->as<ASTTableIdentifier &>();
auto & identifier_node_typed = identifier_node->as<ASTIdentifier &>();
auto & name_parts = identifier_node_typed.name_parts;
/// ParserCompoundIdentifier parse identifier.COLUMNS
if (identifier_node_typed.name_parts.size() == 1 || identifier_node_typed.name_parts.back() != "COLUMNS")
if (name_parts.size() == 1 || name_parts.back() != "COLUMNS")
return false;
/// TODO: ASTTableIdentifier can contain only 2 parts
if (identifier_node_typed.name_parts.size() == 2)
{
auto table_name = identifier_node_typed.name_parts[0];
identifier_node = std::make_shared<ASTTableIdentifier>(table_name);
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected identifier to contain no more than 2 parts. Actual {}",
identifier_node_typed.full_name);
}
name_parts.pop_back();
identifier_node = std::make_shared<ASTIdentifier>(std::move(name_parts), false, std::move(node->children));
if (!parseColumnsMatcherBody(pos, node, expected, allowed_transformers))
return false;
@ -1747,28 +1805,36 @@ bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected
if (auto * columns_list_matcher = node->as<ASTColumnsListMatcher>())
{
auto result = std::make_shared<ASTQualifiedColumnsListMatcher>();
result->qualifier = std::move(identifier_node);
result->column_list = std::move(columns_list_matcher->column_list);
result->children.reserve(columns_list_matcher->children.size() + 1);
result->children.push_back(std::move(identifier_node));
result->children.push_back(result->qualifier);
result->children.push_back(result->column_list);
for (auto && child : columns_list_matcher->children)
result->children.push_back(std::move(child));
if (columns_list_matcher->transformers)
{
result->transformers = std::move(columns_list_matcher->transformers);
result->children.push_back(result->transformers);
}
node = result;
node = std::move(result);
}
else if (auto * column_regexp_matcher = node->as<ASTColumnsRegexpMatcher>())
{
auto result = std::make_shared<ASTQualifiedColumnsRegexpMatcher>();
result->setPattern(column_regexp_matcher->getPattern(), false);
result->setMatcher(column_regexp_matcher->getMatcher());
result->children.reserve(column_regexp_matcher->children.size() + 1);
result->children.push_back(std::move(identifier_node));
result->qualifier = std::move(identifier_node);
result->children.push_back(result->qualifier);
for (auto && child : column_regexp_matcher->children)
result->children.push_back(std::move(child));
if (column_regexp_matcher->transformers)
{
result->transformers = std::move(column_regexp_matcher->transformers);
result->children.push_back(result->transformers);
}
node = result;
node = std::move(result);
}
else
{

View File

@ -4,6 +4,7 @@
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
@ -2194,7 +2195,7 @@ struct ParserExpressionImpl
using Layers = std::vector<std::unique_ptr<Layer>>;
Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected);
static Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
};
@ -2523,8 +2524,6 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected)
{
ASTPtr tmp;
/// ParserExpression can be called in this part of the query:
/// ALTER TABLE partition_all2 CLEAR INDEX [ p ] IN PARTITION ALL
///
@ -2544,17 +2543,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
if (cur_op == operators_table.end())
{
ASTPtr alias;
ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword);
auto old_pos = pos;
if (layers.back()->allow_alias &&
!layers.back()->parsed_alias &&
alias_parser.parse(pos, tmp, expected) &&
layers.back()->insertAlias(tmp))
alias_parser.parse(pos, alias, expected) &&
layers.back()->insertAlias(alias))
{
layers.back()->parsed_alias = true;
return Action::OPERATOR;
}
pos = old_pos;
return Action::NONE;
}
@ -2618,33 +2617,57 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
layers.back()->pushOperand(function);
}
/// Dot (TupleElement operator) can be a beginning of a .* or .COLUMNS expressions
if (op.type == OperatorType::TupleElement)
{
ASTPtr tmp;
if (asterisk_parser.parse(pos, tmp, expected) ||
columns_matcher_parser.parse(pos, tmp, expected))
{
if (auto * asterisk = tmp->as<ASTAsterisk>())
{
if (!layers.back()->popOperand(asterisk->expression))
return Action::NONE;
}
else if (auto * columns_list_matcher = tmp->as<ASTColumnsListMatcher>())
{
if (!layers.back()->popOperand(columns_list_matcher->expression))
return Action::NONE;
}
else if (auto * columns_regexp_matcher = tmp->as<ASTColumnsRegexpMatcher>())
{
if (!layers.back()->popOperand(columns_regexp_matcher->expression))
return Action::NONE;
}
layers.back()->pushOperand(std::move(tmp));
return Action::OPERATOR;
}
}
layers.back()->pushOperator(op);
if (op.type == OperatorType::ArrayElement)
layers.push_back(std::make_unique<ArrayElementLayer>());
Action next = Action::OPERAND;
/// isNull & isNotNull are postfix unary operators
if (op.type == OperatorType::IsNull)
next = Action::OPERATOR;
if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween)
layers.back()->between_counter++;
return Action::OPERATOR;
if (op.type == OperatorType::Cast)
{
next = Action::OPERATOR;
ASTPtr type_ast;
if (!ParserDataType().parse(pos, type_ast, expected))
return Action::NONE;
layers.back()->pushOperand(std::make_shared<ASTLiteral>(queryToString(type_ast)));
return Action::OPERATOR;
}
return next;
if (op.type == OperatorType::ArrayElement)
layers.push_back(std::make_unique<ArrayElementLayer>());
if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween)
layers.back()->between_counter++;
return Action::OPERAND;
}
}

View File

@ -60,6 +60,7 @@ namespace ErrorCodes
extern const int TOO_MANY_PARTITIONS;
extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int LOGICAL_ERROR;
}
@ -365,18 +366,22 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(
const std::string & relative_path_,
ConnectionPoolPtr pool_,
ActionBlocker & monitor_blocker_,
BackgroundSchedulePool & bg_pool)
BackgroundSchedulePool & bg_pool,
bool initialize_from_disk)
: storage(storage_)
, pool(std::move(pool_))
, disk(disk_)
, relative_path(relative_path_)
, path(fs::path(disk->getPath()) / relative_path / "")
, broken_relative_path(fs::path(relative_path) / "broken")
, broken_path(fs::path(path) / "broken" / "")
, should_batch_inserts(storage.getDistributedSettingsRef().monitor_batch_inserts)
, split_batch_on_failure(storage.getDistributedSettingsRef().monitor_split_batch_on_failure)
, dir_fsync(storage.getDistributedSettingsRef().fsync_directories)
, min_batched_block_size_rows(storage.getContext()->getSettingsRef().min_insert_block_size_rows)
, min_batched_block_size_bytes(storage.getContext()->getSettingsRef().min_insert_block_size_bytes)
, current_batch_file_path(path + "current_batch.txt")
, pending_files(std::numeric_limits<size_t>::max())
, default_sleep_time(storage.getDistributedSettingsRef().monitor_sleep_time_ms.totalMilliseconds())
, sleep_time(default_sleep_time)
, max_sleep_time(storage.getDistributedSettingsRef().monitor_max_sleep_time_ms.totalMilliseconds())
@ -385,6 +390,11 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(
, metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0)
, metric_broken_files(CurrentMetrics::BrokenDistributedFilesToInsert, 0)
{
fs::create_directory(broken_path);
if (initialize_from_disk)
initializeFilesFromDisk();
task_handle = bg_pool.createTask(getLoggerName() + "/Bg", [this]{ run(); });
task_handle->activateAndSchedule();
}
@ -392,35 +402,29 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(
StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor()
{
if (!quit)
if (!pending_files.isFinished())
{
quit = true;
pending_files.clearAndFinish();
task_handle->deactivate();
}
}
void StorageDistributedDirectoryMonitor::flushAllData()
{
if (quit)
if (pending_files.isFinished())
return;
std::lock_guard lock{mutex};
const auto & files = getFiles();
if (!files.empty())
{
processFiles(files);
/// Update counters.
getFiles();
}
if (!hasPendingFiles())
return;
processFiles();
}
void StorageDistributedDirectoryMonitor::shutdownAndDropAllData()
{
if (!quit)
if (!pending_files.isFinished())
{
quit = true;
pending_files.clearAndFinish();
task_handle->deactivate();
}
@ -434,19 +438,21 @@ void StorageDistributedDirectoryMonitor::run()
std::lock_guard lock{mutex};
bool do_sleep = false;
while (!quit)
while (!pending_files.isFinished())
{
do_sleep = true;
const auto & files = getFiles();
if (files.empty())
if (!hasPendingFiles())
break;
if (!monitor_blocker.isCancelled())
{
try
{
do_sleep = !processFiles(files);
processFiles();
/// No errors while processing existing files.
/// Let's see maybe there are more files to process.
do_sleep = false;
std::lock_guard status_lock(status_mutex);
status.last_exception = std::exception_ptr{};
@ -470,9 +476,7 @@ void StorageDistributedDirectoryMonitor::run()
}
}
else
{
LOG_DEBUG(log, "Skipping send data over distributed table.");
}
const auto now = std::chrono::system_clock::now();
if (now - last_decrease_time > decrease_error_count_period)
@ -487,10 +491,7 @@ void StorageDistributedDirectoryMonitor::run()
break;
}
/// Update counters.
getFiles();
if (!quit && do_sleep)
if (!pending_files.isFinished() && do_sleep)
task_handle->scheduleAfter(sleep_time.count());
}
@ -568,41 +569,83 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
settings.distributed_replica_error_cap);
}
std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles()
bool StorageDistributedDirectoryMonitor::hasPendingFiles() const
{
std::map<UInt64, std::string> files;
return fs::exists(current_batch_file_path) || !current_batch_file.empty() || !pending_files.empty();
}
void StorageDistributedDirectoryMonitor::initializeFilesFromDisk()
{
/// NOTE: This method does not requires to hold status_mutex, hence, no TSA
/// annotations in the header file.
fs::directory_iterator end;
for (fs::directory_iterator it{path}; it != end; ++it)
/// Initialize pending files
{
const auto & file_path_str = it->path();
if (!it->is_directory() && startsWith(fs::path(file_path_str).extension(), ".bin"))
size_t bytes_count = 0;
for (fs::directory_iterator it{path}; it != end; ++it)
{
files[parse<UInt64>(fs::path(file_path_str).stem())] = file_path_str;
const auto & file_path = it->path();
const auto & base_name = file_path.stem().string();
if (!it->is_directory() && startsWith(fs::path(file_path).extension(), ".bin") && parse<UInt64>(base_name))
{
const std::string & file_path_str = file_path.string();
if (!pending_files.push(file_path_str))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file");
bytes_count += fs::file_size(file_path);
}
else if (base_name != "tmp" && base_name != "broken")
{
/// It is OK to log current_batch.txt here too (useful for debugging).
LOG_WARNING(log, "Unexpected file {} in {}", file_path.string(), path);
}
}
LOG_TRACE(log, "Files set to {}", pending_files.size());
LOG_TRACE(log, "Bytes set to {}", bytes_count);
metric_pending_files.changeTo(pending_files.size());
status.files_count = pending_files.size();
status.bytes_count = bytes_count;
}
return files;
/// Initialize broken files
{
size_t broken_bytes_count = 0;
size_t broken_files = 0;
for (fs::directory_iterator it{broken_path}; it != end; ++it)
{
const auto & file_path = it->path();
if (!it->is_directory() && startsWith(fs::path(file_path).extension(), ".bin") && parse<UInt64>(file_path.stem()))
broken_bytes_count += fs::file_size(file_path);
else
LOG_WARNING(log, "Unexpected file {} in {}", file_path.string(), broken_path);
}
LOG_TRACE(log, "Broken files set to {}", broken_files);
LOG_TRACE(log, "Broken bytes set to {}", broken_bytes_count);
metric_broken_files.changeTo(broken_files);
status.broken_files_count = broken_files;
status.broken_bytes_count = broken_bytes_count;
}
}
bool StorageDistributedDirectoryMonitor::processFiles(const std::map<UInt64, std::string> & files)
void StorageDistributedDirectoryMonitor::processFiles()
{
if (should_batch_inserts)
{
processFilesWithBatching(files);
}
processFilesWithBatching();
else
{
for (const auto & file : files)
{
if (quit)
return true;
/// Process unprocessed file.
if (!current_batch_file.empty())
processFile(current_batch_file);
processFile(file.second);
}
while (pending_files.tryPop(current_batch_file))
processFile(current_batch_file);
}
return true;
}
void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path)
@ -649,7 +692,11 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa
thread_trace_context->root_span.addAttribute(std::current_exception());
e.addMessage(fmt::format("While sending {}", file_path));
maybeMarkAsBroken(file_path, e);
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
markAsBroken(file_path);
current_batch_file.clear();
}
throw;
}
catch (...)
@ -662,6 +709,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa
auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path);
markAsSend(file_path);
current_batch_file.clear();
LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds());
}
@ -701,23 +749,19 @@ struct StorageDistributedDirectoryMonitor::BatchHeader
struct StorageDistributedDirectoryMonitor::Batch
{
std::vector<UInt64> file_indices;
size_t total_rows = 0;
size_t total_bytes = 0;
bool recovered = false;
StorageDistributedDirectoryMonitor & parent;
const std::map<UInt64, String> & file_index_to_path;
std::vector<std::string> files;
bool split_batch_on_failure = true;
bool fsync = false;
bool dir_fsync = false;
Batch(
StorageDistributedDirectoryMonitor & parent_,
const std::map<UInt64, String> & file_index_to_path_)
explicit Batch(StorageDistributedDirectoryMonitor & parent_)
: parent(parent_)
, file_index_to_path(file_index_to_path_)
, split_batch_on_failure(parent.split_batch_on_failure)
, fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert)
, dir_fsync(parent.dir_fsync)
@ -732,7 +776,7 @@ struct StorageDistributedDirectoryMonitor::Batch
void send()
{
if (file_indices.empty())
if (files.empty())
return;
CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend};
@ -775,7 +819,7 @@ struct StorageDistributedDirectoryMonitor::Batch
}
catch (const Exception & e)
{
if (split_batch_on_failure && file_indices.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException()))
if (split_batch_on_failure && files.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException()))
{
tryLogCurrentException(parent.log, "Trying to split batch due to");
sendSeparateFiles();
@ -795,44 +839,28 @@ struct StorageDistributedDirectoryMonitor::Batch
}
else
{
std::vector<std::string> files;
for (const auto && file_info : file_index_to_path | boost::adaptors::indexed())
{
if (file_info.index() > 8)
{
files.push_back("...");
break;
}
files.push_back(file_info.value().second);
}
e.addMessage(fmt::format("While sending batch, nums: {}, files: {}", file_index_to_path.size(), fmt::join(files, "\n")));
e.addMessage(fmt::format("While sending a batch of {} files, files: {}", files.size(), fmt::join(files, "\n")));
throw;
}
}
if (!batch_broken)
{
LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", file_indices.size(), watch.elapsedMilliseconds());
LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", files.size(), watch.elapsedMilliseconds());
auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path);
for (UInt64 file_index : file_indices)
parent.markAsSend(file_index_to_path.at(file_index));
for (const auto & file : files)
parent.markAsSend(file);
}
else if (!batch_marked_as_broken)
{
LOG_ERROR(parent.log, "Marking a batch of {} files as broken.", file_indices.size());
LOG_ERROR(parent.log, "Marking a batch of {} files as broken, files: {}", files.size(), fmt::join(files, "\n"));
for (UInt64 file_idx : file_indices)
{
auto file_path = file_index_to_path.find(file_idx);
if (file_path != file_index_to_path.end())
parent.markAsBroken(file_path->second);
}
for (const auto & file : files)
parent.markAsBroken(file);
}
file_indices.clear();
files.clear();
total_rows = 0;
total_bytes = 0;
recovered = false;
@ -842,8 +870,11 @@ struct StorageDistributedDirectoryMonitor::Batch
void writeText(WriteBuffer & out)
{
for (UInt64 file_idx : file_indices)
out << file_idx << '\n';
for (const auto & file : files)
{
UInt64 file_index = parse<UInt64>(fs::path(file).stem());
out << file_index << '\n';
}
}
void readText(ReadBuffer & in)
@ -852,8 +883,9 @@ struct StorageDistributedDirectoryMonitor::Batch
{
UInt64 idx;
in >> idx >> "\n";
file_indices.push_back(idx);
files.push_back(fmt::format("{}/{}.bin", parent.path, idx));
}
recovered = true;
}
@ -865,14 +897,9 @@ private:
IConnectionPool::Entry connection;
for (UInt64 file_idx : file_indices)
for (const auto & file : files)
{
auto file_path = file_index_to_path.find(file_idx);
if (file_path == file_index_to_path.end())
throw Exception(ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO,
"Failed to send batch: file with index {} is absent", file_idx);
ReadBufferFromFile in(file_path->second);
ReadBufferFromFile in(file);
const auto & distributed_header = readDistributedHeader(in, parent.log);
OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__,
@ -886,7 +913,7 @@ private:
compression_expected = connection->getCompression() == Protocol::Compression::Enable;
LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).",
file_indices.size(),
files.size(),
connection->getDescription(),
formatReadableQuantity(total_rows),
formatReadableSizeWithBinarySuffix(total_bytes));
@ -907,19 +934,11 @@ private:
{
size_t broken_files = 0;
for (UInt64 file_idx : file_indices)
for (const auto & file : files)
{
auto file_path = file_index_to_path.find(file_idx);
if (file_path == file_index_to_path.end())
{
LOG_ERROR(parent.log, "Failed to send one file from batch: file with index {} is absent", file_idx);
++broken_files;
continue;
}
try
{
ReadBufferFromFile in(file_path->second);
ReadBufferFromFile in(file);
const auto & distributed_header = readDistributedHeader(in, parent.log);
// this function is called in a separated thread, so we set up the trace context from the file
@ -941,9 +960,11 @@ private:
}
catch (Exception & e)
{
e.addMessage(fmt::format("While sending {}", file_path->second));
parent.maybeMarkAsBroken(file_path->second, e);
++broken_files;
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
parent.markAsBroken(file);
++broken_files;
}
}
}
@ -1023,13 +1044,18 @@ std::shared_ptr<ISource> StorageDistributedDirectoryMonitor::createSourceFromFil
return std::make_shared<DirectoryMonitorSource>(file_name);
}
bool StorageDistributedDirectoryMonitor::addAndSchedule(size_t file_size, size_t ms)
bool StorageDistributedDirectoryMonitor::addAndSchedule(const std::string & file_path, size_t file_size, size_t ms)
{
if (quit)
/// NOTE: It is better not to throw in this case, since the file is already
/// on disk (see DistributedSink), and it will be processed next time.
if (pending_files.isFinished())
return false;
if (!pending_files.push(file_path))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file");
{
std::lock_guard status_lock(status_mutex);
std::lock_guard lock(status_mutex);
metric_pending_files.add();
status.bytes_count += file_size;
++status.files_count;
@ -1045,33 +1071,25 @@ StorageDistributedDirectoryMonitor::Status StorageDistributedDirectoryMonitor::g
return current_status;
}
void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map<UInt64, std::string> & files)
void StorageDistributedDirectoryMonitor::processFilesWithBatching()
{
std::unordered_set<UInt64> file_indices_to_skip;
/// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch.
if (fs::exists(current_batch_file_path))
{
/// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch.
Batch batch(*this, files);
Batch batch(*this);
ReadBufferFromFile in{current_batch_file_path};
batch.readText(in);
file_indices_to_skip.insert(batch.file_indices.begin(), batch.file_indices.end());
batch.send();
auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path);
fs::remove(current_batch_file_path);
}
std::unordered_map<BatchHeader, Batch, BatchHeader::Hash> header_to_batch;
for (const auto & file : files)
std::string file_path;
while (pending_files.tryPop(file_path))
{
if (quit)
return;
UInt64 file_idx = file.first;
const String & file_path = file.second;
if (file_indices_to_skip.contains(file_idx))
continue;
size_t total_rows = 0;
size_t total_bytes = 0;
Block header;
@ -1110,8 +1128,9 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
}
catch (const Exception & e)
{
if (maybeMarkAsBroken(file_path, e))
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
markAsBroken(file_path);
tryLogCurrentException(log, "File is marked broken due to");
continue;
}
@ -1125,9 +1144,9 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
std::move(distributed_header.client_info),
std::move(header)
);
Batch & batch = header_to_batch.try_emplace(batch_header, *this, files).first->second;
Batch & batch = header_to_batch.try_emplace(batch_header, *this).first->second;
batch.file_indices.push_back(file_idx);
batch.files.push_back(file_path);
batch.total_rows += total_rows;
batch.total_bytes += total_bytes;
@ -1155,16 +1174,10 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_path)
{
const auto last_path_separator_pos = file_path.rfind('/');
const auto & base_path = file_path.substr(0, last_path_separator_pos + 1);
const auto & file_name = file_path.substr(last_path_separator_pos + 1);
const String & broken_path = fs::path(base_path) / "broken/";
const String & broken_file_path = fs::path(broken_path) / file_name;
fs::create_directory(broken_path);
const String & broken_file_path = fs::path(broken_path) / fs::path(file_path).filename();
auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path);
auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, fs::path(relative_path) / "broken/");
auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, broken_relative_path);
{
std::lock_guard status_lock(status_mutex);
@ -1198,21 +1211,9 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat
fs::remove(file_path);
}
bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & file_path, const Exception & e)
{
/// Mark file as broken if necessary.
if (isFileBrokenErrorCode(e.code(), e.isRemoteException()))
{
markAsBroken(file_path);
return true;
}
else
return false;
}
std::string StorageDistributedDirectoryMonitor::getLoggerName() const
{
return storage.getStorageID().getFullTableName() + ".DirectoryMonitor";
return storage.getStorageID().getFullTableName() + ".DirectoryMonitor." + disk->getName();
}
void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_relative_path)

View File

@ -1,6 +1,7 @@
#pragma once
#include <Core/BackgroundSchedulePool.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Client/ConnectionPool.h>
#include <atomic>
@ -38,7 +39,8 @@ public:
const std::string & relative_path_,
ConnectionPoolPtr pool_,
ActionBlocker & monitor_blocker_,
BackgroundSchedulePool & bg_pool);
BackgroundSchedulePool & bg_pool,
bool initialize_from_disk);
~StorageDistributedDirectoryMonitor();
@ -53,7 +55,7 @@ public:
static std::shared_ptr<ISource> createSourceFromFile(const String & file_name);
/// For scheduling via DistributedSink.
bool addAndSchedule(size_t file_size, size_t ms);
bool addAndSchedule(const std::string & file_path, size_t file_size, size_t ms);
struct InternalStatus
{
@ -78,14 +80,15 @@ public:
private:
void run();
std::map<UInt64, std::string> getFiles();
bool processFiles(const std::map<UInt64, std::string> & files);
bool hasPendingFiles() const;
void initializeFilesFromDisk();
void processFiles();
void processFile(const std::string & file_path);
void processFilesWithBatching(const std::map<UInt64, std::string> & files);
void processFilesWithBatching();
void markAsBroken(const std::string & file_path);
void markAsSend(const std::string & file_path);
bool maybeMarkAsBroken(const std::string & file_path, const Exception & e);
std::string getLoggerName() const;
@ -95,25 +98,33 @@ private:
DiskPtr disk;
std::string relative_path;
std::string path;
std::string broken_relative_path;
std::string broken_path;
const bool should_batch_inserts = false;
const bool split_batch_on_failure = true;
const bool dir_fsync = false;
const size_t min_batched_block_size_rows = 0;
const size_t min_batched_block_size_bytes = 0;
String current_batch_file_path;
/// This is pending data (due to some error) for should_batch_inserts==true
std::string current_batch_file_path;
/// This is pending data (due to some error) for should_batch_inserts==false
std::string current_batch_file;
struct BatchHeader;
struct Batch;
std::mutex status_mutex;
InternalStatus status;
ConcurrentBoundedQueue<std::string> pending_files;
const std::chrono::milliseconds default_sleep_time;
std::chrono::milliseconds sleep_time;
const std::chrono::milliseconds max_sleep_time;
std::chrono::time_point<std::chrono::system_clock> last_decrease_time {std::chrono::system_clock::now()};
std::atomic<bool> quit {false};
std::mutex mutex;
Poco::Logger * log;
ActionBlocker & monitor_blocker;

View File

@ -724,6 +724,9 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
return guard;
};
std::vector<std::string> bin_files;
bin_files.reserve(dir_names.size());
auto it = dir_names.begin();
/// on first iteration write block to a temporary directory for subsequent
/// hardlinking to ensure the inode is not freed until we're done
@ -802,8 +805,8 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
}
// Create hardlink here to reuse increment number
const std::string block_file_path(fs::path(path) / file_name);
createHardLink(first_file_tmp_path, block_file_path);
bin_files.push_back(fs::path(path) / file_name);
createHardLink(first_file_tmp_path, bin_files.back());
auto dir_sync_guard = make_directory_sync_guard(*it);
}
++it;
@ -814,8 +817,8 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
const std::string path(fs::path(disk_path) / (data_path + *it));
fs::create_directory(path);
const std::string block_file_path(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin"));
createHardLink(first_file_tmp_path, block_file_path);
bin_files.push_back(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin"));
createHardLink(first_file_tmp_path, bin_files.back());
auto dir_sync_guard = make_directory_sync_guard(*it);
}
@ -826,10 +829,13 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
/// Notify
auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms;
for (const auto & dir_name : dir_names)
for (size_t i = 0; i < dir_names.size(); ++i)
{
const auto & dir_name = dir_names[i];
const auto & bin_file = bin_files[i];
auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name, /* startup= */ false);
directory_monitor.addAndSchedule(file_size, sleep_ms.totalMilliseconds());
directory_monitor.addAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds());
}
}

View File

@ -193,7 +193,8 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned.
{
result.emplace_back(Block(block), Row{});
result[0].offsets = chunk_offsets;
if (chunk_offsets != nullptr)
result[0].offsets = std::move(chunk_offsets->offsets);
return result;
}
@ -230,7 +231,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
/// do not interfere with possible calculated primary key columns of the same name.
result.emplace_back(Block(block), get_partition(0));
if (!chunk_offsets_with_partition.empty())
result[0].offsets = chunk_offsets_with_partition[0];
result[0].offsets = std::move(chunk_offsets_with_partition[0]->offsets);
return result;
}
@ -245,7 +246,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
}
for (size_t i = 0; i < chunk_offsets_with_partition.size(); ++i)
result[i].offsets = chunk_offsets_with_partition[i];
result[i].offsets = std::move(chunk_offsets_with_partition[i]->offsets);
return result;
}

View File

@ -22,15 +22,15 @@ struct BlockWithPartition
{
Block block;
Row partition;
ChunkOffsetsPtr offsets;
std::vector<size_t> offsets;
BlockWithPartition(Block && block_, Row && partition_)
: block(block_), partition(std::move(partition_))
{
}
BlockWithPartition(Block && block_, Row && partition_, ChunkOffsetsPtr chunk_offsets_)
: block(block_), partition(std::move(partition_)), offsets(chunk_offsets_)
BlockWithPartition(Block && block_, Row && partition_, std::vector<size_t> && offsets_)
: block(block_), partition(std::move(partition_)), offsets(std::move(offsets_))
{
}
};

View File

@ -41,15 +41,17 @@ struct ReplicatedMergeTreeSinkImpl<async_insert>::DelayedChunk
{
struct Partition
{
Poco::Logger * log;
MergeTreeDataWriter::TemporaryPart temp_part;
UInt64 elapsed_ns;
BlockIDsType block_id;
BlockWithPartition block_with_partition;
std::unordered_map<String, size_t> block_id_to_offset_idx;
std::unordered_map<String, std::vector<size_t>> block_id_to_offset_idx;
Partition() = default;
Partition(MergeTreeDataWriter::TemporaryPart && temp_part_, UInt64 elapsed_ns_, BlockIDsType && block_id_, BlockWithPartition && block_)
: temp_part(std::move(temp_part_)),
Partition(Poco::Logger * log_, MergeTreeDataWriter::TemporaryPart && temp_part_, UInt64 elapsed_ns_, BlockIDsType && block_id_, BlockWithPartition && block_)
: log(log_),
temp_part(std::move(temp_part_)),
elapsed_ns(elapsed_ns_),
block_id(std::move(block_id_)),
block_with_partition(std::move(block_))
@ -64,11 +66,105 @@ struct ReplicatedMergeTreeSinkImpl<async_insert>::DelayedChunk
block_id_to_offset_idx.clear();
for (size_t i = 0; i < block_id.size(); ++i)
{
block_id_to_offset_idx[block_id[i]] = i;
block_id_to_offset_idx[block_id[i]].push_back(i);
}
}
}
/// this function check if the block contains duplicate inserts.
/// if so, we keep only one insert for every duplicate ones.
bool filterSelfDuplicate()
{
if constexpr (async_insert)
{
std::vector<String> dup_block_ids;
for (const auto & [hash_id, offset_indexes] : block_id_to_offset_idx)
{
/// It means more than one inserts have the same hash id, in this case, we should keep only one of them.
if (offset_indexes.size() > 1)
dup_block_ids.push_back(hash_id);
}
if (dup_block_ids.empty())
return false;
filterBlockDuplicate(dup_block_ids, true);
return true;
}
return false;
}
/// remove the conflict parts of block for rewriting again.
void filterBlockDuplicate(const std::vector<String> & block_paths, bool self_dedup)
{
if constexpr (async_insert)
{
std::vector<size_t> offset_idx;
for (const auto & raw_path : block_paths)
{
std::filesystem::path p(raw_path);
String conflict_block_id = p.filename();
auto it = block_id_to_offset_idx.find(conflict_block_id);
if (it == block_id_to_offset_idx.end())
throw Exception("Unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR);
/// if this filter is for self_dedup, that means the block paths is selected by `filterSelfDuplicate`, which is a self purge.
/// in this case, we don't know if zk has this insert, then we should keep one insert, to avoid missing this insert.
offset_idx.insert(std::end(offset_idx), std::begin(it->second) + self_dedup, std::end(it->second));
}
std::sort(offset_idx.begin(), offset_idx.end());
auto & offsets = block_with_partition.offsets;
size_t idx = 0, remove_count = 0;
auto it = offset_idx.begin();
std::vector<size_t> new_offsets;
std::vector<String> new_block_ids;
/// construct filter
size_t rows = block_with_partition.block.rows();
auto filter_col = ColumnUInt8::create(rows, 1u);
ColumnUInt8::Container & vec = filter_col->getData();
UInt8 * pos = vec.data();
for (auto & offset : offsets)
{
if (it != offset_idx.end() && *it == idx)
{
size_t start_pos = idx > 0 ? offsets[idx - 1] : 0;
size_t end_pos = offset;
remove_count += end_pos - start_pos;
while (start_pos < end_pos)
{
*(pos + start_pos) = 0;
start_pos++;
}
it++;
}
else
{
new_offsets.push_back(offset - remove_count);
new_block_ids.push_back(block_id[idx]);
}
idx++;
}
LOG_TRACE(log, "New block IDs: {}, new offsets: {}, size: {}", toString(new_block_ids), toString(new_offsets), new_offsets.size());
block_with_partition.offsets = std::move(new_offsets);
block_id = std::move(new_block_ids);
auto cols = block_with_partition.block.getColumns();
for (auto & col : cols)
{
col = col->filter(vec, rows - remove_count);
}
block_with_partition.block.setColumns(cols);
LOG_TRACE(log, "New block rows {}", block_with_partition.block.rows());
initBlockIDMap();
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "sync insert should not call rewriteBlock");
}
}
};
DelayedChunk() = default;
@ -79,6 +175,30 @@ struct ReplicatedMergeTreeSinkImpl<async_insert>::DelayedChunk
std::vector<Partition> partitions;
};
std::vector<Int64> testSelfDeduplicate(std::vector<Int64> data, std::vector<size_t> offsets, std::vector<String> hashes)
{
MutableColumnPtr column = DataTypeInt64().createColumn();
for (auto datum : data)
{
column->insert(datum);
}
Block block({ColumnWithTypeAndName(std::move(column), DataTypePtr(new DataTypeInt64()), "a")});
BlockWithPartition block1(std::move(block), Row(), std::move(offsets));
ReplicatedMergeTreeSinkImpl<true>::DelayedChunk::Partition part(
&Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1));
part.filterSelfDuplicate();
ColumnPtr col = part.block_with_partition.block.getColumns()[0];
std::vector<Int64> result;
for (size_t i = 0; i < col->size(); i++)
{
result.push_back(col->getInt(i));
}
return result;
}
namespace
{
/// Convert block id vector to string. Output at most 50 ids.
@ -90,76 +210,12 @@ namespace
return fmt::format("({})", fmt::join(vec.begin(), vec.begin() + size, ","));
}
/// remove the conflict parts of block for rewriting again.
void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSinkImpl<true>::DelayedChunk::Partition & partition, const std::vector<String> & block_paths)
{
std::vector<size_t> offset_idx;
for (const auto & raw_path : block_paths)
{
std::filesystem::path p(raw_path);
String conflict_block_id = p.filename();
auto it = partition.block_id_to_offset_idx.find(conflict_block_id);
if (it == partition.block_id_to_offset_idx.end())
throw Exception("Unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR);
offset_idx.push_back(it->second);
}
std::sort(offset_idx.begin(), offset_idx.end());
auto & offsets = partition.block_with_partition.offsets->offsets;
size_t idx = 0, remove_count = 0;
auto it = offset_idx.begin();
std::vector<size_t> new_offsets;
std::vector<String> new_block_ids;
/// construct filter
size_t rows = partition.block_with_partition.block.rows();
auto filter_col = ColumnUInt8::create(rows, 1u);
ColumnUInt8::Container & vec = filter_col->getData();
UInt8 * pos = vec.data();
for (auto & offset : offsets)
{
if (it != offset_idx.end() && *it == idx)
{
size_t start_pos = idx > 0 ? offsets[idx - 1] : 0;
size_t end_pos = offset;
remove_count += end_pos - start_pos;
while (start_pos < end_pos)
{
*(pos + start_pos) = 0;
start_pos ++;
}
it++;
}
else
{
new_offsets.push_back(offset - remove_count);
new_block_ids.push_back(partition.block_id[idx]);
}
idx++;
}
LOG_TRACE(log, "New block IDs: {}, new offsets: {}, size: {}", toString(new_block_ids), toString(new_offsets), new_offsets.size());
offsets = std::move(new_offsets);
partition.block_id = std::move(new_block_ids);
auto cols = partition.block_with_partition.block.getColumns();
for (auto & col : cols)
{
col = col->filter(vec, rows - remove_count);
}
partition.block_with_partition.block.setColumns(cols);
LOG_TRACE(log, "New block rows {}", partition.block_with_partition.block.rows());
partition.initBlockIDMap();
}
std::vector<String> getHashesForBlocks(BlockWithPartition & block, String partition_id)
{
size_t start = 0;
auto cols = block.block.getColumns();
std::vector<String> block_id_vec;
for (auto offset : block.offsets->offsets)
for (auto offset : block.offsets)
{
SipHash hash;
for (size_t i = start; i < offset; ++i)
@ -369,7 +425,7 @@ void ReplicatedMergeTreeSinkImpl<async_insert>::consume(Chunk chunk)
{
/// TODO consider insert_deduplication_token
block_id = getHashesForBlocks(current_block, temp_part.part->info.partition_id);
LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets->offsets), current_block.offsets->offsets.size());
LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets), current_block.offsets.size());
}
else if (deduplicate)
{
@ -416,6 +472,7 @@ void ReplicatedMergeTreeSinkImpl<async_insert>::consume(Chunk chunk)
}
partitions.emplace_back(DelayedPartition(
log,
std::move(temp_part),
elapsed_ns,
std::move(block_id),
@ -479,6 +536,14 @@ void ReplicatedMergeTreeSinkImpl<true>::finishDelayedChunk(const ZooKeeperWithFa
for (auto & partition: delayed_chunk->partitions)
{
int retry_times = 0;
/// users may have lots of same inserts. It will be helpful to deduplicate in advance.
if (partition.filterSelfDuplicate())
{
LOG_TRACE(log, "found duplicated inserts in the block");
partition.block_with_partition.partition = std::move(partition.temp_part.part->partition.value);
partition.temp_part = storage.writer.writeTempPart(partition.block_with_partition, metadata_snapshot, context);
}
while (true)
{
partition.temp_part.finalize();
@ -488,7 +553,7 @@ void ReplicatedMergeTreeSinkImpl<true>::finishDelayedChunk(const ZooKeeperWithFa
++retry_times;
LOG_DEBUG(log, "Found duplicate block IDs: {}, retry times {}", toString(conflict_block_ids), retry_times);
/// partition clean conflict
rewriteBlock(log, partition, conflict_block_ids);
partition.filterBlockDuplicate(conflict_block_ids, false);
if (partition.block_id.empty())
break;
partition.block_with_partition.partition = std::move(partition.temp_part.part->partition.value);

View File

@ -1,3 +1,4 @@
#include "Storages/MergeTree/ReplicatedMergeTreeSink.h"
#include "config.h"
#include <gtest/gtest.h>
@ -42,4 +43,21 @@ TEST(AsyncInsertsTest, testScatterOffsetsBySelector)
test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}});
}
std::vector<Int64> testSelfDeduplicate(std::vector<Int64> data, std::vector<size_t> offsets, std::vector<String> hashes);
TEST(AsyncInsertsTest, testSelfDeduplicate)
{
auto test_impl = [](std::vector<Int64> data, std::vector<size_t> offsets, std::vector<String> hashes, std::vector<Int64> answer)
{
auto result = testSelfDeduplicate(data, offsets, hashes);
ASSERT_EQ(answer.size(), result.size());
for (size_t i = 0; i < result.size(); i++)
ASSERT_EQ(answer[i], result[i]);
};
test_impl({1,2,3,1,2,3,4,5,6,1,2,3},{3,6,9,12},{"a","a","b","a"},{1,2,3,4,5,6});
test_impl({1,2,3,1,2,3,1,2,3,1,2,3},{2,3,5,6,8,9,11,12},{"a","b","a","b","a","b","a","b"},{1,2,3});
test_impl({1,2,3,1,2,4,1,2,5,1,2},{2,3,5,6,8,9,11},{"a","b","a","c","a","d","a"},{1,2,3,4,5});
test_impl({1,2,1,2,1,2,1,2,1,2},{2,4,6,8,10},{"a","a","a","a","a"},{1,2});
}
}

View File

@ -1205,12 +1205,15 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk)
const auto & dir_path = it->path();
if (std::filesystem::is_directory(dir_path))
{
/// Created by DistributedSink
const auto & tmp_path = dir_path / "tmp";
/// "tmp" created by DistributedSink
if (std::filesystem::is_directory(tmp_path) && std::filesystem::is_empty(tmp_path))
std::filesystem::remove(tmp_path);
const auto & broken_path = dir_path / "broken";
if (std::filesystem::is_directory(broken_path) && std::filesystem::is_empty(broken_path))
std::filesystem::remove(broken_path);
if (std::filesystem::is_empty(dir_path))
{
LOG_DEBUG(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string());
@ -1239,7 +1242,8 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(
*this, disk, relative_data_path + name,
data.connection_pool,
monitors_blocker,
getContext()->getDistributedSchedulePool());
getContext()->getDistributedSchedulePool(),
/* initialize_from_disk= */ startup);
return data;
};

View File

@ -1,6 +1,9 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/StorageValues.h>
#include <TableFunctions/ITableFunction.h>
@ -20,22 +23,58 @@ namespace ErrorCodes
void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/)
{
const auto * function = ast_function->as<ASTFunction>();
if (function && function->arguments && function->arguments->children.size() == 1)
{
const auto & query_arg = function->arguments->children[0];
if (!query_arg->as<ASTExplainQuery>())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a explain query argument, got '{}'",
getName(), queryToString(query_arg));
query = query_arg;
}
else
{
if (!function || !function->arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' cannot be called directly, use `SELECT * FROM (EXPLAIN ...)` syntax", getName());
size_t num_args = function->arguments->children.size();
if (num_args != 2 && num_args != 3)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires 2 or 3 arguments, got {}", getName(), num_args);
const auto & kind_arg = function->arguments->children[0];
const auto * kind_literal = kind_arg->as<ASTLiteral>();
if (!kind_literal || kind_literal->value.getType() != Field::Types::String)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a String argument for EXPLAIN kind, got '{}'",
getName(), queryToString(kind_arg));
ASTExplainQuery::ExplainKind kind = ASTExplainQuery::fromString(kind_literal->value.get<String>());
auto explain_query = std::make_shared<ASTExplainQuery>(kind);
const auto * settings_arg = function->arguments->children[1]->as<ASTLiteral>();
if (!settings_arg || settings_arg->value.getType() != Field::Types::String)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a serialized string settings argument, got '{}'",
getName(), queryToString(function->arguments->children[1]));
const auto & settings_str = settings_arg->value.get<String>();
if (!settings_str.empty())
{
constexpr UInt64 max_size = 4096;
constexpr UInt64 max_depth = 16;
/// parse_only_internals_ = true - we don't want to parse `SET` keyword
ParserSetQuery settings_parser(/* parse_only_internals_ = */ true);
ASTPtr settings_ast = parseQuery(settings_parser, settings_str, max_size, max_depth);
explain_query->setSettings(std::move(settings_ast));
}
if (function->arguments->children.size() > 2)
{
const auto & query_arg = function->arguments->children[2];
if (!query_arg->as<ASTSelectWithUnionQuery>())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a EXPLAIN SELECT query argument, got EXPLAIN '{}'",
getName(), queryToString(query_arg));
explain_query->setExplainedQuery(query_arg);
}
else if (kind != ASTExplainQuery::ExplainKind::CurrentTransaction)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a query argument", getName());
}
query = std::move(explain_query);
}
ColumnsDescription TableFunctionExplain::getActualTableStructure(ContextPtr context) const

View File

@ -25,6 +25,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/)
@ -37,9 +38,7 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co
ASTs & args = args_func.at(0)->children;
if (args.empty())
throw Exception("Table function '" + getName() + "' requires at least one argument: "
" structure, [random_seed, max_string_length, max_array_length].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return;
if (args.size() > 4)
throw Exception("Table function '" + getName() + "' requires at most four arguments: "
@ -77,12 +76,23 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co
ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const
{
if (structure == "auto")
{
if (structure_hint.empty())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Table function '{}' was used without structure argument but structure could not be determined automatically. Please, "
"provide structure manually",
getName());
return structure_hint;
}
return parseColumnsListFromString(structure, context);
}
StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
auto columns = getActualTableStructure(context);
ColumnsDescription columns = getActualTableStructure(context);
auto res = std::make_shared<StorageGenerateRandom>(
StorageID(getDatabaseName(), table_name), columns, String{}, max_array_length, max_string_length, random_seed);
res->startup();

View File

@ -5,7 +5,7 @@
namespace DB
{
/* generateRandom(structure, [max_array_length, max_string_length, random_seed])
/* generateRandom([structure, max_array_length, max_string_length, random_seed])
* - creates a temporary storage that generates columns with random data
*/
class TableFunctionGenerateRandom : public ITableFunction
@ -13,7 +13,11 @@ class TableFunctionGenerateRandom : public ITableFunction
public:
static constexpr auto name = "generateRandom";
std::string getName() const override { return name; }
bool hasStaticStructure() const override { return true; }
bool hasStaticStructure() const override { return structure != "auto"; }
bool needStructureHint() const override { return structure == "auto"; }
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override;
const char * getStorageTypeName() const override { return "GenerateRandom"; }
@ -21,11 +25,11 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String structure;
String structure = "auto";
UInt64 max_string_length = 10;
UInt64 max_array_length = 10;
std::optional<UInt64> random_seed;
ColumnsDescription structure_hint;
};

View File

@ -120,6 +120,7 @@ if __name__ == "__main__":
"fuzzer.log": os.path.join(workspace_path, "fuzzer.log"),
"report.html": os.path.join(workspace_path, "report.html"),
"core.zst": os.path.join(workspace_path, "core.zst"),
"dmesg.log": os.path.join(workspace_path, "dmesg.log"),
}
s3_helper = S3Helper()

205
tests/ci/merge_pr.py Normal file
View File

@ -0,0 +1,205 @@
#!/usr/bin/env python
"""Script to check if PR is mergeable and merge it"""
import argparse
import logging
from datetime import datetime
from os import getenv
from typing import Dict, List
from github.PullRequestReview import PullRequestReview
from commit_status_helper import get_commit_filtered_statuses
from get_robot_token import get_best_robot_token
from github_helper import GitHub, NamedUser, PullRequest
from pr_info import PRInfo
# The team name for accepted approvals
TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core")
class Reviews:
STATES = ["CHANGES_REQUESTED", "APPROVED"]
def __init__(self, pr: PullRequest):
"""The reviews are proceed in the next logic:
- if review for an author does not exist, set it
- the review status can be changed from CHANGES_REQUESTED and APPROVED
only to either one
"""
logging.info("Checking the PR for approvals")
self.pr = pr
self.reviews = pr.get_reviews()
# the reviews are ordered by time
self._review_per_user = {} # type: Dict[NamedUser, PullRequestReview]
self.approved_at = datetime.fromtimestamp(0)
for r in self.reviews:
user = r.user
if self._review_per_user.get(user):
if r.state in self.STATES:
self._review_per_user[user] = r
if r.state == "APPROVED":
self.approved_at = max(r.submitted_at, self.approved_at)
continue
self._review_per_user[user] = r
def is_approved(self, team: List[NamedUser]) -> bool:
"""Checks if the PR is approved, and no changes made after the last approval"""
if not self.reviews:
logging.info("There aren't reviews for PR #%s", self.pr.number)
return False
# We consider reviews only from the given list of users
statuses = {
r.state
for user, r in self._review_per_user.items()
if r.state == "CHANGES_REQUESTED"
or (r.state == "APPROVED" and user in team)
}
if "CHANGES_REQUESTED" in statuses:
logging.info(
"The following users requested changes for the PR: %s",
", ".join(
user.login
for user, r in self._review_per_user.items()
if r.state == "CHANGES_REQUESTED"
),
)
return False
if "APPROVED" in statuses:
logging.info(
"The following users from %s team approved the PR: %s",
TEAM_NAME,
", ".join(
user.login
for user, r in self._review_per_user.items()
if r.state == "APPROVED" and user in team
),
)
# The only reliable place to get the 100% accurate last_modified
# info is when the commit was pushed to GitHub. The info is
# available as a header 'last-modified' of /{org}/{repo}/commits/{sha}.
# Unfortunately, it's formatted as 'Wed, 04 Jan 2023 11:05:13 GMT'
commit = self.pr.head.repo.get_commit(self.pr.head.sha)
if commit.stats.last_modified is None:
logging.warning(
"Unable to get info about the commit %s", self.pr.head.sha
)
return False
last_changed = datetime.strptime(
commit.stats.last_modified, "%a, %d %b %Y %H:%M:%S GMT"
)
if self.approved_at < last_changed:
logging.info(
"There are changes after approve at %s",
self.approved_at.isoformat(),
)
return False
return True
logging.info("The PR #%s is not approved", self.pr.number)
return False
def parse_args() -> argparse.Namespace:
pr_info = PRInfo()
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Script to merge the given PR. Additional checks for approved "
"status and green commit statuses could be done",
)
parser.add_argument(
"--check-approved",
action="store_true",
help="if set, checks that the PR is approved and no changes required",
)
parser.add_argument("--check-green", default=True, help=argparse.SUPPRESS)
parser.add_argument(
"--no-check-green",
dest="check_green",
action="store_false",
default=argparse.SUPPRESS,
help="(dangerous) if set, skip check commit to having all green statuses",
)
parser.add_argument(
"--repo",
default=pr_info.repo_full_name,
help="PR number to check",
)
parser.add_argument(
"--pr",
type=int,
default=pr_info.number,
help="PR number to check",
)
parser.add_argument(
"--token",
type=str,
default="",
help="a token to use for GitHub API requests, will be received from SSM "
"if empty",
)
args = parser.parse_args()
args.pr_info = pr_info
return args
def main():
logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
args = parse_args()
logging.info("Going to process PR #%s in repo %s", args.pr, args.repo)
token = args.token or get_best_robot_token()
gh = GitHub(token, per_page=100)
repo = gh.get_repo(args.repo)
# An ugly and not nice fix to patch the wrong organization URL,
# see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710
# pylint: disable=protected-access
repo.organization._url.value = repo.organization.url.replace( # type: ignore
"/users/", "/orgs/", 1
)
# pylint: enable=protected-access
pr = repo.get_pull(args.pr)
if pr.is_merged():
logging.info("The PR #%s is already merged", pr.number)
return
not_ready_to_merge = pr.draft or "WIP" in pr.title
if not_ready_to_merge:
logging.info("The PR #%s is not ready for merge, stopping", pr.number)
return
if args.check_green:
logging.info("Checking that all PR's statuses are green")
commit = repo.get_commit(pr.head.sha)
failed_statuses = [
status.context
for status in get_commit_filtered_statuses(commit)
if status.state != "success"
]
if failed_statuses:
logging.warning(
"Some statuses aren't success:\n %s", ",\n ".join(failed_statuses)
)
return
if args.check_approved:
reviews = Reviews(pr)
team = repo.organization.get_team_by_slug(TEAM_NAME)
members = list(team.get_members())
if not reviews.is_approved(members):
logging.warning("We don't merge the PR")
return
logging.info("Merging the PR")
pr.merge()
if __name__ == "__main__":
main()

View File

@ -3,6 +3,7 @@ from ast import literal_eval
from dataclasses import dataclass
from pathlib import Path
from typing import List, Optional, Tuple
from html import escape
import csv
import os
import datetime
@ -372,9 +373,10 @@ def create_test_html_report(
row += "</tr>"
rows_part += row
if test_result.raw_logs is not None:
raw_logs = escape(test_result.raw_logs)
row = (
'<tr class="failed-content">'
f'<td colspan="{colspan}"><pre>{test_result.raw_logs}</pre></td>'
f'<td colspan="{colspan}"><pre>{raw_logs}</pre></td>'
"</tr>"
)
rows_part += row

View File

@ -1958,19 +1958,22 @@ def main(args):
def find_binary(name):
if os.path.exists(name) and os.access(name, os.X_OK):
return True
if os.access(name, os.X_OK):
return name
paths = os.environ.get("PATH").split(":")
for path in paths:
if os.access(os.path.join(path, name), os.X_OK):
return True
bin_path = os.path.join(path, name)
if os.access(bin_path, os.X_OK):
return bin_path
# maybe it wasn't in PATH
if os.access(os.path.join("/usr/local/bin", name), os.X_OK):
return True
if os.access(os.path.join("/usr/bin", name), os.X_OK):
return True
return False
bin_path = os.path.join("/usr/local/bin", name)
if os.access(bin_path, os.X_OK):
return bin_path
bin_path = os.path.join("/usr/bin", name)
if os.access(bin_path, os.X_OK):
return bin_path
return None
def get_additional_client_options(args):
@ -2010,9 +2013,8 @@ if __name__ == "__main__":
parser.add_argument(
"-b",
"--binary",
default="clickhouse",
help="Path to clickhouse"
"binary or name of binary in PATH",
default=find_binary("clickhouse"),
help="Path to clickhouse binary or name of binary in PATH",
)
parser.add_argument(
@ -2282,18 +2284,13 @@ if __name__ == "__main__":
if args.tmp is None:
args.tmp = args.queries
if args.client is None:
if find_binary(args.binary + "-client"):
args.client = args.binary + "-client"
print("Using " + args.client + " as client program")
elif find_binary(args.binary):
client_bin = find_binary(args.binary + "-client")
if client_bin is not None:
args.client = client_bin
print("Using {args.client} as client program")
elif args.binary:
args.client = args.binary + " client"
print(
"Using "
+ args.client
+ " as client program (expecting monolithic build)"
)
print(f"Using {args.client} as client program (expecting monolithic build)")
else:
print(
"No 'clickhouse' or 'clickhouse-client' client binary found",

View File

@ -1625,6 +1625,12 @@ def test_rename(start_cluster):
"""
)
# We want to check that after inserts, some parts were moved to external disk
# and some parts are still on the main disk, but because of merge all parts
# might end up on external disk.
node1.query("SYSTEM STOP MERGES default.renaming_table")
# jbod1 disk is 40mb
for _ in range(5):
data = []
for i in range(10):
@ -1635,8 +1641,14 @@ def test_rename(start_cluster):
)
)
disks = get_used_disks_for_table(node1, "renaming_table")
assert len(disks) > 1
# data is moved in the background, so check with retries
num_try = 0
while get_used_disks_for_table(node1, "renaming_table") == 1:
time.sleep(1)
num_try += 1
if num_try == 20:
break
assert len(get_used_disks_for_table(node1, "renaming_table")) > 1
assert node1.query("SELECT COUNT() FROM default.renaming_table") == "50\n"
node1.query("RENAME TABLE default.renaming_table TO default.renaming_table1")

View File

@ -13,7 +13,6 @@
<value>Delta</value>
<value>T64</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
</values>
</substitution>
<substitution>

View File

@ -13,7 +13,6 @@
<value>Delta</value>
<value>T64</value>
<value>DoubleDelta</value>
<value>Gorilla</value>
</values>
</substitution>
<substitution>

View File

@ -1,3 +1,4 @@
10 [1,1,1,1,10,10,10,10,100,100,100]
10 [1,1,2,4,7,10,35,61,87,100,100]
100 100
61 61

View File

@ -1,3 +1,4 @@
SELECT quantileExactWeighted(0.5)(x, 1) AS q5, quantilesExactWeighted(0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1)(x, 1) AS qs FROM (SELECT arrayJoin([1, 1, 1, 10, 10, 10, 10, 100, 100, 100]) AS x);
SELECT quantileInterpolatedWeighted(0.5)(x, 1) AS q5, quantilesInterpolatedWeighted(0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1)(x, 1) AS qs FROM (SELECT arrayJoin([1, 1, 1, 10, 10, 10, 10, 100, 100, 100]) AS x);
SELECT quantileExact(0)(x), quantileTiming(0)(x) FROM (SELECT number + 100 AS x FROM system.numbers LIMIT 10000);
SELECT quantileExact(x), quantileTiming(x) FROM (SELECT number % 123 AS x FROM system.numbers LIMIT 10000);

View File

@ -0,0 +1 @@
[(-1,-1,2)]

View File

@ -0,0 +1,6 @@
drop table if exists histogram;
create table histogram(num Int64) engine=TinyLog;
insert into histogram values(-1);
insert into histogram values(-1);
select histogram(2)(num) from histogram;
drop table if exists histogram;

View File

@ -6,6 +6,8 @@
['2016-06-15 23:00:00']
2016-06-15 23:00:00
['2016-06-15 23:00:00']
2016-06-15 23:00:00
['2016-06-15 23:00:00']
30000
[30000]
30000

View File

@ -15,6 +15,9 @@ SELECT quantilesExact(0.2)(d) FROM datetime;
SELECT quantileExactWeighted(0.2)(d, 1) FROM datetime;
SELECT quantilesExactWeighted(0.2)(d, 1) FROM datetime;
SELECT quantileInterpolatedWeighted(0.2)(d, 1) FROM datetime;
SELECT quantilesInterpolatedWeighted(0.2)(d, 1) FROM datetime;
SELECT quantileTiming(0.2)(d) FROM datetime;
SELECT quantilesTiming(0.2)(d) FROM datetime;

View File

@ -1,5 +1,5 @@
SELECT * FROM generateRandom('i8', 1, 10, 10); -- { serverError 62 }
SELECT * FROM generateRandom; -- { serverError 60 }
SELECT * FROM generateRandom(); -- { serverError 42 }
SELECT * FROM generateRandom(); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
SELECT * FROM generateRandom('i8 UInt8', 1, 10, 10, 10, 10); -- { serverError 42 }
SELECT * FROM generateRandom('', 1, 10, 10); -- { serverError 62 }

View File

@ -6,6 +6,8 @@ CREATE TABLE default.codecs5\n(\n `a` UInt8 CODEC(LZ4, ZSTD(1))\n)\nENGINE =
CREATE TABLE default.codecs6\n(\n `a` UInt8 CODEC(Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs7\n(\n `a` UInt8 CODEC(Delta(1), Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs8\n(\n `a` UInt8 CODEC(LZ4, Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs9\n(\n `a` Float64 CODEC(Delta(8), FPC(12))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs10\n(\n `a` Float64 CODEC(DoubleDelta, Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs1\n(\n `a` UInt8 CODEC(NONE, NONE)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs2\n(\n `a` UInt8 CODEC(NONE, LZ4)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs3\n(\n `a` UInt8 CODEC(LZ4, NONE)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
@ -14,3 +16,5 @@ CREATE TABLE default.codecs5\n(\n `a` UInt8 CODEC(LZ4, ZSTD(1))\n)\nENGINE =
CREATE TABLE default.codecs6\n(\n `a` UInt8 CODEC(Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs7\n(\n `a` UInt8 CODEC(Delta(1), Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs8\n(\n `a` UInt8 CODEC(LZ4, Delta(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs9\n(\n `a` Float64 CODEC(Delta(8), FPC(12))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.codecs10\n(\n `a` Float64 CODEC(DoubleDelta, Gorilla)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -11,24 +11,28 @@ CREATE TABLE codecs
c Float32 CODEC(Gorilla),
d UInt8 CODEC(Delta, LZ4),
e Float64 CODEC(Gorilla, ZSTD),
f UInt32 CODEC(Delta, Delta, Gorilla),
f UInt64 CODEC(Delta, Delta, T64),
g DateTime CODEC(DoubleDelta),
h DateTime64 CODEC(DoubleDelta, LZ4),
i String CODEC(NONE)
i String CODEC(NONE),
j Float64 (Gorilla, Delta),
k Float32 (FPC, DoubleDelta)
) ENGINE = MergeTree ORDER BY tuple();
DROP TABLE codecs;
-- test what should not work
CREATE TABLE codecs (a UInt8 CODEC(NONE, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(NONE, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError 36 }
CREATE TABLE codecs (a UInt8 CODEC(NONE, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(NONE, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, NONE)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, LZ4)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(Delta, FPC)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
CREATE TABLE codecs (a UInt8 CODEC(DoubleDelta, Gorilla)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError BAD_ARGUMENTS }
-- test that sanity check is not performed in ATTACH query
@ -40,6 +44,8 @@ DROP TABLE IF EXISTS codecs5;
DROP TABLE IF EXISTS codecs6;
DROP TABLE IF EXISTS codecs7;
DROP TABLE IF EXISTS codecs8;
DROP TABLE IF EXISTS codecs9;
DROP TABLE IF EXISTS codecs10;
SET allow_suspicious_codecs = 1;
@ -51,6 +57,8 @@ CREATE TABLE codecs5 (a UInt8 CODEC(LZ4, ZSTD)) ENGINE = MergeTree ORDER BY tupl
CREATE TABLE codecs6 (a UInt8 CODEC(Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs7 (a UInt8 CODEC(Delta, Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs8 (a UInt8 CODEC(LZ4, Delta)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs9 (a Float64 CODEC(Delta, FPC)) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE codecs10 (a Float64 CODEC(DoubleDelta, Gorilla)) ENGINE = MergeTree ORDER BY tuple();
SET allow_suspicious_codecs = 0;
@ -62,6 +70,8 @@ SHOW CREATE TABLE codecs5;
SHOW CREATE TABLE codecs6;
SHOW CREATE TABLE codecs7;
SHOW CREATE TABLE codecs8;
SHOW CREATE TABLE codecs9;
SHOW CREATE TABLE codecs10;
DETACH TABLE codecs1;
DETACH TABLE codecs2;
@ -71,6 +81,8 @@ DETACH TABLE codecs5;
DETACH TABLE codecs6;
DETACH TABLE codecs7;
DETACH TABLE codecs8;
DETACH TABLE codecs9;
DETACH TABLE codecs10;
ATTACH TABLE codecs1;
ATTACH TABLE codecs2;
@ -80,6 +92,8 @@ ATTACH TABLE codecs5;
ATTACH TABLE codecs6;
ATTACH TABLE codecs7;
ATTACH TABLE codecs8;
ATTACH TABLE codecs9;
ATTACH TABLE codecs10;
SHOW CREATE TABLE codecs1;
SHOW CREATE TABLE codecs2;
@ -89,6 +103,8 @@ SHOW CREATE TABLE codecs5;
SHOW CREATE TABLE codecs6;
SHOW CREATE TABLE codecs7;
SHOW CREATE TABLE codecs8;
SHOW CREATE TABLE codecs9;
SHOW CREATE TABLE codecs10;
SELECT * FROM codecs1;
SELECT * FROM codecs2;
@ -98,6 +114,8 @@ SELECT * FROM codecs5;
SELECT * FROM codecs6;
SELECT * FROM codecs7;
SELECT * FROM codecs8;
SELECT * FROM codecs9;
SELECT * FROM codecs10;
DROP TABLE codecs1;
DROP TABLE codecs2;
@ -107,3 +125,5 @@ DROP TABLE codecs5;
DROP TABLE codecs6;
DROP TABLE codecs7;
DROP TABLE codecs8;
DROP TABLE codecs9;
DROP TABLE codecs10;

View File

@ -1,7 +1,7 @@
<Warning> DistributedSink: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 1)), implicit conversion will be done.
<Warning> DistributedSink: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 1)), implicit conversion will be done.
<Warning> default.dist_01683.DirectoryMonitor: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done
<Warning> default.dist_01683.DirectoryMonitor: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done
<Warning> default.dist_01683.DirectoryMonitor.default: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done
<Warning> default.dist_01683.DirectoryMonitor.default: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done
1
1
2

View File

@ -1 +1,3 @@
-- Tags: no-ordinary-database
create table test_table engine MergeTree order by a as select a_table.a, b_table.b_arr from (select arrayJoin(range(10000)) as a) a_table cross join (select range(10000) as b_arr) b_table settings max_memory_usage = 1; -- { serverError MEMORY_LIMIT_EXCEEDED }

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, distributed
# Tags: long, distributed, no-tsan
# These tests don't use `current_database = currentDatabase()` condition, because database name isn't propagated during remote queries.

View File

@ -0,0 +1,12 @@
quantileInterpolatedWeighted
0 0 0 Decimal(38, 8)
-25.5 -8.49999999 -5.1 Decimal(38, 8)
0 0 0
9.7 3.23333333 1.94
19.9 6.63333332 3.98
30.1 10.03333333 6.02
40.3 13.43333332 8.06
50 16.66666666 10
[-50,-40.4,-30.3,-20.2,-10.1,0,10.1,20.2,30.3,40.4,50]
[-16.66666666,-13.46666666,-10.09999999,-6.73333332,-3.36666666,0,3.36666666,6.73333332,10.09999999,13.46666666,16.66666666]
[-10,-8.08,-6.06,-4.04,-2.02,0,2.02,4.04,6.06,8.08,10]

View File

@ -0,0 +1,27 @@
DROP TABLE IF EXISTS decimal;
CREATE TABLE decimal
(
a Decimal32(4),
b Decimal64(8),
c Decimal128(8)
) ENGINE = Memory;
INSERT INTO decimal (a, b, c)
SELECT toDecimal32(number - 50, 4), toDecimal64(number - 50, 8) / 3, toDecimal128(number - 50, 8) / 5
FROM system.numbers LIMIT 101;
SELECT 'quantileInterpolatedWeighted';
SELECT medianInterpolatedWeighted(a, 1), medianInterpolatedWeighted(b, 2), medianInterpolatedWeighted(c, 3) as x, toTypeName(x) FROM decimal;
SELECT quantileInterpolatedWeighted(a, 1), quantileInterpolatedWeighted(b, 2), quantileInterpolatedWeighted(c, 3) as x, toTypeName(x) FROM decimal WHERE a < 0;
SELECT quantileInterpolatedWeighted(0.0)(a, 1), quantileInterpolatedWeighted(0.0)(b, 2), quantileInterpolatedWeighted(0.0)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileInterpolatedWeighted(0.2)(a, 1), quantileInterpolatedWeighted(0.2)(b, 2), quantileInterpolatedWeighted(0.2)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileInterpolatedWeighted(0.4)(a, 1), quantileInterpolatedWeighted(0.4)(b, 2), quantileInterpolatedWeighted(0.4)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileInterpolatedWeighted(0.6)(a, 1), quantileInterpolatedWeighted(0.6)(b, 2), quantileInterpolatedWeighted(0.6)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileInterpolatedWeighted(0.8)(a, 1), quantileInterpolatedWeighted(0.8)(b, 2), quantileInterpolatedWeighted(0.8)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantileInterpolatedWeighted(1.0)(a, 1), quantileInterpolatedWeighted(1.0)(b, 2), quantileInterpolatedWeighted(1.0)(c, 3) FROM decimal WHERE a >= 0;
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(a, 1) FROM decimal;
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(b, 2) FROM decimal;
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(c, 3) FROM decimal;
DROP TABLE IF EXISTS decimal;

View File

@ -6,3 +6,11 @@
1
1
1
1
1
1
1
1
1
1
1

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 0;
SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number UInt64%';
SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
@ -15,9 +17,69 @@ SELECT * FROM (
)
) FORMAT Null;
SELECT (EXPLAIN SYNTAX oneline = 1 SELECT 1) == 'SELECT 1';
SELECT * FROM viewExplain('', ''); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', ''); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', '', 1); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', '', ''); -- { serverError BAD_ARGUMENTS }
CREATE TABLE t1 ( a UInt64 ) Engine = MergeTree ORDER BY tuple() AS SELECT number AS a FROM system.numbers LIMIT 100000;
SELECT rows > 1000 FROM (EXPLAIN ESTIMATE SELECT sum(a) FROM t1);
SELECT count() == 1 FROM (EXPLAIN ESTIMATE SELECT sum(a) FROM t1);
DROP TABLE IF EXISTS t1;
SET allow_experimental_analyzer = 1;
SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: system.numbers.number__ UInt64%';
SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION);
SELECT count() == 1 FROM (EXPLAIN SYNTAX SELECT number FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE 'SELECT%';
-- We have `Identifier number` instead of `Asterisk` because query argument of `viewExplain` table function was analyzed.
-- Compare:
-- :) EXPLAIN AST SELECT *;
-- ┌─explain───────────────────────────┐
-- │ SelectWithUnionQuery (children 1) │
-- │ ExpressionList (children 1) │
-- │ SelectQuery (children 1) │
-- │ ExpressionList (children 1) │
-- │ Asterisk │
-- └───────────────────────────────────┘
-- :) SELECT * FROM (EXPLAIN AST SELECT *);
-- ┌─explain─────────────────────────────────────┐
-- │ SelectWithUnionQuery (children 1) │
-- │ ExpressionList (children 1) │
-- │ SelectQuery (children 2) │
-- │ ExpressionList (children 1) │
-- │ Identifier dummy │
-- │ TablesInSelectQuery (children 1) │
-- │ TablesInSelectQueryElement (children 1) │
-- │ TableExpression (children 1) │
-- │ TableIdentifier system.one │
-- └─────────────────────────────────────────────┘
-- TODO: argument of `viewExplain` (and subquery in `EXAPLAN ...`) should not be analyzed.
-- See _Support query tree in table functions_ in https://github.com/ClickHouse/ClickHouse/issues/42648
SELECT trim(explain) == 'Identifier number' FROM (EXPLAIN AST SELECT * FROM system.numbers LIMIT 10) WHERE explain LIKE '%Identifier number%';
SELECT * FROM (
EXPLAIN AST SELECT * FROM (
EXPLAIN PLAN SELECT * FROM (
EXPLAIN SYNTAX SELECT trim(explain) == 'Asterisk' FROM (
EXPLAIN AST SELECT * FROM system.numbers LIMIT 10
) WHERE explain LIKE '%Asterisk%'
)
)
) FORMAT Null;
SELECT (EXPLAIN SYNTAX oneline = 1 SELECT 1) == 'SELECT 1 FROM system.one';
SELECT * FROM viewExplain('', ''); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', ''); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', '', 1); -- { serverError BAD_ARGUMENTS }
SELECT * FROM viewExplain('EXPLAIN AST', '', ''); -- { serverError BAD_ARGUMENTS }
-- EXPLAIN ESTIMATE is not supported in experimental analyzer

View File

@ -0,0 +1,2 @@
CREATE TABLE default.dep\n(\n `id` Int32,\n `country` LowCardinality(String),\n `purchase_location` UInt16 MATERIALIZED if(id IN joinGet(\'default.id_join\', \'location\', \'CLICK\'), 123, 456)\n)\nENGINE = ReplicatedMergeTree(\'/test/02433/default/dep\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.dep2\n(\n `id` Int32,\n `country` LowCardinality(String),\n `purchase_location` UInt16 MATERIALIZED if(id IN joinGet(\'default.id_join\', \'location\', \'CLICK\'), 123, 456)\n)\nENGINE = ReplicatedMergeTree(\'/test/02433/default/dep\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,37 @@
DROP TABLE IF EXISTS dep;
DROP TABLE IF EXISTS dep2;
DROP TABLE IF EXISTS id_join;
CREATE TABLE id_join (`country` String, `location` Array(Int32)) ENGINE = Join(ANY, LEFT, country);
INSERT INTO id_join values ('CLICK', [1234]);
CREATE TABLE dep
(
`id` Int32,
`country` LowCardinality(String),
`purchase_location` UInt16 MATERIALIZED if(id IN joinGet(concat(currentDatabase(), '.id_join'), 'location', 'CLICK'), 123, 456)
)
ENGINE = ReplicatedMergeTree('/test/02433/{database}/dep', '1') ORDER BY tuple();
SHOW CREATE TABLE dep;
TRUNCATE TABLE id_join;
CREATE TABLE dep2
(
`id` Int32,
`country` LowCardinality(String),
`purchase_location` UInt16 MATERIALIZED if(id IN joinGet(concat(currentDatabase(), '.id_join'), 'location', 'CLICK'), 123, 456)
)
ENGINE = ReplicatedMergeTree('/test/02433/{database}/dep', '2') ORDER BY tuple();
SHOW CREATE TABLE dep2;
-- Ensure that a table name cannot be passed to IN as string literal
create table test (n int, m default n in 'default.table_name') engine=Memory; -- { serverError TYPE_MISMATCH }
create table test (n int, m default in(n, 'default.table_name')) engine=Memory; -- { serverError TYPE_MISMATCH }
DROP TABLE dep;
DROP TABLE dep2;
DROP TABLE id_join;

View File

@ -0,0 +1 @@
ServerStartupMilliseconds

View File

@ -0,0 +1 @@
SELECT event FROM system.events WHERE event = 'ServerStartupMilliseconds'

View File

@ -0,0 +1,6 @@
drop table if exists test;
create table test (x UInt32, y String) engine=Memory;
insert into test select * from generateRandom() limit 10;
select count() from test;
drop table test;

View File

@ -0,0 +1,4 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (id UInt64, val Decimal(15,5) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }
CREATE TABLE test (id UInt64, val FixedString(2) CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }
CREATE TABLE test (id UInt64, val UInt64 CODEC (Gorilla)) ENGINE = MergeTree() ORDER BY id; -- { serverError BAD_ARGUMENTS }

View File

@ -23,7 +23,7 @@ export CLICKHOUSE_TEST_UNIQUE_NAME="${CLICKHOUSE_TEST_NAME}_${CLICKHOUSE_DATABAS
[ -n "${CLICKHOUSE_DATABASE:-}" ] && CLICKHOUSE_BENCHMARK_OPT0+=" --database=${CLICKHOUSE_DATABASE} "
[ -n "${CLICKHOUSE_LOG_COMMENT:-}" ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment $(printf '%q' ${CLICKHOUSE_LOG_COMMENT}) "
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="$(command -v clickhouse)"}
# client
[ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
[ -x "$CLICKHOUSE_BINARY" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY client}