Merge remote-tracking branch 'origin/master' into igor/remove_redundant_order_by

This commit is contained in:
Igor Nikonov 2023-01-17 16:28:17 +00:00
commit 0cfa08df7a
103 changed files with 2538 additions and 845 deletions

View File

@ -1203,12 +1203,14 @@ SELECT * FROM json_each_row_nested
- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `false`.
- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `false`.
- [input_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#input_format_json_named_tuples_as_objects) - parse named tuple columns as JSON objects. Default value - `true`.
- [input_format_json_defaults_for_missing_elements_in_named_tuple](/docs/en/operations/settings/settings.md/#input_format_json_defaults_for_missing_elements_in_named_tuple) - insert default values for missing elements in JSON object while parsing named tuple. Default value - `true`.
- [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`.
- [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`.
- [output_format_json_quote_denormals](/docs/en/operations/settings/settings.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`.
- [output_format_json_quote_decimals](/docs/en/operations/settings/settings.md/#output_format_json_quote_decimals) - controls quoting of decimals in JSON output format. Default value - `false`.
- [output_format_json_escape_forward_slashes](/docs/en/operations/settings/settings.md/#output_format_json_escape_forward_slashes) - controls escaping forward slashes for string outputs in JSON output format. Default value - `true`.
- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `false`.
- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `true`.
- [output_format_json_array_of_rows](/docs/en/operations/settings/settings.md/#output_format_json_array_of_rows) - output a JSON array of all rows in JSONEachRow(Compact) format. Default value - `false`.
- [output_format_json_validate_utf8](/docs/en/operations/settings/settings.md/#output_format_json_validate_utf8) - enables validation of UTF-8 sequences in JSON output formats (note that it doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8). Default value - `false`.

View File

@ -266,7 +266,7 @@ Default value: 0.
Limits the size in bytes of the hash table used when joining tables.
This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md).
This setting applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md).
If the query contains joins, ClickHouse checks this setting for every intermediate result.

View File

@ -402,40 +402,62 @@ Default value: `ALL`.
## join_algorithm {#settings-join_algorithm}
Specifies [JOIN](../../sql-reference/statements/select/join.md) algorithm.
Specifies which [JOIN](../../sql-reference/statements/select/join.md) algorithm is used.
Several algorithms can be specified, and an available one would be chosen for a particular query based on kind/strictness and table engine.
Possible values:
- `default``hash` or `direct`, if possible (same as `direct,hash`)
### `default`
- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section.
This is the equivalent of `hash` or `direct`, if possible (same as `direct,hash`)
- `parallel_hash` - a variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process.
### `grace_hash`
[Grace hash join](https://en.wikipedia.org/wiki/Hash_join#Grace_hash_join) is used. Grace hash provides an algorithm option that provides performant complex joins while limiting memory use.
The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which dont belong to the current bucket are flushed and reassigned.
### `hash`
[Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section.
### `parallel_hash`
A variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process.
When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM.
- `partial_merge` — a variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted.
### `partial_merge`
A variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted.
The `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported).
When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks.
When using the `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by the `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks.
- `direct` - can be applied when the right storage supports key-value requests.
### `direct`
This algorithm can be applied when the storage for the right table supports key-value requests.
The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs.
- `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated.
### `auto`
- `full_sorting_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining.
When set to `auto`, `hash` join is tried first, and the algorithm is switched on the fly to another algorithm if the memory limit is violated.
- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`.
### `full_sorting_merge`
[Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining.
### `prefer_partial_merge`
ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`.
## join_any_take_last_row {#settings-join_any_take_last_row}
Changes behaviour of join operations with `ANY` strictness.
Changes the behaviour of join operations with `ANY` strictness.
:::warning
This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables.
@ -498,7 +520,7 @@ Default value: 65536.
Limits the number of files allowed for parallel sorting in MergeJoin operations when they are executed on disk.
The bigger the value of the setting, the more RAM used and the less disk I/O needed.
The bigger the value of the setting, the more RAM is used and the less disk I/O is needed.
Possible values:
@ -514,12 +536,12 @@ Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations.
Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour.
:::
When the legacy behaviour enabled:
When the legacy behaviour is enabled:
- Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are not equal because ClickHouse uses the logic with many-to-one left-to-right table keys mapping.
- Results of `ANY INNER JOIN` operations contain all rows from the left table like the `SEMI LEFT JOIN` operations do.
When the legacy behaviour disabled:
When the legacy behaviour is disabled:
- Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are equal because ClickHouse uses the logic which provides one-to-many keys mapping in `ANY RIGHT JOIN` operations.
- Results of `ANY INNER JOIN` operations contain one row per key from both the left and right tables.
@ -572,7 +594,7 @@ Default value: `163840`.
## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem}
The minimum number of lines to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem.
The minimum number of lines to read from one file before the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem.
Possible values:
@ -706,7 +728,7 @@ log_queries=1
## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms}
If enabled (non-zero), queries faster then the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables:
If enabled (non-zero), queries faster than the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables:
- `system.query_log`
- `system.query_thread_log`
@ -741,7 +763,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING'
Setting up query threads logging.
Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter.
Query threads log into the [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting has effect only when [log_queries](#settings-log-queries) is true. Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter.
Possible values:
@ -760,7 +782,7 @@ log_query_threads=1
Setting up query views logging.
When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter.
When a query run by ClickHouse with this setting enabled has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter.
Example:
@ -787,7 +809,7 @@ It can be used to improve the readability of server logs. Additionally, it helps
Possible values:
- Any string no longer than [max_query_size](#settings-max_query_size). If length is exceeded, the server throws an exception.
- Any string no longer than [max_query_size](#settings-max_query_size). If the max_query_size is exceeded, the server throws an exception.
Default value: empty string.
@ -821,11 +843,11 @@ The setting also does not have a purpose when using INSERT SELECT, since data is
Default value: 1,048,576.
The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM.
The default is slightly more than `max_block_size`. The reason for this is that certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM.
## min_insert_block_size_rows {#min-insert-block-size-rows}
Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones.
Sets the minimum number of rows in the block that can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones.
Possible values:
@ -891,7 +913,7 @@ Higher values will lead to higher memory usage.
## max_compress_block_size {#max-compress-block-size}
The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced.
The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced.
:::warning
This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse.
@ -935,7 +957,7 @@ Default value: 1000.
## interactive_delay {#interactive-delay}
The interval in microseconds for checking whether request execution has been cancelled and sending the progress.
The interval in microseconds for checking whether request execution has been canceled and sending the progress.
Default value: 100,000 (checks for cancelling and sends the progress ten times per second).
@ -4122,7 +4144,20 @@ Enabled by default.
Serialize named tuple columns as JSON objects.
Disabled by default.
Enabled by default.
### input_format_json_named_tuples_as_objects {#input_format_json_named_tuples_as_objects}
Parse named tuple columns as JSON objects.
Enabled by default.
### input_format_json_defaults_for_missing_elements_in_named_tuple {#input_format_json_defaults_for_missing_elements_in_named_tuple}
Insert default values for missing elements in JSON object while parsing named tuple.
This setting works only when setting `input_format_json_named_tuples_as_objects` is enabled.
Enabled by default.
### output_format_json_array_of_rows {#output_format_json_array_of_rows}

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

@ -167,6 +167,7 @@ enum class AccessType
M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \
M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \
M(SYSTEM_RESTORE_REPLICA, "RESTORE REPLICA", TABLE, SYSTEM) \
M(SYSTEM_WAIT_LOADING_PARTS, "WAIT LOADING PARTS", TABLE, SYSTEM) \
M(SYSTEM_SYNC_DATABASE_REPLICA, "SYNC DATABASE REPLICA", DATABASE, SYSTEM) \
M(SYSTEM_SYNC_TRANSACTION_LOG, "SYNC TRANSACTION LOG", GLOBAL, SYSTEM) \
M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \

View File

@ -53,7 +53,7 @@ TEST(AccessRights, Union)
"SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, "
"SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
"SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, "
"SYSTEM RESTORE REPLICA, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
"SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*");
}

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

@ -156,10 +156,9 @@ void BackupWriterS3::copyObjectImpl(
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
size_t size,
const std::optional<ObjectAttributes> & metadata) const
{
size_t size = head.GetContentLength();
LOG_TRACE(log, "Copying {} bytes using single-operation copy", size);
Aws::S3::Model::CopyObjectRequest request;
@ -177,7 +176,7 @@ void BackupWriterS3::copyObjectImpl(
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, size, metadata);
return;
}
@ -191,10 +190,9 @@ void BackupWriterS3::copyObjectMultipartImpl(
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
size_t size,
const std::optional<ObjectAttributes> & metadata) const
{
size_t size = head.GetContentLength();
LOG_TRACE(log, "Copying {} bytes using multipart upload copy", size);
String multipart_upload_id;
@ -309,16 +307,16 @@ void BackupWriterS3::copyFileNative(DiskPtr from_disk, const String & file_name_
std::string source_bucket = object_storage->getObjectsNamespace();
auto file_path = fs::path(s3_uri.key) / file_name_to;
auto head = S3::headObject(*client, source_bucket, objects[0].absolute_path).GetResult();
if (static_cast<size_t>(head.GetContentLength()) < request_settings.getUploadSettings().max_single_operation_copy_size)
auto size = S3::getObjectSize(*client, source_bucket, objects[0].absolute_path);
if (size < request_settings.getUploadSettings().max_single_operation_copy_size)
{
copyObjectImpl(
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head);
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, size);
}
else
{
copyObjectMultipartImpl(
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head);
source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, size);
}
}
}

View File

@ -67,7 +67,7 @@ private:
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
size_t size,
const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
void copyObjectMultipartImpl(
@ -75,7 +75,7 @@ private:
const String & src_key,
const String & dst_bucket,
const String & dst_key,
const Aws::S3::Model::HeadObjectResult & head,
size_t size,
const std::optional<ObjectAttributes> & metadata = std::nullopt) const;
void removeFilesBatch(const Strings & file_names);

View File

@ -309,6 +309,8 @@ The server successfully detected this situation and will download merged part fr
M(S3CopyObject, "Number of S3 API CopyObject calls.") \
M(S3ListObjects, "Number of S3 API ListObjects calls.") \
M(S3HeadObject, "Number of S3 API HeadObject calls.") \
M(S3GetObjectAttributes, "Number of S3 API GetObjectAttributes calls.") \
M(S3GetObjectMetadata, "Number of S3 API GetObject calls for getting metadata.") \
M(S3CreateMultipartUpload, "Number of S3 API CreateMultipartUpload calls.") \
M(S3UploadPartCopy, "Number of S3 API UploadPartCopy calls.") \
M(S3UploadPart, "Number of S3 API UploadPart calls.") \
@ -321,6 +323,8 @@ The server successfully detected this situation and will download merged part fr
M(DiskS3CopyObject, "Number of DiskS3 API CopyObject calls.") \
M(DiskS3ListObjects, "Number of DiskS3 API ListObjects calls.") \
M(DiskS3HeadObject, "Number of DiskS3 API HeadObject calls.") \
M(DiskS3GetObjectAttributes, "Number of DiskS3 API GetObjectAttributes calls.") \
M(DiskS3GetObjectMetadata, "Number of DiskS3 API GetObject calls for getting metadata.") \
M(DiskS3CreateMultipartUpload, "Number of DiskS3 API CreateMultipartUpload calls.") \
M(DiskS3UploadPartCopy, "Number of DiskS3 API UploadPartCopy calls.") \
M(DiskS3UploadPart, "Number of DiskS3 API UploadPart calls.") \

View File

@ -28,7 +28,6 @@ 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,7 +133,6 @@ 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,7 +39,6 @@ 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,7 +123,6 @@ protected:
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
bool isFloatingPointTimeSeries() const override { return true; }
private:
UInt8 data_bytes_size;
@ -445,19 +444,14 @@ void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 sourc
void registerCodecGorilla(CompressionCodecFactory & factory)
{
UInt8 method_code = static_cast<UInt8>(CompressionMethodByte::Gorilla);
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
factory.registerCompressionCodecWithType("Gorilla", method_code,
[&](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,17 +59,15 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
{
ASTPtr codecs_descriptions = std::make_shared<ASTExpressionList>();
bool with_compressing_codec = false;
bool with_none_codec = false;
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
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;
std::set<size_t> encryption_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const ASTPtr & inner_codec_ast = func->arguments->children[i];
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
ASTPtr codec_arguments;
if (const auto * family_name = inner_codec_ast->as<ASTIdentifier>())
@ -85,7 +83,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
else
throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
/// Replace "Default" codec by configured default codec which may depend on different settings and data properties at runtime.
/// Default codec replaced with current default codec which may depend on different
/// settings (and properties of data) in runtime.
CompressionCodecPtr result_codec;
if (codec_family_name == DEFAULT_CODEC_NAME)
{
@ -137,27 +136,21 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
with_compressing_codec |= result_codec->isCompression();
with_none_codec |= result_codec->isNone();
is_compression |= result_codec->isCompression();
has_none |= result_codec->isNone();
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isEncryption())
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;
encryption_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
if (sanity_check)
{
if (codecs_descriptions->children.size() > 1 && with_none_codec)
if (codecs_descriptions->children.size() > 1 && has_none)
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).",
@ -166,7 +159,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 (!with_compressing_codec && !with_none_codec && encryption_codecs_pos.size() != codecs_descriptions->children.size())
if (!is_compression && !has_none && encryption_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
@ -178,8 +171,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any non-encryption codecs
/// after encryption one.
if (!encryption_codecs_pos.empty() &&
*encryption_codecs_pos.begin() != codecs_descriptions->children.size() - encryption_codecs_pos.size())
if (!encryption_codecs.empty() &&
*encryption_codecs.begin() != codecs_descriptions->children.size() - encryption_codecs.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'"
@ -188,18 +181,11 @@ 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_pos.size())
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs.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,12 +113,6 @@ 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,13 +529,6 @@ 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);
}
@ -1211,20 +1204,68 @@ 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;
// };
// };
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")
},
})
)
);
// 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

@ -774,6 +774,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \
M(Bool, input_format_json_read_numbers_as_strings, false, "Allow to parse numbers as strings in JSON input formats", 0) \
M(Bool, input_format_json_read_objects_as_strings, true, "Allow to parse JSON objects as strings in JSON input formats", 0) \
M(Bool, input_format_json_named_tuples_as_objects, true, "Deserialize named tuple columns as JSON objects", 0) \
M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \

View File

@ -80,7 +80,8 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}}},
{"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"},
{"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}}},
{"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"},
{"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"},
{"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}},

View File

@ -16,6 +16,7 @@ namespace ErrorCodes
{
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int INCORRECT_DATA;
}
@ -154,7 +155,7 @@ void SerializationTuple::deserializeText(IColumn & column, ReadBuffer & istr, co
void SerializationTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (settings.json.named_tuples_as_objects
if (settings.json.write_named_tuples_as_objects
&& have_explicit_names)
{
writeChar('{', ostr);
@ -185,7 +186,7 @@ void SerializationTuple::serializeTextJSON(const IColumn & column, size_t row_nu
void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (settings.json.named_tuples_as_objects
if (settings.json.read_named_tuples_as_objects
&& have_explicit_names)
{
skipWhitespaceIfAny(istr);
@ -194,12 +195,15 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr
addElementSafe(elems.size(), column, [&]
{
// Require all elements but in arbitrary order.
for (size_t i = 0; i < elems.size(); ++i)
std::vector<UInt8> seen_elements(elems.size(), 0);
size_t i = 0;
while (!istr.eof() && *istr.position() != '}')
{
if (i == elems.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected number of elements in named tuple. Expected no more than {}", elems.size());
if (i > 0)
{
skipWhitespaceIfAny(istr);
assertChar(',', istr);
skipWhitespaceIfAny(istr);
}
@ -211,12 +215,35 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr
skipWhitespaceIfAny(istr);
const size_t element_pos = getPositionByName(name);
seen_elements[element_pos] = 1;
auto & element_column = extractElementColumn(column, element_pos);
elems[element_pos]->deserializeTextJSON(element_column, istr, settings);
skipWhitespaceIfAny(istr);
++i;
}
skipWhitespaceIfAny(istr);
assertChar('}', istr);
/// Check if we have missing elements.
if (i != elems.size())
{
for (size_t element_pos = 0; element_pos != seen_elements.size(); ++element_pos)
{
if (seen_elements[element_pos])
continue;
if (!settings.json.defaults_for_missing_elements_in_named_tuple)
throw Exception(
ErrorCodes::INCORRECT_DATA,
"JSON object doesn't contain tuple element {}. If you want to insert defaults in case of missing elements, "
"enable setting input_format_json_defaults_for_missing_elements_in_named_tuple",
elems[element_pos]->getElementName());
auto & element_column = extractElementColumn(column, element_pos);
element_column.insertDefault();
}
}
});
}
else

View File

@ -171,8 +171,9 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl()
if (!hasPendingDataToRead())
return false;
size_t size, offset;
chassert(file_offset_of_buffer_end <= impl->getFileSize());
size_t size, offset;
if (prefetch_future.valid())
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::AsynchronousRemoteReadWaitMicroseconds);
@ -210,8 +211,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl()
/// In case of multiple files for the same file in clickhouse (i.e. log family)
/// file_offset_of_buffer_end will not match getImplementationBufferOffset()
/// so we use [impl->getImplementationBufferOffset(), impl->getFileSize()]
assert(file_offset_of_buffer_end >= impl->getImplementationBufferOffset());
assert(file_offset_of_buffer_end <= impl->getFileSize());
chassert(file_offset_of_buffer_end >= impl->getImplementationBufferOffset());
chassert(file_offset_of_buffer_end <= impl->getFileSize());
return bytes_read;
}
@ -277,6 +278,15 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence)
/// First reset the buffer so the next read will fetch new data to the buffer.
resetWorkingBuffer();
if (read_until_position && new_pos > *read_until_position)
{
ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset);
impl->reset();
file_offset_of_buffer_end = new_pos = *read_until_position; /// read_until_position is a non-included boundary.
return new_pos;
}
/**
* Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer.
* Note: we read in range [file_offset_of_buffer_end, read_until_position).

View File

@ -256,7 +256,7 @@ size_t ReadBufferFromRemoteFSGather::getFileSize() const
String ReadBufferFromRemoteFSGather::getInfoForLog()
{
if (!current_buf)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get info: buffer not initialized");
return "";
return current_buf->getInfoForLog();
}

View File

@ -125,14 +125,19 @@ std::string S3ObjectStorage::generateBlobNameForPath(const std::string & /* path
getRandomASCIIString(key_name_total_size - key_name_prefix_size));
}
Aws::S3::Model::HeadObjectOutcome S3ObjectStorage::requestObjectHeadData(const std::string & bucket_from, const std::string & key) const
size_t S3ObjectStorage::getObjectSize(const std::string & bucket_from, const std::string & key) const
{
return S3::headObject(*client.get(), bucket_from, key, "", true);
return S3::getObjectSize(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true);
}
bool S3ObjectStorage::exists(const StoredObject & object) const
{
return S3::objectExists(*client.get(), bucket, object.absolute_path, "", true);
return S3::objectExists(*client.get(), bucket, object.absolute_path, {}, /* for_disk_s3= */ true);
}
void S3ObjectStorage::checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const
{
return S3::checkObjectExists(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true, description);
}
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
@ -409,13 +414,10 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons
{
ObjectMetadata result;
auto object_head = requestObjectHeadData(bucket, path);
throwIfError(object_head);
auto & object_head_result = object_head.GetResult();
result.size_bytes = object_head_result.GetContentLength();
result.last_modified = object_head_result.GetLastModified().Millis();
result.attributes = object_head_result.GetMetadata();
auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, /* for_disk_s3= */ true);
result.size_bytes = object_info.size;
result.last_modified = object_info.last_modification_time;
result.attributes = S3::getObjectMetadata(*client.get(), bucket, path, {}, /* for_disk_s3= */ true);
return result;
}
@ -442,7 +444,7 @@ void S3ObjectStorage::copyObjectImpl(
const String & src_key,
const String & dst_bucket,
const String & dst_key,
std::optional<Aws::S3::Model::HeadObjectResult> head,
size_t size,
std::optional<ObjectAttributes> metadata) const
{
auto client_ptr = client.get();
@ -464,7 +466,7 @@ void S3ObjectStorage::copyObjectImpl(
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, size, metadata);
return;
}
@ -472,12 +474,7 @@ void S3ObjectStorage::copyObjectImpl(
auto settings_ptr = s3_settings.get();
if (settings_ptr->request_settings.check_objects_after_upload)
{
auto object_head = requestObjectHeadData(dst_bucket, dst_key);
if (!object_head.IsSuccess())
throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", dst_key, dst_bucket);
}
checkObjectExists(dst_bucket, dst_key, "Immediately after upload");
}
void S3ObjectStorage::copyObjectMultipartImpl(
@ -485,15 +482,11 @@ void S3ObjectStorage::copyObjectMultipartImpl(
const String & src_key,
const String & dst_bucket,
const String & dst_key,
std::optional<Aws::S3::Model::HeadObjectResult> head,
size_t size,
std::optional<ObjectAttributes> metadata) const
{
if (!head)
head = requestObjectHeadData(src_bucket, src_key).GetResult();
auto settings_ptr = s3_settings.get();
auto client_ptr = client.get();
size_t size = head->GetContentLength();
String multipart_upload_id;
@ -569,29 +562,24 @@ void S3ObjectStorage::copyObjectMultipartImpl(
}
if (settings_ptr->request_settings.check_objects_after_upload)
{
auto object_head = requestObjectHeadData(dst_bucket, dst_key);
if (!object_head.IsSuccess())
throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", dst_key, dst_bucket);
}
checkObjectExists(dst_bucket, dst_key, "Immediately after upload");
}
void S3ObjectStorage::copyObject( // NOLINT
const StoredObject & object_from, const StoredObject & object_to, std::optional<ObjectAttributes> object_to_attributes)
{
auto head = requestObjectHeadData(bucket, object_from.absolute_path).GetResult();
auto size = getObjectSize(bucket, object_from.absolute_path);
static constexpr int64_t multipart_upload_threashold = 5UL * 1024 * 1024 * 1024;
if (head.GetContentLength() >= multipart_upload_threashold)
if (size >= multipart_upload_threashold)
{
copyObjectMultipartImpl(
bucket, object_from.absolute_path, bucket, object_to.absolute_path, head, object_to_attributes);
bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes);
}
else
{
copyObjectImpl(
bucket, object_from.absolute_path, bucket, object_to.absolute_path, head, object_to_attributes);
bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes);
}
}

View File

@ -172,7 +172,7 @@ private:
const String & src_key,
const String & dst_bucket,
const String & dst_key,
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
size_t size,
std::optional<ObjectAttributes> metadata = std::nullopt) const;
void copyObjectMultipartImpl(
@ -180,13 +180,14 @@ private:
const String & src_key,
const String & dst_bucket,
const String & dst_key,
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
size_t size,
std::optional<ObjectAttributes> metadata = std::nullopt) const;
void removeObjectImpl(const StoredObject & object, bool if_exists);
void removeObjectsImpl(const StoredObjects & objects, bool if_exists);
Aws::S3::Model::HeadObjectOutcome requestObjectHeadData(const std::string & bucket_from, const std::string & key) const;
size_t getObjectSize(const std::string & bucket_from, const std::string & key) const;
void checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const;
std::string bucket;

View File

@ -90,7 +90,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
format_settings.json.array_of_rows = settings.output_format_json_array_of_rows;
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
format_settings.json.named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects;
format_settings.json.write_named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects;
format_settings.json.read_named_tuples_as_objects = settings.input_format_json_named_tuples_as_objects;
format_settings.json.defaults_for_missing_elements_in_named_tuple = settings.input_format_json_defaults_for_missing_elements_in_named_tuple;
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_64bit_floats = settings.output_format_json_quote_64bit_floats;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;

View File

@ -153,7 +153,9 @@ struct FormatSettings
bool quote_denormals = true;
bool quote_decimals = false;
bool escape_forward_slashes = true;
bool named_tuples_as_objects = false;
bool read_named_tuples_as_objects = false;
bool write_named_tuples_as_objects = false;
bool defaults_for_missing_elements_in_named_tuple = false;
bool serialize_as_strings = false;
bool read_bools_as_numbers = true;
bool read_numbers_as_strings = true;

View File

@ -250,7 +250,7 @@ size_t ReadBufferFromS3::getFileSize()
if (file_size)
return *file_size;
auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id, true, read_settings.for_object_storage);
auto object_size = S3::getObjectSize(*client_ptr, bucket, key, version_id, /* for_disk_s3= */ read_settings.for_object_storage);
file_size = object_size;
return *file_size;

View File

@ -27,6 +27,8 @@
# include <aws/core/utils/UUID.h>
# include <aws/core/http/HttpClientFactory.h>
# include <aws/s3/S3Client.h>
# include <aws/s3/model/GetObjectAttributesRequest.h>
# include <aws/s3/model/GetObjectRequest.h>
# include <aws/s3/model/HeadObjectRequest.h>
# include <IO/S3/PocoHTTPClientFactory.h>
@ -40,7 +42,11 @@
namespace ProfileEvents
{
extern const Event S3GetObjectAttributes;
extern const Event S3GetObjectMetadata;
extern const Event S3HeadObject;
extern const Event DiskS3GetObjectAttributes;
extern const Event DiskS3GetObjectMetadata;
extern const Event DiskS3HeadObject;
}
@ -699,6 +705,92 @@ public:
}
};
/// Extracts the endpoint from a constructed S3 client.
String getEndpoint(const Aws::S3::S3Client & client)
{
const auto * endpoint_provider = dynamic_cast<const Aws::S3::Endpoint::S3DefaultEpProviderBase *>(const_cast<Aws::S3::S3Client &>(client).accessEndpointProvider().get());
if (!endpoint_provider)
return {};
String endpoint;
endpoint_provider->GetBuiltInParameters().GetParameter("Endpoint").GetString(endpoint);
return endpoint;
}
/// Performs a request to get the size and last modification time of an object.
/// The function performs either HeadObject or GetObjectAttributes request depending on the endpoint.
std::pair<std::optional<DB::S3::ObjectInfo>, Aws::S3::S3Error> tryGetObjectInfo(
const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3)
{
auto endpoint = getEndpoint(client);
bool use_get_object_attributes_request = (endpoint.find(".amazonaws.com") != String::npos);
if (use_get_object_attributes_request)
{
/// It's better not to use `HeadObject` requests for AWS S3 because they don't work well with the global region.
/// Details: `HeadObject` request never returns a response body (even if there is an error) however
/// if the request was sent without specifying a region in the endpoint (i.e. for example "https://test.s3.amazonaws.com/mydata.csv"
/// instead of "https://test.s3-us-west-2.amazonaws.com/mydata.csv") then that response body is one of the main ways
/// to determine the correct region and try to repeat the request again with the correct region.
/// For any other request type (`GetObject`, `ListObjects`, etc.) AWS SDK does that because they have response bodies,
/// but for `HeadObject` there is no response body so this way doesn't work. That's why we use `GetObjectAttributes` request instead.
/// See https://github.com/aws/aws-sdk-cpp/issues/1558 and also the function S3ErrorMarshaller::ExtractRegion() for more information.
ProfileEvents::increment(ProfileEvents::S3GetObjectAttributes);
if (for_disk_s3)
ProfileEvents::increment(ProfileEvents::DiskS3GetObjectAttributes);
Aws::S3::Model::GetObjectAttributesRequest req;
req.SetBucket(bucket);
req.SetKey(key);
if (!version_id.empty())
req.SetVersionId(version_id);
req.SetObjectAttributes({Aws::S3::Model::ObjectAttributes::ObjectSize});
auto outcome = client.GetObjectAttributes(req);
if (outcome.IsSuccess())
{
const auto & result = outcome.GetResult();
DB::S3::ObjectInfo object_info;
object_info.size = static_cast<size_t>(result.GetObjectSize());
object_info.last_modification_time = result.GetLastModified().Millis() / 1000;
return {object_info, {}};
}
return {std::nullopt, outcome.GetError()};
}
else
{
/// By default we use `HeadObject` requests.
/// We cannot just use `GetObjectAttributes` requests always because some S3 providers (e.g. Minio)
/// don't support `GetObjectAttributes` requests.
ProfileEvents::increment(ProfileEvents::S3HeadObject);
if (for_disk_s3)
ProfileEvents::increment(ProfileEvents::DiskS3HeadObject);
Aws::S3::Model::HeadObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
if (!version_id.empty())
req.SetVersionId(version_id);
auto outcome = client.HeadObject(req);
if (outcome.IsSuccess())
{
const auto & result = outcome.GetResult();
DB::S3::ObjectInfo object_info;
object_info.size = static_cast<size_t>(result.GetContentLength());
object_info.last_modification_time = result.GetLastModified().Millis() / 1000;
return {object_info, {}};
}
return {std::nullopt, outcome.GetError()};
}
}
}
@ -894,54 +986,33 @@ namespace S3
return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || error == Aws::S3::S3Errors::NO_SUCH_KEY;
}
Aws::S3::Model::HeadObjectOutcome headObject(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3)
ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3, bool throw_on_error)
{
ProfileEvents::increment(ProfileEvents::S3HeadObject);
if (for_disk_s3)
ProfileEvents::increment(ProfileEvents::DiskS3HeadObject);
Aws::S3::Model::HeadObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
if (!version_id.empty())
req.SetVersionId(version_id);
return client.HeadObject(req);
}
S3::ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error, bool for_disk_s3)
{
auto outcome = headObject(client, bucket, key, version_id, for_disk_s3);
if (outcome.IsSuccess())
auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, for_disk_s3);
if (object_info)
{
auto read_result = outcome.GetResultWithOwnership();
return {.size = static_cast<size_t>(read_result.GetContentLength()), .last_modification_time = read_result.GetLastModified().Millis() / 1000};
return *object_info;
}
else if (throw_on_error)
{
const auto & error = outcome.GetError();
throw DB::Exception(ErrorCodes::S3_ERROR,
"Failed to HEAD object: {}. HTTP response code: {}",
"Failed to get object attributes: {}. HTTP response code: {}",
error.GetMessage(), static_cast<size_t>(error.GetResponseCode()));
}
return {};
}
size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error, bool for_disk_s3)
size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3, bool throw_on_error)
{
return getObjectInfo(client, bucket, key, version_id, throw_on_error, for_disk_s3).size;
return getObjectInfo(client, bucket, key, version_id, for_disk_s3, throw_on_error).size;
}
bool objectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3)
{
auto outcome = headObject(client, bucket, key, version_id, for_disk_s3);
if (outcome.IsSuccess())
auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, for_disk_s3);
if (object_info)
return true;
const auto & error = outcome.GetError();
if (isNotFoundError(error.GetErrorType()))
return false;
@ -949,6 +1020,48 @@ namespace S3
"Failed to check existence of key {} in bucket {}: {}",
key, bucket, error.GetMessage());
}
void checkObjectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3, std::string_view description)
{
auto [object_info, error] = tryGetObjectInfo(client, bucket, key, version_id, for_disk_s3);
if (object_info)
return;
throw S3Exception(error.GetErrorType(), "{}Object {} in bucket {} suddenly disappeared: {}",
(description.empty() ? "" : (String(description) + ": ")), key, bucket, error.GetMessage());
}
std::map<String, String> getObjectMetadata(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3, bool throw_on_error)
{
ProfileEvents::increment(ProfileEvents::S3GetObjectMetadata);
if (for_disk_s3)
ProfileEvents::increment(ProfileEvents::DiskS3GetObjectMetadata);
/// We must not use the `HeadObject` request, see the comment about `HeadObjectRequest` in S3Common.h.
Aws::S3::Model::GetObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
/// Only the first byte will be read.
/// We don't need that first byte but the range should be set otherwise the entire object will be read.
req.SetRange("bytes=0-0");
if (!version_id.empty())
req.SetVersionId(version_id);
auto outcome = client.GetObject(req);
if (outcome.IsSuccess())
return outcome.GetResult().GetMetadata();
if (!throw_on_error)
return {};
const auto & error = outcome.GetError();
throw S3Exception(error.GetErrorType(),
"Failed to get metadata of key {} in bucket {}: {}",
key, bucket, error.GetMessage());
}
}
}

View File

@ -11,15 +11,15 @@
#if USE_AWS_S3
#include <base/types.h>
#include <aws/core/Aws.h>
#include <aws/core/client/ClientConfiguration.h>
#include <aws/s3/S3Client.h>
#include <aws/s3/S3Errors.h>
#include <Poco/URI.h>
#include <Common/Exception.h>
#include <Common/Throttler_fwd.h>
#include <Poco/URI.h>
#include <aws/core/Aws.h>
#include <aws/s3/S3Errors.h>
namespace Aws::S3 { class S3Client; }
namespace DB
{
@ -121,22 +121,29 @@ struct URI
static void validateBucket(const String & bucket, const Poco::URI & uri);
};
/// WARNING: Don't use `HeadObjectRequest`! Use the functions below instead.
/// For explanation see the comment about `HeadObject` request in the function tryGetObjectInfo().
struct ObjectInfo
{
size_t size = 0;
time_t last_modification_time = 0;
};
bool isNotFoundError(Aws::S3::S3Errors error);
ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false, bool throw_on_error = true);
Aws::S3::Model::HeadObjectOutcome headObject(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false);
S3::ObjectInfo getObjectInfo(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error, bool for_disk_s3);
size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool throw_on_error, bool for_disk_s3);
size_t getObjectSize(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false, bool throw_on_error = true);
bool objectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false);
/// Throws an exception if a specified object doesn't exist. `description` is used as a part of the error message.
void checkObjectExists(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false, std::string_view description = {});
bool isNotFoundError(Aws::S3::S3Errors error);
/// Returns the object's metadata.
std::map<String, String> getObjectMetadata(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id = "", bool for_disk_s3 = false, bool throw_on_error = true);
}
#endif

View File

@ -182,12 +182,8 @@ void WriteBufferFromS3::finalizeImpl()
if (check_objects_after_upload)
{
LOG_TRACE(log, "Checking object {} exists after upload", key);
auto response = S3::headObject(*client_ptr, bucket, key, "", write_settings.for_object_storage);
if (!response.IsSuccess())
throw S3Exception(fmt::format("Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", key, bucket), response.GetError().GetErrorType());
else
LOG_TRACE(log, "Object {} exists after upload", key);
S3::checkObjectExists(*client_ptr, bucket, key, {}, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload");
LOG_TRACE(log, "Object {} exists after upload", key);
}
}

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

@ -487,7 +487,7 @@ BlockIO InterpreterSystemQuery::execute()
dropDatabaseReplica(query);
break;
case Type::SYNC_REPLICA:
syncReplica(query);
syncReplica();
break;
case Type::SYNC_DATABASE_REPLICA:
syncReplicatedDatabase(query);
@ -507,6 +507,9 @@ BlockIO InterpreterSystemQuery::execute()
case Type::RESTORE_REPLICA:
restoreReplica();
break;
case Type::WAIT_LOADING_PARTS:
waitLoadingParts();
break;
case Type::RESTART_DISK:
restartDisk(query.disk);
case Type::FLUSH_LOGS:
@ -852,7 +855,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query)
throw Exception("Invalid query", ErrorCodes::LOGICAL_ERROR);
}
void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
void InterpreterSystemQuery::syncReplica()
{
getContext()->checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
@ -872,6 +875,23 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs());
}
void InterpreterSystemQuery::waitLoadingParts()
{
getContext()->checkAccess(AccessType::SYSTEM_WAIT_LOADING_PARTS, table_id);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
if (auto * merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
LOG_TRACE(log, "Waiting for loading of parts of table {}", table_id.getFullTableName());
merge_tree->waitForOutdatedPartsToBeLoaded();
LOG_TRACE(log, "Finished waiting for loading of parts of table {}", table_id.getFullTableName());
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Command WAIT LOADING PARTS is supported only for MergeTree table, but got: {}", table->getName());
}
}
void InterpreterSystemQuery::syncReplicatedDatabase(ASTSystemQuery & query)
{
@ -1071,6 +1091,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
required_access.emplace_back(AccessType::SYSTEM_RESTART_REPLICA);
break;
}
case Type::WAIT_LOADING_PARTS:
{
required_access.emplace_back(AccessType::SYSTEM_WAIT_LOADING_PARTS, query.getDatabase(), query.getTable());
break;
}
case Type::SYNC_DATABASE_REPLICA:
{
required_access.emplace_back(AccessType::SYSTEM_SYNC_DATABASE_REPLICA, query.getDatabase());

View File

@ -56,7 +56,8 @@ private:
void restartReplica(const StorageID & replica, ContextMutablePtr system_context);
void restartReplicas(ContextMutablePtr system_context);
void syncReplica(ASTSystemQuery & query);
void syncReplica();
void waitLoadingParts();
void syncReplicatedDatabase(ASTSystemQuery & query);

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

@ -303,7 +303,6 @@ bool MergeTreeTransaction::rollback() noexcept
part->version.unlockRemovalTID(tid, TransactionInfoContext{part->storage.getStorageID(), part->name});
}
assert([&]()
{
std::lock_guard lock{mutex};

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

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

@ -166,6 +166,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
else if ( type == Type::RESTART_REPLICA
|| type == Type::RESTORE_REPLICA
|| type == Type::SYNC_REPLICA
|| type == Type::WAIT_LOADING_PARTS
|| type == Type::FLUSH_DISTRIBUTED
|| type == Type::RELOAD_DICTIONARY
|| type == Type::RELOAD_MODEL

View File

@ -35,6 +35,7 @@ public:
RESTART_REPLICAS,
RESTART_REPLICA,
RESTORE_REPLICA,
WAIT_LOADING_PARTS,
DROP_REPLICA,
DROP_DATABASE_REPLICA,
SYNC_REPLICA,

View File

@ -1657,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;
@ -1672,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)
@ -1684,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;
}
@ -1714,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;
}
@ -1751,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;
@ -1781,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

@ -253,6 +253,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::RESTART_REPLICA:
case Type::SYNC_REPLICA:
case Type::WAIT_LOADING_PARTS:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;

View File

@ -304,7 +304,9 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
};
}
if (null_as_default)
/// If the Union is ['Null', Nested-Type], since the Nested-Type can not be inside
/// Nullable, so we will get Nested-Type, instead of Nullable type.
if (null_as_default || !target.isNullable())
{
auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type);
return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder)
@ -1001,7 +1003,7 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node)
case avro::Type::AVRO_STRING:
return std::make_shared<DataTypeString>();
case avro::Type::AVRO_BYTES:
return std::make_shared<DataTypeFloat32>();
return std::make_shared<DataTypeString>();
case avro::Type::AVRO_ENUM:
{
if (node->names() < 128)

View File

@ -402,14 +402,10 @@ void TCPHandler::runImpl()
{
auto callback = [this]()
{
{
std::lock_guard task_callback_lock(task_callback_mutex);
std::scoped_lock lock(task_callback_mutex, fatal_error_mutex);
if (isQueryCancelled())
return true;
}
std::lock_guard lock(fatal_error_mutex);
if (isQueryCancelled())
return true;
sendProgress();
sendSelectProfileEvents();
@ -424,6 +420,9 @@ void TCPHandler::runImpl()
}
state.io.onFinish();
std::lock_guard lock(task_callback_mutex);
/// Send final progress after calling onFinish(), since it will update the progress.
///
/// NOTE: we cannot send Progress for regular INSERT (with VALUES)
@ -446,8 +445,11 @@ void TCPHandler::runImpl()
if (state.is_connection_closed)
break;
sendLogs();
sendEndOfStream();
{
std::lock_guard lock(task_callback_mutex);
sendLogs();
sendEndOfStream();
}
/// QueryState should be cleared before QueryScope, since otherwise
/// the MemoryTracker will be wrong for possible deallocations.
@ -760,6 +762,9 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
}
}
/// Defer locking to cover a part of the scope below and everything after it
std::unique_lock progress_lock(task_callback_mutex, std::defer_lock);
{
PullingAsyncPipelineExecutor executor(pipeline);
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
@ -796,6 +801,11 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
}
}
/// This lock wasn't acquired before and we make .lock() call here
/// so everything under this line is covered even together
/// with sendProgress() out of the scope
progress_lock.lock();
/** If data has run out, we will send the profiling data and total values to
* the last zero block to be able to use
* this information in the suffix output of stream.

File diff suppressed because it is too large Load Diff

View File

@ -1052,6 +1052,8 @@ public:
/// Returns an object that protects temporary directory from cleanup
scope_guard getTemporaryPartDirectoryHolder(const String & part_dir_name) const;
void waitForOutdatedPartsToBeLoaded() const;
protected:
friend class IMergeTreeDataPart;
friend class MergeTreeDataMergerMutator;
@ -1068,7 +1070,6 @@ protected:
/// under lockForShare if rename is possible.
String relative_data_path;
/// Current column sizes in compressed and uncompressed form.
ColumnSizeByName column_sizes;
@ -1330,6 +1331,88 @@ protected:
void resetObjectColumnsFromActiveParts(const DataPartsLock & lock);
void updateObjectColumns(const DataPartPtr & part, const DataPartsLock & lock);
/** A structure that explicitly represents a "merge tree" of parts
* which is implicitly presented by min-max block numbers and levels of parts.
* The children of node are parts which are covered by parent part.
* This tree provides the order of loading of parts.
*
* We start to traverse tree from the top level and load parts
* corresposponded to nodes. If part is loaded successfully then
* we stop traversal at this node. Otherwise part is broken and we
* traverse its children and try to load covered parts which will
* replace broken covering part. Unloaded nodes represent outdated parts
* nd they are pushed to background task and loaded asynchronoulsy.
*/
class PartLoadingTree
{
public:
struct Node
{
Node(const MergeTreePartInfo & info_, const String & name_, const DiskPtr & disk_)
: info(info_), name(name_), disk(disk_)
{
}
const MergeTreePartInfo info;
const String name;
const DiskPtr disk;
bool is_loaded = false;
std::map<MergeTreePartInfo, std::shared_ptr<Node>> children;
};
struct PartLoadingInfo
{
PartLoadingInfo(const MergeTreePartInfo & info_, const String & name_, const DiskPtr & disk_)
: info(info_), name(name_), disk(disk_)
{
}
/// Store name explicitly because it cannot be easily
/// retrieved from info in tables with old syntax.
MergeTreePartInfo info;
String name;
DiskPtr disk;
};
using NodePtr = std::shared_ptr<Node>;
using PartLoadingInfos = std::vector<PartLoadingInfo>;
/// Builds a tree from the list of part infos.
static PartLoadingTree build(PartLoadingInfos nodes);
/// Traverses a tree and call @func on each node.
/// If recursive is false traverses only the top level.
template <typename Func>
void traverse(bool recursive, Func && func);
private:
/// NOTE: Parts should be added in descending order of their levels
/// because rearranging tree to the new root is not supported.
void add(const MergeTreePartInfo & info, const String & name, const DiskPtr & disk);
std::unordered_map<String, NodePtr> root_by_partition;
};
using PartLoadingTreeNodes = std::vector<PartLoadingTree::NodePtr>;
struct LoadPartResult
{
bool is_broken = false;
std::optional<size_t> size_of_part;
MutableDataPartPtr part;
};
mutable std::mutex outdated_data_parts_mutex;
mutable std::condition_variable outdated_data_parts_cv;
BackgroundSchedulePool::TaskHolder outdated_data_parts_loading_task;
PartLoadingTreeNodes outdated_unloaded_data_parts TSA_GUARDED_BY(outdated_data_parts_mutex);
bool outdated_data_parts_loading_canceled TSA_GUARDED_BY(outdated_data_parts_mutex) = false;
void loadOutdatedDataParts(bool is_async);
void startOutdatedDataPartsLoadingTask();
void stopOutdatedDataPartsLoadingTask();
static void incrementInsertedPartsProfileEvent(MergeTreeDataPartType type);
static void incrementMergedPartsProfileEvent(MergeTreeDataPartType type);
@ -1408,18 +1491,20 @@ private:
/// Returns default settings for storage with possible changes from global config.
virtual std::unique_ptr<MergeTreeSettings> getDefaultSettings() const = 0;
void loadDataPartsFromDisk(
MutableDataPartsVector & broken_parts_to_detach,
MutableDataPartsVector & duplicate_parts_to_remove,
LoadPartResult loadDataPart(
const MergeTreePartInfo & part_info,
const String & part_name,
const DiskPtr & part_disk_ptr,
MergeTreeDataPartState to_state,
std::mutex & part_loading_mutex);
std::vector<LoadPartResult> loadDataPartsFromDisk(
ThreadPool & pool,
size_t num_parts,
std::queue<std::vector<std::pair<String, DiskPtr>>> & parts_queue,
bool skip_sanity_checks,
std::queue<PartLoadingTreeNodes> & parts_queue,
const MergeTreeSettingsPtr & settings);
void loadDataPartsFromWAL(
MutableDataPartsVector & duplicate_parts_to_remove,
MutableDataPartsVector & parts_from_wal);
void loadDataPartsFromWAL(MutableDataPartsVector & parts_from_wal);
/// Create zero-copy exclusive lock for part and disk. Useful for coordination of
/// distributed operations which can lead to data duplication. Implemented only in ReplicatedMergeTree.
@ -1430,7 +1515,7 @@ private:
/// Otherwise, in non-parallel case will break and return.
void clearPartsFromFilesystemImpl(const DataPartsVector & parts, NameSet * part_names_succeed);
static MutableDataPartPtr preparePartForRemoval(const DataPartPtr & part);
static MutableDataPartPtr asMutableDeletingPart(const DataPartPtr & part);
mutable TemporaryParts temporary_parts;
};

View File

@ -147,7 +147,6 @@ bool ReplicatedMergeTreeRestartingThread::runImpl()
storage.part_check_thread.start();
LOG_DEBUG(log, "Table started successfully");
return true;
}

View File

@ -112,7 +112,6 @@ StorageMergeTree::StorageMergeTree(
increment.set(getMaxBlockNumber());
loadMutations();
loadDeduplicationLog();
}
@ -138,6 +137,7 @@ void StorageMergeTree::startup()
{
background_operations_assignee.start();
startBackgroundMovesIfNeeded();
startOutdatedDataPartsLoadingTask();
}
catch (...)
{
@ -171,6 +171,8 @@ void StorageMergeTree::shutdown()
if (shutdown_called.exchange(true))
return;
stopOutdatedDataPartsLoadingTask();
/// Unlock all waiting mutations
{
std::lock_guard lock(mutation_wait_mutex);
@ -1189,6 +1191,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign
scheduled = true;
}
return scheduled;
}
@ -1509,6 +1512,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
waitForOutdatedPartsToBeLoaded();
Stopwatch watch;

View File

@ -907,6 +907,16 @@ void StorageReplicatedMergeTree::drop()
dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings());
}
/// Wait for loading of all outdated parts because
/// in case of zero copy recursive removal of directory
/// is not supported and table cannot be dropped.
if (canUseZeroCopyReplication())
{
/// Load remaining parts synchronously because task
/// for loading is already cancelled in shutdown().
loadOutdatedDataParts(/*is_async=*/ false);
}
dropAllData();
}
@ -1866,6 +1876,11 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::executeFetchShared(
void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
{
LOG_TRACE(log, "Executing DROP_RANGE {}", entry.new_part_name);
/// Wait for loading of outdated parts because DROP_RANGE
/// command must be applied to all parts on disk.
waitForOutdatedPartsToBeLoaded();
auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version);
getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range_info.partition_id, drop_range_info.max_block);
queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry, /* fetch_entry_znode= */ {});
@ -1930,6 +1945,11 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
LOG_DEBUG(log, "Executing log entry {} to replace parts range {} with {} parts from {}.{}",
entry.znode_name, entry_replace.drop_range_part_name, entry_replace.new_part_names.size(),
entry_replace.from_database, entry_replace.from_table);
/// Wait for loading of outdated parts because REPLACE_RANGE
/// command must be applied to all parts on disk.
waitForOutdatedPartsToBeLoaded();
auto metadata_snapshot = getInMemoryMetadataPtr();
auto storage_settings_ptr = getSettings();
@ -4286,6 +4306,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart(
void StorageReplicatedMergeTree::startup()
{
startOutdatedDataPartsLoadingTask();
if (attach_thread)
{
attach_thread->start();
@ -4402,6 +4423,7 @@ void StorageReplicatedMergeTree::shutdown()
return;
session_expired_callback_handler.reset();
stopOutdatedDataPartsLoadingTask();
/// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
fetcher.blocker.cancelForever();
@ -5295,7 +5317,6 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
if (!is_readonly)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly");
if (getZooKeeper()->exists(replica_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Replica path is present at {} - nothing to restore. "
@ -5312,6 +5333,7 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
auto metadata_snapshot = getInMemoryMetadataPtr();
waitForOutdatedPartsToBeLoaded();
const DataPartsVector all_parts = getAllDataPartsVector();
Strings active_parts_names;
@ -5426,6 +5448,7 @@ void StorageReplicatedMergeTree::truncate(
if (!is_leader)
throw Exception("TRUNCATE cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER);
waitForOutdatedPartsToBeLoaded();
zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly();
dropAllPartitionsImpl(zookeeper, /* detach */ false, query_context);
}

View File

@ -444,7 +444,7 @@ public:
/// (which means we eventually need this info anyway, so it should be ok to do it now)
if (object_infos_)
{
info = S3::getObjectInfo(client_, bucket, key, version_id_, true, false);
info = S3::getObjectInfo(client_, bucket, key, version_id_);
total_size += info->size;
String path = fs::path(bucket) / key;
@ -569,9 +569,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader()
if (current_key.empty())
return {};
size_t object_size = info
? info->size
: S3::getObjectSize(*client, bucket, current_key, version_id, true, false);
size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id);
int zstd_window_log_max = static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max);
auto read_buf = wrapReadBufferWithCompressionMethod(
@ -1523,7 +1521,7 @@ std::optional<ColumnsDescription> StorageS3::tryGetColumnsFromCache(
/// Note that in case of exception in getObjectInfo returned info will be empty,
/// but schema cache will handle this case and won't return columns from cache
/// because we can't say that it's valid without last modification time.
info = S3::getObjectInfo(*s3_configuration.client, s3_configuration.uri.bucket, *it, s3_configuration.uri.version_id, false, false);
info = S3::getObjectInfo(*s3_configuration.client, s3_configuration.uri.bucket, *it, s3_configuration.uri.version_id, {}, /* throw_on_error= */ false);
if (object_infos)
(*object_infos)[path] = info;
}

View File

@ -176,6 +176,24 @@ private:
}
ReaderHolder() = default;
ReaderHolder(const ReaderHolder & other) = delete;
ReaderHolder & operator=(const ReaderHolder & other) = delete;
ReaderHolder(ReaderHolder && other) noexcept
{
*this = std::move(other);
}
ReaderHolder & operator=(ReaderHolder && other) noexcept
{
/// The order of destruction is important.
/// reader uses pipeline, pipeline uses read_buf.
reader = std::move(other.reader);
pipeline = std::move(other.pipeline);
read_buf = std::move(other.read_buf);
path = std::move(other.path);
return *this;
}
explicit operator bool() const { return reader != nullptr; }
PullingPipelineExecutor * operator->() { return reader.get(); }

View File

@ -38,13 +38,18 @@ class Reviews:
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)
if r.state not in self.STATES:
continue
self._review_per_user[user] = r
if r.state == "APPROVED":
self.approved_at = max(r.submitted_at, self.approved_at)
if not self._review_per_user.get(user):
self._review_per_user[user] = r
continue
if r.submitted_at < self._review_per_user[user].submitted_at:
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"""

View File

@ -181,7 +181,7 @@ def test_grant_all_on_table():
== "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, "
"DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, "
"SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, "
"SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n"
"SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n"
)
instance.query("REVOKE ALL ON test.table FROM B", user="A")
assert instance.query("SHOW GRANTS FOR B") == ""

View File

@ -0,0 +1,9 @@
<clickhouse>
<background_processing_pool_thread_sleep_seconds>1</background_processing_pool_thread_sleep_seconds>
<background_processing_pool_thread_sleep_seconds_random_part>0</background_processing_pool_thread_sleep_seconds_random_part>
<background_processing_pool_thread_sleep_seconds_if_nothing_to_do>0.0</background_processing_pool_thread_sleep_seconds_if_nothing_to_do>
<background_processing_pool_task_sleep_seconds_when_no_work_min>0</background_processing_pool_task_sleep_seconds_when_no_work_min>
<background_processing_pool_task_sleep_seconds_when_no_work_max>1</background_processing_pool_task_sleep_seconds_when_no_work_max>
<background_processing_pool_task_sleep_seconds_when_no_work_multiplier>1</background_processing_pool_task_sleep_seconds_when_no_work_multiplier>
<background_processing_pool_task_sleep_seconds_when_no_work_random_part>0</background_processing_pool_task_sleep_seconds_when_no_work_random_part>
</clickhouse>

View File

@ -0,0 +1,196 @@
import pytest
import helpers.client
import helpers.cluster
import time
from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk
cluster = helpers.cluster.ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
main_configs=["configs/fast_background_pool.xml"],
with_zookeeper=True,
stay_alive=True,
)
node2 = cluster.add_instance(
"node2",
main_configs=["configs/fast_background_pool.xml"],
with_zookeeper=True,
stay_alive=True,
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_merge_tree_load_parts(started_cluster):
node1.query(
"""
CREATE TABLE mt_load_parts (pk UInt32, id UInt32, s String)
ENGINE = MergeTree ORDER BY id PARTITION BY pk"""
)
node1.query("SYSTEM STOP MERGES mt_load_parts")
for i in range(20):
node1.query(
f"INSERT INTO mt_load_parts VALUES (44, {i}, randomPrintableASCII(10))"
)
node1.restart_clickhouse(kill=True)
for i in range(1, 21):
assert node1.contains_in_log(f"Loading Active part 44_{i}_{i}_0")
node1.query("OPTIMIZE TABLE mt_load_parts FINAL")
node1.restart_clickhouse(kill=True)
node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts")
assert node1.contains_in_log("Loading Active part 44_1_20")
for i in range(1, 21):
assert not node1.contains_in_log(f"Loading Active part 44_{i}_{i}_0")
assert node1.contains_in_log(f"Loading Outdated part 44_{i}_{i}_0")
assert node1.query("SELECT count() FROM mt_load_parts") == "20\n"
assert (
node1.query(
"SELECT count() FROM system.parts WHERE table = 'mt_load_parts' AND active"
)
== "1\n"
)
node1.query("ALTER TABLE mt_load_parts MODIFY SETTING old_parts_lifetime = 1")
node1.query("DETACH TABLE mt_load_parts")
node1.query("ATTACH TABLE mt_load_parts")
node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts")
table_path = node1.query(
"SELECT data_paths[1] FROM system.tables WHERE table = 'mt_load_parts'"
).strip()
part_dirs = node1.exec_in_container(["bash", "-c", f"ls {table_path}"], user="root")
part_dirs = list(
set(part_dirs.strip().split("\n")) - {"detached", "format_version.txt"}
)
MAX_RETRY = 10
part_dirs_ok = False
for _ in range(MAX_RETRY):
part_dirs = node1.exec_in_container(
["bash", "-c", f"ls {table_path}"], user="root"
)
part_dirs = list(
set(part_dirs.strip().split("\n")) - {"detached", "format_version.txt"}
)
part_dirs_ok = len(part_dirs) == 1 and part_dirs[0].startswith("44_1_20")
if part_dirs_ok:
break
time.sleep(2)
assert part_dirs_ok
def test_merge_tree_load_parts_corrupted(started_cluster):
for i, node in enumerate([node1, node2]):
node.query(
f"""
CREATE TABLE mt_load_parts_2 (pk UInt32, id UInt32, s String)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/mt_load_parts_2', '{i}') ORDER BY id PARTITION BY pk"""
)
"""min-max blocks in created parts: 1_1_0, 2_2_0, 1_2_1, 3_3_0, 1_3_2"""
for partition in [111, 222, 333]:
node1.query(
f"INSERT INTO mt_load_parts_2 VALUES ({partition}, 0, randomPrintableASCII(10))"
)
node1.query(
f"INSERT INTO mt_load_parts_2 VALUES ({partition}, 1, randomPrintableASCII(10))"
)
node1.query(f"OPTIMIZE TABLE mt_load_parts_2 PARTITION {partition} FINAL")
node1.query(
f"INSERT INTO mt_load_parts_2 VALUES ({partition}, 2, randomPrintableASCII(10))"
)
node1.query(f"OPTIMIZE TABLE mt_load_parts_2 PARTITION {partition} FINAL")
node2.query("SYSTEM SYNC REPLICA mt_load_parts_2", timeout=30)
def get_part_name(node, partition, min_block, max_block):
return node.query(
f"""
SELECT name FROM system.parts
WHERE table = 'mt_load_parts_2'
AND partition = '{partition}'
AND min_block_number = {min_block}
AND max_block_number = {max_block}"""
).strip()
corrupt_part_data_on_disk(node1, "mt_load_parts_2", get_part_name(node1, 111, 0, 2))
corrupt_part_data_on_disk(node1, "mt_load_parts_2", get_part_name(node1, 222, 0, 2))
corrupt_part_data_on_disk(node1, "mt_load_parts_2", get_part_name(node1, 222, 0, 1))
corrupt_part_data_on_disk(node1, "mt_load_parts_2", get_part_name(node1, 333, 0, 1))
corrupt_part_data_on_disk(node1, "mt_load_parts_2", get_part_name(node1, 333, 2, 2))
node1.restart_clickhouse(kill=True)
node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2")
def check_parts_loading(node, partition, loaded, failed, skipped):
for (min_block, max_block) in loaded:
part_name = f"{partition}_{min_block}_{max_block}"
assert node.contains_in_log(f"Loading Active part {part_name}")
assert node.contains_in_log(f"Finished loading Active part {part_name}")
for (min_block, max_block) in failed:
part_name = f"{partition}_{min_block}_{max_block}"
assert node.contains_in_log(f"Loading Active part {part_name}")
assert not node.contains_in_log(f"Finished loading Active part {part_name}")
for (min_block, max_block) in skipped:
part_name = f"{partition}_{min_block}_{max_block}"
assert not node.contains_in_log(f"Loading Active part {part_name}")
assert not node.contains_in_log(f"Finished loading Active part {part_name}")
check_parts_loading(
node1, 111, loaded=[(0, 1), (2, 2)], failed=[(0, 2)], skipped=[(0, 0), (1, 1)]
)
check_parts_loading(
node1, 222, loaded=[(0, 0), (1, 1), (2, 2)], failed=[(0, 2), (0, 1)], skipped=[]
)
check_parts_loading(
node1, 333, loaded=[(0, 2)], failed=[], skipped=[(0, 0), (1, 1), (2, 2), (0, 1)]
)
node1.query("SYSTEM SYNC REPLICA mt_load_parts_2", timeout=30)
node1.query("OPTIMIZE TABLE mt_load_parts_2 FINAL")
node1.query("SYSTEM SYNC REPLICA mt_load_parts_2", timeout=30)
assert (
node1.query(
"""
SELECT pk, count() FROM mt_load_parts_2
GROUP BY pk ORDER BY pk"""
)
== "111\t3\n222\t3\n333\t3\n"
)
assert (
node1.query(
"""
SELECT partition, count()
FROM system.parts WHERE table = 'mt_load_parts_2' AND active
GROUP BY partition ORDER BY partition"""
)
== "111\t1\n222\t1\n333\t1\n"
)

View File

@ -26,7 +26,7 @@ def tx(session, query):
def test_rollback_unfinished_on_restart1(start_cluster):
node.query(
"create table mt (n int, m int) engine=MergeTree order by n partition by n % 2"
"create table mt (n int, m int) engine=MergeTree order by n partition by n % 2 settings remove_empty_parts = 0"
)
node.query("insert into mt values (1, 10), (2, 20)")
tid0 = "(1,1,'00000000-0000-0000-0000-000000000000')"
@ -82,6 +82,7 @@ def test_rollback_unfinished_on_restart1(start_cluster):
).strip()
node.restart_clickhouse(kill=True)
node.query("SYSTEM WAIT LOADING PARTS mt")
assert (
node.query("select *, _part from mt order by n")
@ -116,7 +117,7 @@ def test_rollback_unfinished_on_restart1(start_cluster):
def test_rollback_unfinished_on_restart2(start_cluster):
node.query(
"create table mt2 (n int, m int) engine=MergeTree order by n partition by n % 2"
"create table mt2 (n int, m int) engine=MergeTree order by n partition by n % 2 settings remove_empty_parts = 0"
)
node.query("insert into mt2 values (1, 10), (2, 20)")
tid0 = "(1,1,'00000000-0000-0000-0000-000000000000')"
@ -171,6 +172,7 @@ def test_rollback_unfinished_on_restart2(start_cluster):
).strip()
node.restart_clickhouse(kill=True)
node.query("SYSTEM WAIT LOADING PARTS mt2")
assert (
node.query("select *, _part from mt2 order by n")

View File

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

View File

@ -13,6 +13,7 @@
<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

@ -121,6 +121,7 @@ SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM
SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM
SYSTEM RESTART REPLICA ['RESTART REPLICA'] TABLE SYSTEM
SYSTEM RESTORE REPLICA ['RESTORE REPLICA'] TABLE SYSTEM
SYSTEM WAIT LOADING PARTS ['WAIT LOADING PARTS'] TABLE SYSTEM
SYSTEM SYNC DATABASE REPLICA ['SYNC DATABASE REPLICA'] DATABASE SYSTEM
SYSTEM SYNC TRANSACTION LOG ['SYNC TRANSACTION LOG'] GLOBAL SYSTEM
SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH

View File

@ -6,8 +6,6 @@ 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
@ -16,5 +14,3 @@ 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,28 +11,24 @@ CREATE TABLE codecs
c Float32 CODEC(Gorilla),
d UInt8 CODEC(Delta, LZ4),
e Float64 CODEC(Gorilla, ZSTD),
f UInt64 CODEC(Delta, Delta, T64),
f UInt32 CODEC(Delta, Delta, Gorilla),
g DateTime CODEC(DoubleDelta),
h DateTime64 CODEC(DoubleDelta, LZ4),
i String CODEC(NONE),
j Float64 (Gorilla, Delta),
k Float32 (FPC, DoubleDelta)
i String CODEC(NONE)
) 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 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 }
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 }
-- test that sanity check is not performed in ATTACH query
@ -44,8 +40,6 @@ 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;
@ -57,8 +51,6 @@ 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;
@ -70,8 +62,6 @@ 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;
@ -81,8 +71,6 @@ DETACH TABLE codecs5;
DETACH TABLE codecs6;
DETACH TABLE codecs7;
DETACH TABLE codecs8;
DETACH TABLE codecs9;
DETACH TABLE codecs10;
ATTACH TABLE codecs1;
ATTACH TABLE codecs2;
@ -92,8 +80,6 @@ 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;
@ -103,8 +89,6 @@ 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;
@ -114,8 +98,6 @@ SELECT * FROM codecs5;
SELECT * FROM codecs6;
SELECT * FROM codecs7;
SELECT * FROM codecs8;
SELECT * FROM codecs9;
SELECT * FROM codecs10;
DROP TABLE codecs1;
DROP TABLE codecs2;
@ -125,5 +107,3 @@ DROP TABLE codecs5;
DROP TABLE codecs6;
DROP TABLE codecs7;
DROP TABLE codecs8;
DROP TABLE codecs9;
DROP TABLE codecs10;

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

@ -286,7 +286,7 @@ CREATE TABLE system.grants
(
`user_name` Nullable(String),
`role_name` Nullable(String),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM WAIT LOADING PARTS' = 123, 'SYSTEM SYNC DATABASE REPLICA' = 124, 'SYSTEM SYNC TRANSACTION LOG' = 125, 'SYSTEM FLUSH DISTRIBUTED' = 126, 'SYSTEM FLUSH LOGS' = 127, 'SYSTEM FLUSH' = 128, 'SYSTEM THREAD FUZZER' = 129, 'SYSTEM UNFREEZE' = 130, 'SYSTEM' = 131, 'dictGet' = 132, 'addressToLine' = 133, 'addressToLineWithInlines' = 134, 'addressToSymbol' = 135, 'demangle' = 136, 'INTROSPECTION' = 137, 'FILE' = 138, 'URL' = 139, 'REMOTE' = 140, 'MONGO' = 141, 'MEILISEARCH' = 142, 'MYSQL' = 143, 'POSTGRES' = 144, 'SQLITE' = 145, 'ODBC' = 146, 'JDBC' = 147, 'HDFS' = 148, 'S3' = 149, 'HIVE' = 150, 'SOURCES' = 151, 'CLUSTER' = 152, 'ALL' = 153, 'NONE' = 154),
`database` Nullable(String),
`table` Nullable(String),
`column` Nullable(String),
@ -567,10 +567,10 @@ ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
CREATE TABLE system.privileges
(
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM WAIT LOADING PARTS' = 123, 'SYSTEM SYNC DATABASE REPLICA' = 124, 'SYSTEM SYNC TRANSACTION LOG' = 125, 'SYSTEM FLUSH DISTRIBUTED' = 126, 'SYSTEM FLUSH LOGS' = 127, 'SYSTEM FLUSH' = 128, 'SYSTEM THREAD FUZZER' = 129, 'SYSTEM UNFREEZE' = 130, 'SYSTEM' = 131, 'dictGet' = 132, 'addressToLine' = 133, 'addressToLineWithInlines' = 134, 'addressToSymbol' = 135, 'demangle' = 136, 'INTROSPECTION' = 137, 'FILE' = 138, 'URL' = 139, 'REMOTE' = 140, 'MONGO' = 141, 'MEILISEARCH' = 142, 'MYSQL' = 143, 'POSTGRES' = 144, 'SQLITE' = 145, 'ODBC' = 146, 'JDBC' = 147, 'HDFS' = 148, 'S3' = 149, 'HIVE' = 150, 'SOURCES' = 151, 'CLUSTER' = 152, 'ALL' = 153, 'NONE' = 154),
`aliases` Array(String),
`level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)),
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153))
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM WAIT LOADING PARTS' = 123, 'SYSTEM SYNC DATABASE REPLICA' = 124, 'SYSTEM SYNC TRANSACTION LOG' = 125, 'SYSTEM FLUSH DISTRIBUTED' = 126, 'SYSTEM FLUSH LOGS' = 127, 'SYSTEM FLUSH' = 128, 'SYSTEM THREAD FUZZER' = 129, 'SYSTEM UNFREEZE' = 130, 'SYSTEM' = 131, 'dictGet' = 132, 'addressToLine' = 133, 'addressToLineWithInlines' = 134, 'addressToSymbol' = 135, 'demangle' = 136, 'INTROSPECTION' = 137, 'FILE' = 138, 'URL' = 139, 'REMOTE' = 140, 'MONGO' = 141, 'MEILISEARCH' = 142, 'MYSQL' = 143, 'POSTGRES' = 144, 'SQLITE' = 145, 'ODBC' = 146, 'JDBC' = 147, 'HDFS' = 148, 'S3' = 149, 'HIVE' = 150, 'SOURCES' = 151, 'CLUSTER' = 152, 'ALL' = 153, 'NONE' = 154))
)
ENGINE = SystemPrivileges
COMMENT 'SYSTEM TABLE is built on the fly.'

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

@ -0,0 +1 @@
1

View File

@ -0,0 +1,54 @@
#!/usr/bin/env bash
# Tags: zookeeper
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function query_with_retry
{
retry=0
until [ $retry -ge 5 ]
do
result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1)
if [ "$?" == 0 ]; then
echo -n "$result"
return
else
retry=$(($retry + 1))
sleep 3
fi
done
echo "Query '$1' failed with '$result'"
}
$CLICKHOUSE_CLIENT -n --query "
DROP TABLE IF EXISTS load_parts_refcounts SYNC;
CREATE TABLE load_parts_refcounts (id UInt32)
ENGINE = ReplicatedMergeTree('/test/02482_load_parts_refcounts/{database}/{table}', '1')
ORDER BY id;
SYSTEM STOP MERGES load_parts_refcounts;
INSERT INTO load_parts_refcounts VALUES (1);
INSERT INTO load_parts_refcounts VALUES (2);
INSERT INTO load_parts_refcounts VALUES (3);
SYSTEM START MERGES load_parts_refcounts;
"
query_with_retry "OPTIMIZE TABLE load_parts_refcounts FINAL SETTINGS optimize_throw_if_noop = 1"
$CLICKHOUSE_CLIENT --query "DETACH TABLE load_parts_refcounts"
$CLICKHOUSE_CLIENT --query "ATTACH TABLE load_parts_refcounts"
query_with_retry "
SELECT throwIf(count() == 0) FROM system.parts
WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'load_parts_refcounts' AND NOT active FORMAT Null"
$CLICKHOUSE_CLIENT --query "
SELECT DISTINCT refcount FROM system.parts
WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'load_parts_refcounts' AND NOT active"
$CLICKHOUSE_CLIENT --query "DROP TABLE load_parts_refcounts SYNC"

View File

@ -0,0 +1,15 @@
manifest_path String
manifest_length Int64
partition_spec_id Int32
added_snapshot_id Nullable(Int64)
added_data_files_count Nullable(Int32)
existing_data_files_count Nullable(Int32)
deleted_data_files_count Nullable(Int32)
partitions Array(Tuple(contains_null Bool, contains_nan Nullable(Bool), lower_bound Nullable(String), upper_bound Nullable(String)))
added_rows_count Nullable(Int64)
existing_rows_count Nullable(Int64)
deleted_rows_count Nullable(Int64)
file:/warehouse/nyc.db/taxis/metadata/f9e891e9-fbd3-4411-a5c6-0cc14a2f1392-m0.avro 6488 0 1793608066486471262 8 0 0 [(false,false,'\0\0\0\0\0\0\0','\b\0\0\0\0\0\0\0')] 12 0 0
file:/warehouse/nyc.db/taxis/metadata/a51dd31d-ea86-42dd-82d1-1981332a0f6d-m0.avro 6363 0 5735460159761889536 4 0 0 [(false,false,'\0\0\0\0\0\0\0','\b\0\0\0\0\0\0\0')] 4 0 0
file:/warehouse/nyc.db/taxis/metadata/7ae325bd-fe20-4a55-917c-36cb8f6a488c-m0.avro 6370 0 7171740521400098346 4 0 0 [(false,false,'\0\0\0\0\0\0\0','\0\0\0\0\0\0\0')] 4 0 0
file:/warehouse/nyc.db/taxis/metadata/5e3c62a9-1537-455f-98e5-0a067af5752a-m0.avro 6324 0 6850377589038341628 2 0 0 [(false,false,'\0\0\0\0\0\0\0','\0\0\0\0\0\0\0')] 4 0 0

View File

@ -0,0 +1,13 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
set -e
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
DATA_DIR=$CUR_DIR/data_avro
$CLICKHOUSE_LOCAL -q "desc file('$DATA_DIR/union_null_nested.avro')"
$CLICKHOUSE_LOCAL -q "select * from file('$DATA_DIR/union_null_nested.avro')"

View File

@ -0,0 +1,5 @@
status Int32
snapshot_id Nullable(Int64)
data_file Tuple(file_path String, file_format String, partition Tuple(vendor_id Nullable(Int64)), record_count Int64, file_size_in_bytes Int64, block_size_in_bytes Int64, column_sizes Array(Tuple(key Int32, value Int64)), value_counts Array(Tuple(key Int32, value Int64)), null_value_counts Array(Tuple(key Int32, value Int64)), nan_value_counts Array(Tuple(key Int32, value Int64)), lower_bounds Array(Tuple(key Int32, value String)), upper_bounds Array(Tuple(key Int32, value String)), key_metadata Nullable(String), split_offsets Array(Int64), sort_order_id Nullable(Int32))
1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=1/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00001.parquet','PARQUET',(1),2,1565,67108864,[(1,87),(2,51),(3,51),(4,57),(5,51)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'³C\0\0\0\0\0'),(3,'ffæ?'),(4,'¤p=\n×£.@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'¶C\0\0\0\0\0'),(3,'ffA'),(4,'q=\n×£E@'),(5,'Y')],NULL,[4],0)
1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=2/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00002.parquet','PARQUET',(2),2,1620,67108864,[(1,87),(2,51),(3,51),(4,57),(5,89)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'´C\0\0\0\0\0'),(3,'fff?'),(4,'…ëQ¸"@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'µC\0\0\0\0\0'),(3,'\0\0 @'),(4,'fffff&6@'),(5,'N')],NULL,[4],0)

View File

@ -0,0 +1,13 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
set -e
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
DATA_DIR=$CUR_DIR/data_avro
$CLICKHOUSE_LOCAL -q "desc file('$DATA_DIR/complicated_schema.avro')"
$CLICKHOUSE_LOCAL -q "select * from file('$DATA_DIR/complicated_schema.avro')"

View File

@ -0,0 +1,6 @@
(1,2,NULL)
(1,2,NULL)
(NULL,NULL,NULL)
1
1
1

View File

@ -0,0 +1,19 @@
#!/usr/bin/env bash
# Tags: long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo '{"t" : {"a" : 1, "b" : 2}}' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table"
echo '{"t" : { "a" : 1 , "b" : 2 } }' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table"
echo '{"t" : {}}' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table"
echo '{"t" : {"a" : 1, "b" : 2}}' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table" --input_format_json_defaults_for_missing_elements_in_named_tuple=0 2>&1 | grep -F "INCORRECT_DATA" -c
echo '{"t" : {"a" : 1, "d" : 2}}' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table" 2>&1 | grep -F "NOT_FOUND_COLUMN_IN_BLOCK" -c
echo '{"t" : {"a" : 1, "b" : 2, "c" : 3, "d" : 4}}' | $CLICKHOUSE_LOCAL --input-format=NDJSON --structure='t Tuple(a Nullable(UInt32), b Nullable(UInt32), c Nullable(UInt32))' -q "select * from table" 2>&1 | grep -F "INCORRECT_DATA" -c

View File

@ -1,4 +0,0 @@
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

@ -0,0 +1,4 @@
1 2 3
4 5 6
7 8 9
0 0 0

View File

@ -0,0 +1,5 @@
-- Tags: no-fasttest
-- Tag no-fasttest: Depends on AWS
select * from s3('http://localhost:11111/test/a.tsv', CustomSeparated);

Some files were not shown because too many files have changed in this diff Show More