mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge remote-tracking branch 'origin/master' into make-rabbitmq-reject-broken-messages
This commit is contained in:
commit
29ae0800b0
@ -123,7 +123,6 @@
|
||||
* Something was wrong with Apache Hive, which is experimental and not supported. [#60262](https://github.com/ClickHouse/ClickHouse/pull/60262) ([shanfengp](https://github.com/Aed-p)).
|
||||
* An improvement for experimental parallel replicas: force reanalysis if parallel replicas changed [#60362](https://github.com/ClickHouse/ClickHouse/pull/60362) ([Raúl Marín](https://github.com/Algunenano)).
|
||||
* Fix usage of plain metadata type with new disks configuration option [#60396](https://github.com/ClickHouse/ClickHouse/pull/60396) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
* Don't allow to set max_parallel_replicas to 0 as it doesn't make sense [#60430](https://github.com/ClickHouse/ClickHouse/pull/60430) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike [#60451](https://github.com/ClickHouse/ClickHouse/pull/60451) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Avoid calculation of scalar subqueries for CREATE TABLE. [#60464](https://github.com/ClickHouse/ClickHouse/pull/60464) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
* Fix deadlock in parallel parsing when lots of rows are skipped due to errors [#60516](https://github.com/ClickHouse/ClickHouse/pull/60516) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
|
@ -314,13 +314,13 @@ static int read_unicode(json_stream *json)
|
||||
|
||||
if (l < 0xdc00 || l > 0xdfff) {
|
||||
json_error(json, "invalid surrogate pair continuation \\u%04lx out "
|
||||
"of range (dc00-dfff)", l);
|
||||
"of range (dc00-dfff)", (unsigned long)l);
|
||||
return -1;
|
||||
}
|
||||
|
||||
cp = ((h - 0xd800) * 0x400) + ((l - 0xdc00) + 0x10000);
|
||||
} else if (cp >= 0xdc00 && cp <= 0xdfff) {
|
||||
json_error(json, "dangling surrogate \\u%04lx", cp);
|
||||
json_error(json, "dangling surrogate \\u%04lx", (unsigned long)cp);
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
2
contrib/NuRaft
vendored
2
contrib/NuRaft
vendored
@ -1 +1 @@
|
||||
Subproject commit 4a12f99dfc9d47c687ff7700b927cc76856225d1
|
||||
Subproject commit 08ac76ea80a37f89b12109c805eafe9f1dc9b991
|
@ -32,6 +32,7 @@ set(SRCS
|
||||
"${LIBRARY_DIR}/src/handle_custom_notification.cxx"
|
||||
"${LIBRARY_DIR}/src/handle_vote.cxx"
|
||||
"${LIBRARY_DIR}/src/launcher.cxx"
|
||||
"${LIBRARY_DIR}/src/log_entry.cxx"
|
||||
"${LIBRARY_DIR}/src/srv_config.cxx"
|
||||
"${LIBRARY_DIR}/src/snapshot_sync_req.cxx"
|
||||
"${LIBRARY_DIR}/src/snapshot_sync_ctx.cxx"
|
||||
|
@ -36,7 +36,7 @@ E.g. configuration option
|
||||
<s3>
|
||||
<type>s3</type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3>
|
||||
```
|
||||
|
||||
@ -47,7 +47,7 @@ is equal to configuration (from `24.1`):
|
||||
<object_storage_type>s3</object_storage_type>
|
||||
<metadata_type>local</metadata_type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3>
|
||||
```
|
||||
|
||||
@ -56,7 +56,7 @@ Configuration
|
||||
<s3_plain>
|
||||
<type>s3_plain</type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3_plain>
|
||||
```
|
||||
|
||||
@ -67,7 +67,7 @@ is equal to
|
||||
<object_storage_type>s3</object_storage_type>
|
||||
<metadata_type>plain</metadata_type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3_plain>
|
||||
```
|
||||
|
||||
@ -79,7 +79,7 @@ Example of full storage configuration will look like:
|
||||
<s3>
|
||||
<type>s3</type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3>
|
||||
</disks>
|
||||
<policies>
|
||||
@ -105,7 +105,7 @@ Starting with 24.1 clickhouse version, it can also look like:
|
||||
<object_storage_type>s3</object_storage_type>
|
||||
<metadata_type>local</metadata_type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3>
|
||||
</disks>
|
||||
<policies>
|
||||
@ -324,7 +324,7 @@ Configuration:
|
||||
<s3_plain>
|
||||
<type>s3_plain</type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3_plain>
|
||||
```
|
||||
|
||||
@ -337,7 +337,7 @@ Configuration:
|
||||
<object_storage_type>azure</object_storage_type>
|
||||
<metadata_type>plain</metadata_type>
|
||||
<endpoint>https://s3.eu-west-1.amazonaws.com/clickhouse-eu-west-1.clickhouse.com/data/</endpoint>
|
||||
<use_invironment_credentials>1</use_invironment_credentials>
|
||||
<use_environment_credentials>1</use_environment_credentials>
|
||||
</s3_plain>
|
||||
```
|
||||
|
||||
|
@ -483,7 +483,7 @@ Where:
|
||||
|
||||
- `r1`- the number of unique visitors who visited the site during 2020-01-01 (the `cond1` condition).
|
||||
- `r2`- the number of unique visitors who visited the site during a specific time period between 2020-01-01 and 2020-01-02 (`cond1` and `cond2` conditions).
|
||||
- `r3`- the number of unique visitors who visited the site during a specific time period between 2020-01-01 and 2020-01-03 (`cond1` and `cond3` conditions).
|
||||
- `r3`- the number of unique visitors who visited the site during a specific time period on 2020-01-01 and 2020-01-03 (`cond1` and `cond3` conditions).
|
||||
|
||||
## uniqUpTo(N)(x)
|
||||
|
||||
|
@ -36,9 +36,9 @@ You can explicitly set a time zone for `DateTime`-type columns when creating a t
|
||||
|
||||
The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by default if a time zone isn’t explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter.
|
||||
|
||||
ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionally, you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function.
|
||||
ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings-formats.md#date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionally, you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function.
|
||||
|
||||
When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) setting.
|
||||
When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings-formats.md#date_time_input_format) setting.
|
||||
|
||||
## Examples
|
||||
|
||||
@ -147,8 +147,8 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse
|
||||
- [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md)
|
||||
- [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md)
|
||||
- [Functions for working with arrays](../../sql-reference/functions/array-functions.md)
|
||||
- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#settings-date_time_input_format)
|
||||
- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#settings-date_time_output_format)
|
||||
- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format)
|
||||
- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format)
|
||||
- [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone)
|
||||
- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone)
|
||||
- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime)
|
||||
|
@ -395,3 +395,37 @@ SELECT v, variantType(v) FROM test ORDER by v;
|
||||
│ 100 │ UInt32 │
|
||||
└─────┴────────────────┘
|
||||
```
|
||||
|
||||
## JSONExtract functions with Variant
|
||||
|
||||
All `JSONExtract*` functions support `Variant` type:
|
||||
|
||||
```sql
|
||||
SELECT JSONExtract('{"a" : [1, 2, 3]}', 'a', 'Variant(UInt32, String, Array(UInt32))') AS variant, variantType(variant) AS variant_type;
|
||||
```
|
||||
|
||||
```text
|
||||
┌─variant─┬─variant_type──┐
|
||||
│ [1,2,3] │ Array(UInt32) │
|
||||
└─────────┴───────────────┘
|
||||
```
|
||||
|
||||
```sql
|
||||
SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Variant(UInt32, String, Array(UInt32)))') AS map_of_variants, mapApply((k, v) -> (k, variantType(v)), map_of_variants) AS map_of_variant_types
|
||||
```
|
||||
|
||||
```text
|
||||
┌─map_of_variants──────────────────┬─map_of_variant_types────────────────────────────┐
|
||||
│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'UInt32','b':'String','c':'Array(UInt32)'} │
|
||||
└──────────────────────────────────┴─────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
```sql
|
||||
SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Variant(UInt32, String, Array(UInt32))') AS variants, arrayMap(x -> (x.1, variantType(x.2)), variants) AS variant_types
|
||||
```
|
||||
|
||||
```text
|
||||
┌─variants───────────────────────────────┬─variant_types─────────────────────────────────────────┐
|
||||
│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','UInt32'),('b','String'),('c','Array(UInt32)')] │
|
||||
└────────────────────────────────────────┴───────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
@ -99,7 +99,7 @@ Alias: `OCTET_LENGTH`
|
||||
Returns the length of a string in Unicode code points (not: in bytes or characters). It assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.
|
||||
|
||||
Alias:
|
||||
- `CHAR_LENGTH``
|
||||
- `CHAR_LENGTH`
|
||||
- `CHARACTER_LENGTH`
|
||||
|
||||
## leftPad
|
||||
|
@ -476,7 +476,7 @@ FROM
|
||||
|
||||
- `r1` - количество уникальных посетителей за 2020-01-01 (`cond1`).
|
||||
- `r2` - количество уникальных посетителей в период между 2020-01-01 и 2020-01-02 (`cond1` и `cond2`).
|
||||
- `r3` - количество уникальных посетителей в период между 2020-01-01 и 2020-01-03 (`cond1` и `cond3`).
|
||||
- `r3` - количество уникальных посетителей в период за 2020-01-01 и 2020-01-03 (`cond1` и `cond3`).
|
||||
|
||||
## uniqUpTo(N)(x) {#uniquptonx}
|
||||
|
||||
|
@ -120,7 +120,7 @@ FROM dt
|
||||
- [Функции для работы с датой и временем](../../sql-reference/functions/date-time-functions.md)
|
||||
- [Функции для работы с массивами](../../sql-reference/functions/array-functions.md)
|
||||
- [Настройка `date_time_input_format`](../../operations/settings/index.md#settings-date_time_input_format)
|
||||
- [Настройка `date_time_output_format`](../../operations/settings/index.md)
|
||||
- [Настройка `date_time_output_format`](../../operations/settings/index.md#settings-date_time_output_format)
|
||||
- [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone)
|
||||
- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone)
|
||||
- [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime)
|
||||
|
@ -472,7 +472,7 @@ FROM
|
||||
|
||||
- `r1`-2020-01-01期间访问该网站的独立访问者数量( `cond1` 条件)。
|
||||
- `r2`-在2020-01-01和2020-01-02之间的特定时间段内访问该网站的唯一访问者的数量 (`cond1` 和 `cond2` 条件)。
|
||||
- `r3`-在2020-01-01和2020-01-03之间的特定时间段内访问该网站的唯一访问者的数量 (`cond1` 和 `cond3` 条件)。
|
||||
- `r3`-在2020-01-01和2020-01-03 网站的独立访客数量 (`cond1` 和 `cond3` 条件)。
|
||||
|
||||
## uniqUpTo(N)(x) {#uniquptonx}
|
||||
|
||||
|
@ -17,12 +17,13 @@
|
||||
|
||||
#include <Access/AccessControl.h>
|
||||
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include <Common/Config/getClientConfigPath.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Access/AccessRights.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <boost/container/small_vector.hpp>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <unordered_map>
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.h>
|
||||
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
static constexpr size_t MAX_AGGREGATE_FUNCTION_NAME_LENGTH = 1000;
|
||||
|
||||
|
@ -444,8 +444,8 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express
|
||||
nulls_sort_direction = order_by_element.nulls_direction == 1 ? SortDirection::ASCENDING : SortDirection::DESCENDING;
|
||||
|
||||
std::shared_ptr<Collator> collator;
|
||||
if (order_by_element.collation)
|
||||
collator = std::make_shared<Collator>(order_by_element.collation->as<ASTLiteral &>().value.get<String &>());
|
||||
if (order_by_element.getCollation())
|
||||
collator = std::make_shared<Collator>(order_by_element.getCollation()->as<ASTLiteral &>().value.get<String &>());
|
||||
|
||||
const auto & sort_expression_ast = order_by_element.children.at(0);
|
||||
auto sort_expression = buildExpression(sort_expression_ast, context);
|
||||
@ -455,12 +455,12 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express
|
||||
std::move(collator),
|
||||
order_by_element.with_fill);
|
||||
|
||||
if (order_by_element.fill_from)
|
||||
sort_node->getFillFrom() = buildExpression(order_by_element.fill_from, context);
|
||||
if (order_by_element.fill_to)
|
||||
sort_node->getFillTo() = buildExpression(order_by_element.fill_to, context);
|
||||
if (order_by_element.fill_step)
|
||||
sort_node->getFillStep() = buildExpression(order_by_element.fill_step, context);
|
||||
if (order_by_element.getFillFrom())
|
||||
sort_node->getFillFrom() = buildExpression(order_by_element.getFillFrom(), context);
|
||||
if (order_by_element.getFillTo())
|
||||
sort_node->getFillTo() = buildExpression(order_by_element.getFillTo(), context);
|
||||
if (order_by_element.getFillStep())
|
||||
sort_node->getFillStep() = buildExpression(order_by_element.getFillStep(), context);
|
||||
|
||||
list_node->getNodes().push_back(std::move(sort_node));
|
||||
}
|
||||
@ -558,7 +558,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
}
|
||||
else if (const auto * function = expression->as<ASTFunction>())
|
||||
{
|
||||
if (function->is_lambda_function)
|
||||
if (function->is_lambda_function || isASTLambdaFunction(*function))
|
||||
{
|
||||
const auto & lambda_arguments_and_expression = function->arguments->as<ASTExpressionList &>().children;
|
||||
auto & lambda_arguments_tuple = lambda_arguments_and_expression.at(0)->as<ASTFunction &>();
|
||||
|
@ -1,14 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
|
||||
class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
|
||||
|
@ -120,17 +120,18 @@ ASTPtr SortNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
|
||||
result->nulls_direction_was_explicitly_specified = nulls_sort_direction.has_value();
|
||||
|
||||
result->with_fill = with_fill;
|
||||
result->fill_from = hasFillFrom() ? getFillFrom()->toAST(options) : nullptr;
|
||||
result->fill_to = hasFillTo() ? getFillTo()->toAST(options) : nullptr;
|
||||
result->fill_step = hasFillStep() ? getFillStep()->toAST(options) : nullptr;
|
||||
result->children.push_back(getExpression()->toAST(options));
|
||||
|
||||
if (collator)
|
||||
{
|
||||
result->children.push_back(std::make_shared<ASTLiteral>(Field(collator->getLocale())));
|
||||
result->collation = result->children.back();
|
||||
}
|
||||
result->setCollation(std::make_shared<ASTLiteral>(Field(collator->getLocale())));
|
||||
|
||||
result->with_fill = with_fill;
|
||||
if (hasFillFrom())
|
||||
result->setFillFrom(getFillFrom()->toAST(options));
|
||||
if (hasFillTo())
|
||||
result->setFillTo(getFillTo()->toAST(options));
|
||||
if (hasFillStep())
|
||||
result->setFillStep(getFillStep()->toAST(options));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -124,11 +124,12 @@ BackupReaderS3::BackupReaderS3(
|
||||
bool allow_s3_native_copy,
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_)
|
||||
const ContextPtr & context_,
|
||||
bool is_internal_backup)
|
||||
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3"))
|
||||
, s3_uri(s3_uri_)
|
||||
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName()))
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup))
|
||||
{
|
||||
auto & request_settings = s3_settings.request_settings;
|
||||
request_settings.updateFromSettings(context_->getSettingsRef());
|
||||
@ -214,11 +215,12 @@ BackupWriterS3::BackupWriterS3(
|
||||
const String & storage_class_name,
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_)
|
||||
const ContextPtr & context_,
|
||||
bool is_internal_backup)
|
||||
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3"))
|
||||
, s3_uri(s3_uri_)
|
||||
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName()))
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup))
|
||||
{
|
||||
auto & request_settings = s3_settings.request_settings;
|
||||
request_settings.updateFromSettings(context_->getSettingsRef());
|
||||
|
@ -18,7 +18,15 @@ namespace DB
|
||||
class BackupReaderS3 : public BackupReaderDefault
|
||||
{
|
||||
public:
|
||||
BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_);
|
||||
BackupReaderS3(
|
||||
const S3::URI & s3_uri_,
|
||||
const String & access_key_id_,
|
||||
const String & secret_access_key_,
|
||||
bool allow_s3_native_copy,
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_,
|
||||
bool is_internal_backup);
|
||||
~BackupReaderS3() override;
|
||||
|
||||
bool fileExists(const String & file_name) override;
|
||||
@ -41,7 +49,16 @@ private:
|
||||
class BackupWriterS3 : public BackupWriterDefault
|
||||
{
|
||||
public:
|
||||
BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const String & storage_class_name, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_);
|
||||
BackupWriterS3(
|
||||
const S3::URI & s3_uri_,
|
||||
const String & access_key_id_,
|
||||
const String & secret_access_key_,
|
||||
bool allow_s3_native_copy,
|
||||
const String & storage_class_name,
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_,
|
||||
bool is_internal_backup);
|
||||
~BackupWriterS3() override;
|
||||
|
||||
bool fileExists(const String & file_name) override;
|
||||
|
@ -940,6 +940,7 @@ void BackupsWorker::doRestore(
|
||||
backup_open_params.use_same_s3_credentials_for_base_backup = restore_settings.use_same_s3_credentials_for_base_backup;
|
||||
backup_open_params.read_settings = getReadSettingsForRestore(context);
|
||||
backup_open_params.write_settings = getWriteSettingsForRestore(context);
|
||||
backup_open_params.is_internal_backup = restore_settings.internal;
|
||||
BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params);
|
||||
|
||||
String current_database = context->getCurrentDatabase();
|
||||
|
@ -110,7 +110,8 @@ void registerBackupEngineS3(BackupFactory & factory)
|
||||
params.allow_s3_native_copy,
|
||||
params.read_settings,
|
||||
params.write_settings,
|
||||
params.context);
|
||||
params.context,
|
||||
params.is_internal_backup);
|
||||
|
||||
return std::make_unique<BackupImpl>(
|
||||
params.backup_info,
|
||||
@ -129,7 +130,8 @@ void registerBackupEngineS3(BackupFactory & factory)
|
||||
params.s3_storage_class,
|
||||
params.read_settings,
|
||||
params.write_settings,
|
||||
params.context);
|
||||
params.context,
|
||||
params.is_internal_backup);
|
||||
|
||||
return std::make_unique<BackupImpl>(
|
||||
params.backup_info,
|
||||
|
@ -207,11 +207,17 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
|
||||
max_entries = nested_pools.size();
|
||||
}
|
||||
else if (pool_mode == PoolMode::GET_ONE)
|
||||
{
|
||||
max_entries = 1;
|
||||
}
|
||||
else if (pool_mode == PoolMode::GET_MANY)
|
||||
{
|
||||
max_entries = settings.max_parallel_replicas;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown pool allocation mode");
|
||||
}
|
||||
|
||||
if (!priority_func)
|
||||
priority_func = makeGetPriorityFunc(settings);
|
||||
|
@ -82,7 +82,7 @@ std::vector<Connection *> HedgedConnectionsFactory::getManyConnections(PoolMode
|
||||
}
|
||||
case PoolMode::GET_MANY:
|
||||
{
|
||||
max_entries = max_parallel_replicas;
|
||||
max_entries = std::min(max_parallel_replicas, shuffled_pools.size());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -158,7 +158,7 @@ private:
|
||||
/// checking the number of requested replicas that are still in process).
|
||||
size_t requested_connections_count = 0;
|
||||
|
||||
const size_t max_parallel_replicas = 0;
|
||||
const size_t max_parallel_replicas = 1;
|
||||
const bool skip_unavailable_shards = false;
|
||||
};
|
||||
|
||||
|
@ -2,10 +2,13 @@
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/SharedLockGuard.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** AtomicLogger allows to atomically change logger.
|
||||
* Standard library does not have atomic_shared_ptr, and we do not use std::atomic* operations,
|
||||
@ -49,3 +52,5 @@ private:
|
||||
mutable DB::SharedMutex log_mutex;
|
||||
LoggerPtr logger;
|
||||
};
|
||||
|
||||
}
|
||||
|
16
src/Common/CurrentThreadHelpers.cpp
Normal file
16
src/Common/CurrentThreadHelpers.cpp
Normal file
@ -0,0 +1,16 @@
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/CurrentThreadHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool currentThreadHasGroup()
|
||||
{
|
||||
return DB::CurrentThread::getGroup() != nullptr;
|
||||
}
|
||||
|
||||
LogsLevel currentThreadLogsLevel()
|
||||
{
|
||||
return DB::CurrentThread::get().getClientLogsLevel();
|
||||
}
|
||||
}
|
9
src/Common/CurrentThreadHelpers.h
Normal file
9
src/Common/CurrentThreadHelpers.h
Normal file
@ -0,0 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/LogsLevel.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool currentThreadHasGroup();
|
||||
LogsLevel currentThreadLogsLevel();
|
||||
}
|
@ -1,13 +1,15 @@
|
||||
#include "DateLUT.h"
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
#include <Poco/DigestStream.h>
|
||||
#include <Poco/Exception.h>
|
||||
#include <Poco/SHA1Engine.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <fstream>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace
|
||||
@ -140,6 +142,38 @@ std::string determineDefaultTimeZone()
|
||||
|
||||
}
|
||||
|
||||
const DateLUTImpl & DateLUT::instance()
|
||||
{
|
||||
const auto & date_lut = getInstance();
|
||||
|
||||
if (DB::CurrentThread::isInitialized())
|
||||
{
|
||||
std::string timezone_from_context;
|
||||
const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext();
|
||||
|
||||
if (query_context)
|
||||
{
|
||||
timezone_from_context = extractTimezoneFromContext(query_context);
|
||||
|
||||
if (!timezone_from_context.empty())
|
||||
return date_lut.getImplementation(timezone_from_context);
|
||||
}
|
||||
|
||||
/// On the server side, timezone is passed in query_context,
|
||||
/// but on CH-client side we have no query context,
|
||||
/// and each time we modify client's global context
|
||||
const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext();
|
||||
if (global_context)
|
||||
{
|
||||
timezone_from_context = extractTimezoneFromContext(global_context);
|
||||
|
||||
if (!timezone_from_context.empty())
|
||||
return date_lut.getImplementation(timezone_from_context);
|
||||
}
|
||||
}
|
||||
return serverTimezoneInstance();
|
||||
}
|
||||
|
||||
DateLUT::DateLUT()
|
||||
{
|
||||
/// Initialize the pointer to the default DateLUTImpl.
|
||||
|
@ -1,17 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include "DateLUTImpl.h"
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include "Common/CurrentThread.h"
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
using ContextPtr = std::shared_ptr<const Context>;
|
||||
}
|
||||
|
||||
class DateLUTImpl;
|
||||
|
||||
|
||||
/// This class provides lazy initialization and lookup of singleton DateLUTImpl objects for a given timezone.
|
||||
class DateLUT : private boost::noncopyable
|
||||
@ -20,38 +26,7 @@ public:
|
||||
/// Return DateLUTImpl instance for session timezone.
|
||||
/// session_timezone is a session-level setting.
|
||||
/// If setting is not set, returns the server timezone.
|
||||
static ALWAYS_INLINE const DateLUTImpl & instance()
|
||||
{
|
||||
const auto & date_lut = getInstance();
|
||||
|
||||
if (DB::CurrentThread::isInitialized())
|
||||
{
|
||||
std::string timezone_from_context;
|
||||
const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext();
|
||||
|
||||
if (query_context)
|
||||
{
|
||||
timezone_from_context = extractTimezoneFromContext(query_context);
|
||||
|
||||
if (!timezone_from_context.empty())
|
||||
return date_lut.getImplementation(timezone_from_context);
|
||||
}
|
||||
|
||||
/// On the server side, timezone is passed in query_context,
|
||||
/// but on CH-client side we have no query context,
|
||||
/// and each time we modify client's global context
|
||||
const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext();
|
||||
if (global_context)
|
||||
{
|
||||
timezone_from_context = extractTimezoneFromContext(global_context);
|
||||
|
||||
if (!timezone_from_context.empty())
|
||||
return date_lut.getImplementation(timezone_from_context);
|
||||
}
|
||||
|
||||
}
|
||||
return serverTimezoneInstance();
|
||||
}
|
||||
static const DateLUTImpl & instance();
|
||||
|
||||
static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone)
|
||||
{
|
||||
|
@ -1,8 +1,5 @@
|
||||
#include "DateLUTImpl.h"
|
||||
|
||||
#include <cctz/civil_time.h>
|
||||
#include <cctz/time_zone.h>
|
||||
#include <cctz/zone_info_source.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <algorithm>
|
||||
@ -11,6 +8,10 @@
|
||||
#include <cstring>
|
||||
#include <memory>
|
||||
|
||||
#include <cctz/civil_time.h>
|
||||
#include <cctz/time_zone.h>
|
||||
#include <cctz/zone_info_source.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -214,6 +215,29 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
}
|
||||
}
|
||||
|
||||
unsigned int DateLUTImpl::toMillisecond(const DB::DateTime64 & datetime, Int64 scale_multiplier) const
|
||||
{
|
||||
constexpr Int64 millisecond_multiplier = 1'000;
|
||||
constexpr Int64 microsecond_multiplier = 1'000 * millisecond_multiplier;
|
||||
constexpr Int64 divider = microsecond_multiplier / millisecond_multiplier;
|
||||
|
||||
auto components = DB::DecimalUtils::splitWithScaleMultiplier(datetime, scale_multiplier);
|
||||
|
||||
if (datetime.value < 0 && components.fractional)
|
||||
{
|
||||
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
|
||||
--components.whole;
|
||||
}
|
||||
Int64 fractional = components.fractional;
|
||||
if (scale_multiplier > microsecond_multiplier)
|
||||
fractional = fractional / (scale_multiplier / microsecond_multiplier);
|
||||
else if (scale_multiplier < microsecond_multiplier)
|
||||
fractional = fractional * (microsecond_multiplier / scale_multiplier);
|
||||
|
||||
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
|
||||
return millisecond;
|
||||
}
|
||||
|
||||
|
||||
/// Prefer to load timezones from blobs linked to the binary.
|
||||
/// The blobs are provided by "tzdata" library.
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <base/DayNum.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
|
||||
#include <ctime>
|
||||
#include <cassert>
|
||||
@ -50,6 +49,11 @@ enum class WeekDayMode
|
||||
WeekStartsSunday1 = 3
|
||||
};
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class DateTime64;
|
||||
}
|
||||
|
||||
/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on.
|
||||
* First time was implemented for OLAPServer, that needed to do billions of such transformations.
|
||||
*/
|
||||
@ -593,29 +597,7 @@ public:
|
||||
return time % 60;
|
||||
}
|
||||
|
||||
template <typename DateOrTime>
|
||||
unsigned toMillisecond(const DateOrTime & datetime, Int64 scale_multiplier) const
|
||||
{
|
||||
constexpr Int64 millisecond_multiplier = 1'000;
|
||||
constexpr Int64 microsecond_multiplier = 1'000 * millisecond_multiplier;
|
||||
constexpr Int64 divider = microsecond_multiplier / millisecond_multiplier;
|
||||
|
||||
auto components = DB::DecimalUtils::splitWithScaleMultiplier(datetime, scale_multiplier);
|
||||
|
||||
if (datetime.value < 0 && components.fractional)
|
||||
{
|
||||
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
|
||||
--components.whole;
|
||||
}
|
||||
Int64 fractional = components.fractional;
|
||||
if (scale_multiplier > microsecond_multiplier)
|
||||
fractional = fractional / (scale_multiplier / microsecond_multiplier);
|
||||
else if (scale_multiplier < microsecond_multiplier)
|
||||
fractional = fractional * (microsecond_multiplier / scale_multiplier);
|
||||
|
||||
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
|
||||
return millisecond;
|
||||
}
|
||||
unsigned toMillisecond(const DB::DateTime64 & datetime, Int64 scale_multiplier) const;
|
||||
|
||||
unsigned toMinute(Time t) const
|
||||
{
|
||||
|
@ -1,26 +1,27 @@
|
||||
#include "Exception.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <cstdlib>
|
||||
#include <cstring>
|
||||
#include <filesystem>
|
||||
#include <cxxabi.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <base/demangle.h>
|
||||
#include <Poco/String.h>
|
||||
#include <Common/AtomicLogger.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/LockMemoryExceptionInThread.h>
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <Common/SensitiveDataMasker.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Common/config_version.h>
|
||||
#include <algorithm>
|
||||
#include <cstdlib>
|
||||
#include <cstring>
|
||||
#include <filesystem>
|
||||
#include <cxxabi.h>
|
||||
|
||||
#include <Poco/String.h>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
@ -1,22 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <cerrno>
|
||||
#include <exception>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/int8_to_string.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Common/AtomicLogger.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/LoggingFormatStringHelpers.h>
|
||||
#include <Common/StackTrace.h>
|
||||
|
||||
#include <cerrno>
|
||||
#include <exception>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include <fmt/format.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -24,6 +22,8 @@ namespace Poco { class Logger; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class AtomicLogger;
|
||||
|
||||
[[noreturn]] void abortOnFailedAssertion(const String & description);
|
||||
|
||||
/// This flag can be set for testing purposes - to check that no exceptions are thrown.
|
||||
|
@ -295,8 +295,13 @@ private:
|
||||
String getTarget() const
|
||||
{
|
||||
if (!Session::getProxyConfig().host.empty())
|
||||
return fmt::format("{} over proxy {}", Session::getHost(), Session::getProxyConfig().host);
|
||||
return Session::getHost();
|
||||
return fmt::format("{}:{} over proxy {}",
|
||||
Session::getHost(),
|
||||
Session::getPort(),
|
||||
Session::getProxyConfig().host);
|
||||
return fmt::format("{}:{}",
|
||||
Session::getHost(),
|
||||
Session::getPort());
|
||||
}
|
||||
|
||||
void flushRequest() override
|
||||
@ -472,7 +477,8 @@ public:
|
||||
String getTarget() const
|
||||
{
|
||||
if (!proxy_configuration.isEmpty())
|
||||
return fmt::format("{} over proxy {}", host, proxy_configuration.host);
|
||||
return fmt::format("{} over proxy {}",
|
||||
host, proxy_configuration.host);
|
||||
return host;
|
||||
}
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#if USE_JEMALLOC
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <jemalloc/jemalloc.h>
|
||||
|
@ -1,9 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstring>
|
||||
#include <string>
|
||||
#include <exception>
|
||||
#include <string>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
/** Stores a calendar date in broken-down form (year, month, day-in-month).
|
||||
|
@ -1,15 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <Poco/Channel.h>
|
||||
#include <memory>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Message.h>
|
||||
|
||||
using LoggerPtr = Poco::LoggerPtr;
|
||||
namespace Poco
|
||||
{
|
||||
class Channel;
|
||||
class Logger;
|
||||
using LoggerPtr = std::shared_ptr<Logger>;
|
||||
}
|
||||
|
||||
using LoggerPtr = std::shared_ptr<Poco::Logger>;
|
||||
using LoggerRawPtr = Poco::Logger *;
|
||||
|
||||
/** RAII wrappers around Poco/Logger.h.
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/LoggingFormatStringHelpers.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,15 +1,16 @@
|
||||
#include "QueryProfiler.h"
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/TraceSender.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/phdr_cache.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/TraceSender.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/phdr_cache.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include <random>
|
||||
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -10,17 +11,17 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/sleep.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/EventNotifier.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
#include "Coordination/KeeperConstants.h"
|
||||
#include "config.h"
|
||||
|
@ -5,13 +5,11 @@
|
||||
#include <fmt/format.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Message.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/LoggingFormatStringHelpers.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/AtomicLogger.h>
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
#include <Common/CurrentThreadHelpers.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/LoggingFormatStringHelpers.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
|
||||
#define LogToStr(x, y) std::make_unique<LogToStrImpl>(x, y)
|
||||
@ -22,7 +20,7 @@ using LogSeriesLimiterPtr = std::shared_ptr<LogSeriesLimiter>;
|
||||
namespace impl
|
||||
{
|
||||
[[maybe_unused]] inline LoggerPtr getLoggerHelper(const LoggerPtr & logger) { return logger; }
|
||||
[[maybe_unused]] inline LoggerPtr getLoggerHelper(const AtomicLogger & logger) { return logger.load(); }
|
||||
[[maybe_unused]] inline LoggerPtr getLoggerHelper(const DB::AtomicLogger & logger) { return logger.load(); }
|
||||
[[maybe_unused]] inline const ::Poco::Logger * getLoggerHelper(const ::Poco::Logger * logger) { return logger; }
|
||||
[[maybe_unused]] inline std::unique_ptr<LogToStrImpl> getLoggerHelper(std::unique_ptr<LogToStrImpl> && logger) { return logger; }
|
||||
[[maybe_unused]] inline std::unique_ptr<LogFrequencyLimiterIml> getLoggerHelper(std::unique_ptr<LogFrequencyLimiterIml> && logger) { return logger; }
|
||||
@ -66,8 +64,7 @@ namespace impl
|
||||
#define LOG_IMPL(logger, priority, PRIORITY, ...) do \
|
||||
{ \
|
||||
auto _logger = ::impl::getLoggerHelper(logger); \
|
||||
const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \
|
||||
(DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \
|
||||
const bool _is_clients_log = DB::currentThreadHasGroup() && DB::currentThreadLogsLevel() >= (priority); \
|
||||
if (!_is_clients_log && !_logger->is((PRIORITY))) \
|
||||
break; \
|
||||
\
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Logger.h>
|
||||
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Common/HostResolvePool.h>
|
||||
#include <base/sleep.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/HostResolvePool.h>
|
||||
|
||||
#include <thread>
|
||||
#include <gtest/gtest.h>
|
||||
|
@ -2219,7 +2219,7 @@ uint64_t Changelog::getStartIndex() const
|
||||
LogEntryPtr Changelog::getLastEntry() const
|
||||
{
|
||||
/// This entry treaded in special way by NuRaft
|
||||
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(sizeof(uint64_t)));
|
||||
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(0));
|
||||
|
||||
auto entry = entry_storage.getEntry(max_log_id);
|
||||
if (entry == nullptr)
|
||||
|
@ -16,7 +16,7 @@ ptr<log_entry> makeClone(const ptr<log_entry> & entry)
|
||||
InMemoryLogStore::InMemoryLogStore()
|
||||
: start_idx(1)
|
||||
{
|
||||
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(sizeof(uint64_t));
|
||||
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(0);
|
||||
logs[0] = nuraft::cs_new<nuraft::log_entry>(0, buf);
|
||||
}
|
||||
|
||||
|
@ -1,21 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <condition_variable>
|
||||
#include <functional>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <thread>
|
||||
#include <vector>
|
||||
#include <base/scope_guard.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Poco/Notification.h>
|
||||
#include <Poco/NotificationQueue.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <thread>
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <vector>
|
||||
#include <map>
|
||||
#include <functional>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -188,7 +188,7 @@ class IColumn;
|
||||
\
|
||||
M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \
|
||||
\
|
||||
M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
|
||||
M(NonZeroUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled. Should be always greater than 0", 0) \
|
||||
M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \
|
||||
M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \
|
||||
M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \
|
||||
@ -590,6 +590,7 @@ class IColumn;
|
||||
M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \
|
||||
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
|
||||
M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \
|
||||
M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \
|
||||
M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \
|
||||
M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \
|
||||
M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
|
||||
|
@ -86,6 +86,7 @@ namespace SettingsChangesHistory
|
||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"},
|
||||
{"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"},
|
||||
}},
|
||||
{"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"},
|
||||
{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},
|
||||
|
@ -575,4 +575,40 @@ void SettingFieldCustom::readBinary(ReadBuffer & in)
|
||||
parseFromString(str);
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64::SettingFieldNonZeroUInt64(UInt64 x) : SettingFieldUInt64(x)
|
||||
{
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64::SettingFieldNonZeroUInt64(const DB::Field & f) : SettingFieldUInt64(f)
|
||||
{
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64 & SettingFieldNonZeroUInt64::operator=(UInt64 x)
|
||||
{
|
||||
SettingFieldUInt64::operator=(x);
|
||||
checkValueNonZero();
|
||||
return *this;
|
||||
}
|
||||
|
||||
SettingFieldNonZeroUInt64 & SettingFieldNonZeroUInt64::operator=(const DB::Field & f)
|
||||
{
|
||||
SettingFieldUInt64::operator=(f);
|
||||
checkValueNonZero();
|
||||
return *this;
|
||||
}
|
||||
|
||||
void SettingFieldNonZeroUInt64::parseFromString(const String & str)
|
||||
{
|
||||
SettingFieldUInt64::parseFromString(str);
|
||||
checkValueNonZero();
|
||||
}
|
||||
|
||||
void SettingFieldNonZeroUInt64::checkValueNonZero() const
|
||||
{
|
||||
if (value == 0)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "A setting's value has to be greater than 0");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -516,4 +516,19 @@ struct SettingFieldCustom
|
||||
void readBinary(ReadBuffer & in);
|
||||
};
|
||||
|
||||
struct SettingFieldNonZeroUInt64 : public SettingFieldUInt64
|
||||
{
|
||||
public:
|
||||
explicit SettingFieldNonZeroUInt64(UInt64 x = 1);
|
||||
explicit SettingFieldNonZeroUInt64(const Field & f);
|
||||
|
||||
SettingFieldNonZeroUInt64 & operator=(UInt64 x);
|
||||
SettingFieldNonZeroUInt64 & operator=(const Field & f);
|
||||
|
||||
void parseFromString(const String & str);
|
||||
|
||||
private:
|
||||
void checkValueNonZero() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,9 +1,11 @@
|
||||
#pragma clang diagnostic ignored "-Wreserved-identifier"
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include <Daemon/SentryWriter.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <sys/stat.h>
|
||||
#include <sys/types.h>
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationDate32.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,6 +16,11 @@ SerializationPtr DataTypeDate32::doGetDefaultSerialization() const
|
||||
return std::make_shared<SerializationDate32>();
|
||||
}
|
||||
|
||||
Field DataTypeDate32::getDefault() const
|
||||
{
|
||||
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
|
||||
}
|
||||
|
||||
void registerDataTypeDate32(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType(
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
namespace DB
|
||||
@ -15,10 +14,7 @@ public:
|
||||
TypeIndex getColumnType() const override { return TypeIndex::Int32; }
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
|
||||
Field getDefault() const override
|
||||
{
|
||||
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
|
||||
}
|
||||
Field getDefault() const override;
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/PoolId.h>
|
||||
|
@ -65,7 +65,20 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage(
|
||||
|
||||
WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage()
|
||||
{
|
||||
finalize();
|
||||
LOG_TRACE(limitedLog, "Close WriteBufferFromAzureBlobStorage. {}.", blob_path);
|
||||
|
||||
/// That destructor could be call with finalized=false in case of exceptions
|
||||
if (!finalized)
|
||||
{
|
||||
LOG_INFO(
|
||||
log,
|
||||
"WriteBufferFromAzureBlobStorage is not finalized in destructor. "
|
||||
"The file might not be written to AzureBlobStorage. "
|
||||
"{}.",
|
||||
blob_path);
|
||||
}
|
||||
|
||||
task_tracker->safeWaitAll();
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func, size_t num_tries, size_t cost)
|
||||
@ -102,9 +115,13 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function<void()> func,
|
||||
}
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
void WriteBufferFromAzureBlobStorage::preFinalize()
|
||||
{
|
||||
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
|
||||
if (is_prefinalized)
|
||||
return;
|
||||
|
||||
// This function should not be run again
|
||||
is_prefinalized = true;
|
||||
|
||||
/// If there is only one block and size is less than or equal to max_single_part_upload_size
|
||||
/// then we use single part upload instead of multi part upload
|
||||
@ -113,6 +130,7 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
size_t data_size = size_t(position() - memory.data());
|
||||
if (data_size <= max_single_part_upload_size)
|
||||
{
|
||||
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
|
||||
Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast<const uint8_t *>(memory.data()), data_size);
|
||||
execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size);
|
||||
LOG_TRACE(log, "Committed single block for blob `{}`", blob_path);
|
||||
@ -120,14 +138,23 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
}
|
||||
}
|
||||
|
||||
writePart();
|
||||
}
|
||||
|
||||
execWithRetry([this](){ next(); }, max_unexpected_write_error_retries);
|
||||
void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
{
|
||||
LOG_TRACE(log, "finalizeImpl WriteBufferFromAzureBlobStorage {}", blob_path);
|
||||
|
||||
task_tracker->waitAll();
|
||||
if (!is_prefinalized)
|
||||
preFinalize();
|
||||
|
||||
execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries);
|
||||
|
||||
LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path);
|
||||
if (!block_ids.empty())
|
||||
{
|
||||
task_tracker->waitAll();
|
||||
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
|
||||
execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries);
|
||||
LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path);
|
||||
}
|
||||
}
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::nextImpl()
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
~WriteBufferFromAzureBlobStorage() override;
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
void preFinalize() override;
|
||||
std::string getFileName() const override { return blob_path; }
|
||||
void sync() override { next(); }
|
||||
|
||||
@ -65,6 +65,9 @@ private:
|
||||
const std::string blob_path;
|
||||
const WriteSettings write_settings;
|
||||
|
||||
/// Track that prefinalize() is called only once
|
||||
bool is_prefinalized = false;
|
||||
|
||||
AzureClientPtr blob_container_client;
|
||||
std::vector<std::string> block_ids;
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Block;
|
||||
struct JSONInferenceInfo;
|
||||
|
||||
namespace JSONUtils
|
||||
|
@ -1527,7 +1527,7 @@ struct ToMillisecondImpl
|
||||
|
||||
static UInt16 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMillisecond<DateTime64>(datetime64, scale_multiplier);
|
||||
return time_zone.toMillisecond(datetime64, scale_multiplier);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32, const DateLUTImpl &)
|
||||
|
@ -1,66 +1,67 @@
|
||||
#include <type_traits>
|
||||
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnStringHelpers.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVariant.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeNested.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeVariant.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnVariant.h>
|
||||
#include <Columns/ColumnStringHelpers.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/toFixedString.h>
|
||||
#include <Functions/TransformDateTime64.h>
|
||||
#include <Functions/FunctionsCodingIP.h>
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionsCodingIP.h>
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
#include <Functions/TransformDateTime64.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Functions/toFixedString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
#include <Common/IPv6ToBinary.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVariant.h>
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
@ -35,6 +36,8 @@
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeVariant.h>
|
||||
#include <DataTypes/Serializations/SerializationVariant.h>
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
@ -257,7 +260,7 @@ private:
|
||||
}
|
||||
case MoveType::Key:
|
||||
{
|
||||
key = (*arguments[j + 1].column).getDataAt(row).toView();
|
||||
key = arguments[j + 1].column->getDataAt(row).toView();
|
||||
if (!moveToElementByKey<JSONParser>(res_element, key))
|
||||
return false;
|
||||
break;
|
||||
@ -334,6 +337,26 @@ private:
|
||||
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
class JSONExtractImpl;
|
||||
|
||||
template <typename T>
|
||||
class JSONExtractKeysAndValuesImpl;
|
||||
|
||||
/**
|
||||
* Functions JSONExtract and JSONExtractKeysAndValues force the return type - it is specified in the last argument.
|
||||
* For example - `SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))')`
|
||||
* But by default ClickHouse decides on its own whether the return type will be LowCardinality based on the types of
|
||||
* input arguments.
|
||||
* And for these specific functions we cannot rely on this mechanism, so these functions have their own implementation -
|
||||
* just convert all of the LowCardinality input columns to full ones, execute and wrap the resulting column in LowCardinality
|
||||
* if needed.
|
||||
*/
|
||||
template <template<typename> typename Impl>
|
||||
constexpr bool functionForcesTheReturnType()
|
||||
{
|
||||
return std::is_same_v<Impl<void>, JSONExtractImpl<void>> || std::is_same_v<Impl<void>, JSONExtractKeysAndValuesImpl<void>>;
|
||||
}
|
||||
|
||||
template <typename Name, template<typename> typename Impl>
|
||||
class ExecutableFunctionJSON : public IExecutableFunction
|
||||
@ -348,17 +371,50 @@ public:
|
||||
String getName() const override { return Name::name; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override
|
||||
{
|
||||
return !functionForcesTheReturnType<Impl>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
if (null_presence.has_null_constant)
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
|
||||
ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments;
|
||||
ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count);
|
||||
if (null_presence.has_nullable)
|
||||
return wrapInNullable(temporary_result, arguments, result_type, input_rows_count);
|
||||
return temporary_result;
|
||||
if constexpr (functionForcesTheReturnType<Impl>())
|
||||
{
|
||||
ColumnsWithTypeAndName columns_without_low_cardinality = arguments;
|
||||
|
||||
for (auto & column : columns_without_low_cardinality)
|
||||
{
|
||||
column.column = recursiveRemoveLowCardinality(column.column);
|
||||
column.type = recursiveRemoveLowCardinality(column.type);
|
||||
}
|
||||
|
||||
ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(columns_without_low_cardinality) : columns_without_low_cardinality;
|
||||
ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count);
|
||||
|
||||
if (null_presence.has_nullable)
|
||||
temporary_result = wrapInNullable(temporary_result, columns_without_low_cardinality, result_type, input_rows_count);
|
||||
|
||||
if (result_type->lowCardinality())
|
||||
temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type);
|
||||
|
||||
return temporary_result;
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments;
|
||||
ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count);
|
||||
|
||||
if (null_presence.has_nullable)
|
||||
temporary_result = wrapInNullable(temporary_result, arguments, result_type, input_rows_count);
|
||||
|
||||
if (result_type->lowCardinality())
|
||||
temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type);
|
||||
|
||||
return temporary_result;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
@ -429,7 +485,6 @@ private:
|
||||
DataTypePtr json_return_type;
|
||||
};
|
||||
|
||||
|
||||
/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing.
|
||||
/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL.
|
||||
template <typename Name, template<typename> typename Impl>
|
||||
@ -450,6 +505,10 @@ public:
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override
|
||||
{
|
||||
return !functionForcesTheReturnType<Impl>();
|
||||
}
|
||||
|
||||
FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
@ -481,7 +540,6 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameJSONHas { static constexpr auto name{"JSONHas"}; };
|
||||
struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; };
|
||||
struct NameJSONLength { static constexpr auto name{"JSONLength"}; };
|
||||
@ -1238,6 +1296,35 @@ struct JSONExtractTree
|
||||
std::unique_ptr<Node> value;
|
||||
};
|
||||
|
||||
class VariantNode : public Node
|
||||
{
|
||||
public:
|
||||
VariantNode(std::vector<std::unique_ptr<Node>> variant_nodes_, std::vector<size_t> order_) : variant_nodes(std::move(variant_nodes_)), order(std::move(order_)) { }
|
||||
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element) override
|
||||
{
|
||||
auto & column_variant = assert_cast<ColumnVariant &>(dest);
|
||||
for (size_t i : order)
|
||||
{
|
||||
auto & variant = column_variant.getVariantByGlobalDiscriminator(i);
|
||||
if (variant_nodes[i]->insertResultToColumn(variant, element))
|
||||
{
|
||||
column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i));
|
||||
column_variant.getOffsets().push_back(variant.size() - 1);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<std::unique_ptr<Node>> variant_nodes;
|
||||
/// Order in which we should try variants nodes.
|
||||
/// For example, String should be always the last one.
|
||||
std::vector<size_t> order;
|
||||
};
|
||||
|
||||
static std::unique_ptr<Node> build(const char * function_name, const DataTypePtr & type)
|
||||
{
|
||||
switch (type->getTypeId())
|
||||
@ -1314,6 +1401,16 @@ struct JSONExtractTree
|
||||
const auto & value_type = map_type.getValueType();
|
||||
return std::make_unique<MapNode>(build(function_name, key_type), build(function_name, value_type));
|
||||
}
|
||||
case TypeIndex::Variant:
|
||||
{
|
||||
const auto & variant_type = static_cast<const DataTypeVariant &>(*type);
|
||||
const auto & variants = variant_type.getVariants();
|
||||
std::vector<std::unique_ptr<Node>> variant_nodes;
|
||||
variant_nodes.reserve(variants.size());
|
||||
for (const auto & variant : variants)
|
||||
variant_nodes.push_back(build(function_name, variant));
|
||||
return std::make_unique<VariantNode>(std::move(variant_nodes), SerializationVariant::getVariantsDeserializeTextOrder(variants));
|
||||
}
|
||||
default:
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} doesn't support the return type schema: {}",
|
||||
|
@ -1,8 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Block.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <array>
|
||||
|
@ -1,11 +1,9 @@
|
||||
#include <Common/DateLUT.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,9 @@
|
||||
#include <Common/DateLUT.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -97,7 +97,7 @@ size_t ReadWriteBufferFromHTTP::getOffset() const
|
||||
|
||||
void ReadWriteBufferFromHTTP::prepareRequest(Poco::Net::HTTPRequest & request, std::optional<HTTPRange> range) const
|
||||
{
|
||||
request.setHost(initial_uri.getHost()); // use original, not resolved host name in header
|
||||
request.setHost(current_uri.getHost());
|
||||
|
||||
if (out_stream_callback)
|
||||
request.setChunkedTransferEncoding(true);
|
||||
@ -237,15 +237,15 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
|
||||
}
|
||||
|
||||
ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callImpl(
|
||||
Poco::Net::HTTPResponse & response, const Poco::URI & uri_, const std::string & method_, const std::optional<HTTPRange> & range, bool allow_redirects) const
|
||||
Poco::Net::HTTPResponse & response, const std::string & method_, const std::optional<HTTPRange> & range, bool allow_redirects) const
|
||||
{
|
||||
if (remote_host_filter)
|
||||
remote_host_filter->checkURL(uri_);
|
||||
remote_host_filter->checkURL(current_uri);
|
||||
|
||||
Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
|
||||
Poco::Net::HTTPRequest request(method_, current_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
|
||||
prepareRequest(request, range);
|
||||
|
||||
auto session = makeHTTPSession(connection_group, uri_, timeouts, proxy_config);
|
||||
auto session = makeHTTPSession(connection_group, current_uri, timeouts, proxy_config);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPRequestsSent);
|
||||
|
||||
@ -263,7 +263,7 @@ ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callImpl(
|
||||
ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callWithRedirects(
|
||||
Poco::Net::HTTPResponse & response, const String & method_, const std::optional<HTTPRange> & range)
|
||||
{
|
||||
auto result = callImpl(response, current_uri, method_, range, true);
|
||||
auto result = callImpl(response, method_, range, true);
|
||||
|
||||
while (isRedirect(response.getStatus()))
|
||||
{
|
||||
@ -279,8 +279,7 @@ ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callWithRedirects(
|
||||
initial_uri.toString(), max_redirects ? "increase the allowed maximum number of" : "allow");
|
||||
|
||||
current_uri = uri_redirect;
|
||||
|
||||
result = callImpl(response, uri_redirect, method_, range, true);
|
||||
result = callImpl(response, method_, range, true);
|
||||
}
|
||||
|
||||
return result;
|
||||
@ -347,9 +346,11 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
|
||||
{
|
||||
if (!mute_logging)
|
||||
LOG_ERROR(log,
|
||||
"Failed to make request to '{}'. Error: '{}'. "
|
||||
"Failed to make request to '{}'{}. "
|
||||
"Error: '{}'. "
|
||||
"Failed at try {}/{}.",
|
||||
initial_uri.toString(), error_message,
|
||||
initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()),
|
||||
error_message,
|
||||
attempt, read_settings.http_max_tries);
|
||||
|
||||
std::rethrow_exception(exception);
|
||||
@ -361,10 +362,12 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
|
||||
|
||||
if (!mute_logging)
|
||||
LOG_INFO(log,
|
||||
"Failed to make request to `{}`. Error: {}. "
|
||||
"Failed to make request to '{}'{}. "
|
||||
"Error: {}. "
|
||||
"Failed at try {}/{}. "
|
||||
"Will retry with current backoff wait is {}/{} ms.",
|
||||
initial_uri.toString(), error_message,
|
||||
initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()),
|
||||
error_message,
|
||||
attempt + 1, read_settings.http_max_tries,
|
||||
milliseconds_to_wait, read_settings.http_retry_max_backoff_ms);
|
||||
|
||||
@ -512,7 +515,7 @@ size_t ReadWriteBufferFromHTTP::readBigAt(char * to, size_t n, size_t offset, co
|
||||
auto range = HTTPRange{offset, offset + n - 1};
|
||||
|
||||
Poco::Net::HTTPResponse response;
|
||||
auto result = callImpl(response, current_uri, method, range, false);
|
||||
auto result = callImpl(response, method, range, false);
|
||||
|
||||
if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT &&
|
||||
(offset != 0 || offset + n < *file_info->file_size))
|
||||
|
@ -107,7 +107,6 @@ private:
|
||||
|
||||
CallResult callImpl(
|
||||
Poco::Net::HTTPResponse & response,
|
||||
const Poco::URI & uri_,
|
||||
const std::string & method_,
|
||||
const std::optional<HTTPRange> & range,
|
||||
bool allow_redirects) const;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <base/getThreadId.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <IO/S3/Client.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <IO/S3/Client.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <IO/TimeoutSetter.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
|
@ -1130,12 +1130,11 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
const auto * query_parameter = child->as<ASTQueryParameter>();
|
||||
if (function && function->name == "lambda")
|
||||
{
|
||||
if (!isASTLambdaFunction(*function))
|
||||
throw Exception(ErrorCodes::SYNTAX_ERROR, "Lambda function definition expects two arguments, first argument must be a tuple of arguments");
|
||||
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (function->arguments->children.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "lambda requires two arguments");
|
||||
|
||||
const auto * lambda_args_tuple = function->arguments->children.at(0)->as<ASTFunction>();
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "First argument of lambda must be a tuple");
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -6,8 +5,10 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/Cache/EvictionCandidates.h>
|
||||
#include <Interpreters/Cache/Metadata.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -90,9 +90,15 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
|
||||
, metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory)
|
||||
{
|
||||
if (settings.cache_policy == "LRU")
|
||||
main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements);
|
||||
{
|
||||
main_priority = std::make_unique<LRUFileCachePriority>(
|
||||
settings.max_size, settings.max_elements, nullptr, cache_name);
|
||||
}
|
||||
else if (settings.cache_policy == "SLRU")
|
||||
main_priority = std::make_unique<SLRUFileCachePriority>(settings.max_size, settings.max_elements, settings.slru_size_ratio);
|
||||
{
|
||||
main_priority = std::make_unique<SLRUFileCachePriority>(
|
||||
settings.max_size, settings.max_elements, settings.slru_size_ratio, nullptr, nullptr, cache_name);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown cache policy: {}", settings.cache_policy);
|
||||
|
||||
|
@ -6,10 +6,11 @@
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/hex.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/OpenTelemetryTraceContext.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
#include <magic_enum.hpp>
|
||||
|
@ -192,8 +192,8 @@ protected:
|
||||
|
||||
virtual void releaseImpl(size_t /* size */, size_t /* elements */) {}
|
||||
|
||||
size_t max_size = 0;
|
||||
size_t max_elements = 0;
|
||||
std::atomic<size_t> max_size = 0;
|
||||
std::atomic<size_t> max_elements = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <Interpreters/Cache/LRUFileCachePriority.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/EvictionCandidates.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/LRUFileCachePriority.h>
|
||||
#include <pcg-random/pcg_random.hpp>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
@ -125,6 +126,9 @@ void LRUFileCachePriority::updateSize(int64_t size)
|
||||
chassert(size != 0);
|
||||
chassert(size > 0 || state->current_size >= size_t(-size));
|
||||
|
||||
LOG_TEST(log, "Updating size with {}, current is {}",
|
||||
size, state->current_size);
|
||||
|
||||
state->current_size += size;
|
||||
CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size);
|
||||
}
|
||||
@ -306,25 +310,29 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
|
||||
|
||||
if (can_fit())
|
||||
{
|
||||
/// As eviction is done without a cache priority lock,
|
||||
/// then if some space was partially available and some needed
|
||||
/// to be freed via eviction, we need to make sure that this
|
||||
/// partially available space is still available
|
||||
/// after we finish with eviction for non-available space.
|
||||
/// So we create a space holder for the currently available part
|
||||
/// of the required space for the duration of eviction of the other
|
||||
/// currently non-available part of the space.
|
||||
/// `res` contains eviction candidates. Do we have any?
|
||||
if (res.size() > 0)
|
||||
{
|
||||
/// As eviction is done without a cache priority lock,
|
||||
/// then if some space was partially available and some needed
|
||||
/// to be freed via eviction, we need to make sure that this
|
||||
/// partially available space is still available
|
||||
/// after we finish with eviction for non-available space.
|
||||
/// So we create a space holder for the currently available part
|
||||
/// of the required space for the duration of eviction of the other
|
||||
/// currently non-available part of the space.
|
||||
|
||||
const size_t hold_size = size > stat.total_stat.releasable_size
|
||||
? size - stat.total_stat.releasable_size
|
||||
: 0;
|
||||
const size_t hold_size = size > stat.total_stat.releasable_size
|
||||
? size - stat.total_stat.releasable_size
|
||||
: 0;
|
||||
|
||||
const size_t hold_elements = elements > stat.total_stat.releasable_count
|
||||
? elements - stat.total_stat.releasable_count
|
||||
: 0;
|
||||
const size_t hold_elements = elements > stat.total_stat.releasable_count
|
||||
? elements - stat.total_stat.releasable_count
|
||||
: 0;
|
||||
|
||||
if (hold_size || hold_elements)
|
||||
res.setSpaceHolder(hold_size, hold_elements, *this, lock);
|
||||
if (hold_size || hold_elements)
|
||||
res.setSpaceHolder(hold_size, hold_elements, *this, lock);
|
||||
}
|
||||
|
||||
// LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). "
|
||||
// "Took hold of size {} and elements {}",
|
||||
@ -439,12 +447,15 @@ void LRUFileCachePriority::LRUIterator::invalidate()
|
||||
assertValid();
|
||||
|
||||
const auto & entry = *iterator;
|
||||
LOG_TEST(cache_priority->log,
|
||||
"Invalidating entry in LRU queue entry {}", entry->toString());
|
||||
|
||||
chassert(entry->size != 0);
|
||||
cache_priority->updateSize(-entry->size);
|
||||
cache_priority->updateElementsCount(-1);
|
||||
|
||||
LOG_TEST(cache_priority->log,
|
||||
"Invalidated entry in LRU queue {}: {}",
|
||||
entry->toString(), cache_priority->getApproxStateInfoForLog());
|
||||
|
||||
entry->size = 0;
|
||||
}
|
||||
|
||||
@ -521,6 +532,12 @@ std::string LRUFileCachePriority::getStateInfoForLog(const CachePriorityGuard::L
|
||||
getSize(lock), max_size, getElementsCount(lock), max_elements, description);
|
||||
}
|
||||
|
||||
std::string LRUFileCachePriority::getApproxStateInfoForLog() const
|
||||
{
|
||||
return fmt::format("size: {}/{}, elements: {}/{} (description: {})",
|
||||
getSizeApprox(), max_size, getElementsCountApprox(), max_elements, description);
|
||||
}
|
||||
|
||||
void LRUFileCachePriority::holdImpl(
|
||||
size_t size,
|
||||
size_t elements,
|
||||
|
@ -116,6 +116,7 @@ private:
|
||||
const CachePriorityGuard::Lock & lock) override;
|
||||
|
||||
void releaseImpl(size_t size, size_t elements) override;
|
||||
std::string getApproxStateInfoForLog() const;
|
||||
};
|
||||
|
||||
class LRUFileCachePriority::LRUIterator : public IFileCachePriority::Iterator
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/Cache/QueryLimit.h>
|
||||
#include <Interpreters/Cache/Metadata.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/Metadata.h>
|
||||
#include <Interpreters/Cache/QueryLimit.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -28,17 +28,19 @@ SLRUFileCachePriority::SLRUFileCachePriority(
|
||||
size_t max_elements_,
|
||||
double size_ratio_,
|
||||
LRUFileCachePriority::StatePtr probationary_state_,
|
||||
LRUFileCachePriority::StatePtr protected_state_)
|
||||
LRUFileCachePriority::StatePtr protected_state_,
|
||||
const std::string & description_)
|
||||
: IFileCachePriority(max_size_, max_elements_)
|
||||
, size_ratio(size_ratio_)
|
||||
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio),
|
||||
getRatio(max_elements_, size_ratio),
|
||||
protected_state_,
|
||||
"protected"))
|
||||
description_ + ", protected"))
|
||||
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio),
|
||||
getRatio(max_elements_, 1 - size_ratio),
|
||||
probationary_state_,
|
||||
"probationary"))
|
||||
description_ + ", probationary"))
|
||||
, log(getLogger("SLRUFileCachePriority(" + description_ + ")"))
|
||||
{
|
||||
LOG_DEBUG(
|
||||
log, "Probationary queue {} in size and {} in elements. "
|
||||
|
@ -19,7 +19,8 @@ public:
|
||||
size_t max_elements_,
|
||||
double size_ratio_,
|
||||
LRUFileCachePriority::StatePtr probationary_state_ = nullptr,
|
||||
LRUFileCachePriority::StatePtr protected_state_ = nullptr);
|
||||
LRUFileCachePriority::StatePtr protected_state_ = nullptr,
|
||||
const std::string & description_ = "none");
|
||||
|
||||
size_t getSize(const CachePriorityGuard::Lock & lock) const override;
|
||||
|
||||
@ -67,7 +68,7 @@ private:
|
||||
double size_ratio;
|
||||
LRUFileCachePriority protected_queue;
|
||||
LRUFileCachePriority probationary_queue;
|
||||
LoggerPtr log = getLogger("SLRUFileCachePriority");
|
||||
LoggerPtr log;
|
||||
|
||||
void increasePriority(SLRUIterator & iterator, const CachePriorityGuard::Lock & lock);
|
||||
|
||||
|
@ -7,8 +7,9 @@
|
||||
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,14 +1,15 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Interpreters/CrashLog.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Interpreters/CrashLog.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/SymbolIndex.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/SymbolIndex.h>
|
||||
|
||||
#include <Common/config_version.h>
|
||||
|
||||
|
@ -1,19 +1,20 @@
|
||||
#include "ExternalLoader.h"
|
||||
|
||||
#include <mutex>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/Config/AbstractConfigurationComparison.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <unordered_set>
|
||||
#include <base/chrono_io.h>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <unordered_set>
|
||||
#include <Common/Config/AbstractConfigurationComparison.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -996,14 +997,6 @@ private:
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No object created and no exception raised for {}", type_name);
|
||||
|
||||
if (!info->object && new_object)
|
||||
{
|
||||
/// If we loaded the object for the first time then we should set `invalidate_query_response` to the current value.
|
||||
/// Otherwise we will immediately try to reload the object again despite the fact that it was just loaded.
|
||||
bool is_modified = new_object->isModified();
|
||||
LOG_TRACE(log, "Object '{}' was{} modified", name, (is_modified ? "" : " not"));
|
||||
}
|
||||
|
||||
/// Saving the result of the loading.
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -6,6 +5,8 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/FilesystemReadPrefetchesLog.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,15 +3,15 @@
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
@ -101,7 +101,7 @@ BlockIO InterpreterDeleteQuery::execute()
|
||||
DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
|
||||
|
||||
auto context = Context::createCopy(getContext());
|
||||
context->setSetting("mutations_sync", 2); /// Lightweight delete is always synchronous
|
||||
context->setSetting("mutations_sync", Field(context->getSettingsRef().lightweight_deletes_sync));
|
||||
InterpreterAlterQuery alter_interpreter(alter_ast, context);
|
||||
return alter_interpreter.execute();
|
||||
}
|
||||
|
@ -798,7 +798,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
!= parallel_replicas_before_analysis)
|
||||
{
|
||||
context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
context->setSetting("max_parallel_replicas", UInt64{0});
|
||||
context->setSetting("max_parallel_replicas", UInt64{1});
|
||||
need_analyze_again = true;
|
||||
}
|
||||
|
||||
@ -945,7 +945,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
|
||||
if (number_of_replicas_to_use <= 1)
|
||||
{
|
||||
context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
context->setSetting("max_parallel_replicas", UInt64{0});
|
||||
context->setSetting("max_parallel_replicas", UInt64{1});
|
||||
LOG_DEBUG(log, "Disabling parallel replicas because there aren't enough rows to read");
|
||||
return true;
|
||||
}
|
||||
@ -1165,13 +1165,13 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or
|
||||
{
|
||||
FillColumnDescription descr;
|
||||
|
||||
if (order_by_elem.fill_from)
|
||||
std::tie(descr.fill_from, descr.fill_from_type) = getWithFillFieldValue(order_by_elem.fill_from, context);
|
||||
if (order_by_elem.fill_to)
|
||||
std::tie(descr.fill_to, descr.fill_to_type) = getWithFillFieldValue(order_by_elem.fill_to, context);
|
||||
if (order_by_elem.getFillFrom())
|
||||
std::tie(descr.fill_from, descr.fill_from_type) = getWithFillFieldValue(order_by_elem.getFillFrom(), context);
|
||||
if (order_by_elem.getFillTo())
|
||||
std::tie(descr.fill_to, descr.fill_to_type) = getWithFillFieldValue(order_by_elem.getFillTo(), context);
|
||||
|
||||
if (order_by_elem.fill_step)
|
||||
std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.fill_step, context);
|
||||
if (order_by_elem.getFillStep())
|
||||
std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.getFillStep(), context);
|
||||
else
|
||||
descr.fill_step = order_by_elem.direction;
|
||||
|
||||
@ -1217,8 +1217,8 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery
|
||||
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
||||
|
||||
std::shared_ptr<Collator> collator;
|
||||
if (order_by_elem.collation)
|
||||
collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>());
|
||||
if (order_by_elem.getCollation())
|
||||
collator = std::make_shared<Collator>(order_by_elem.getCollation()->as<ASTLiteral &>().value.get<String>());
|
||||
|
||||
if (order_by_elem.with_fill)
|
||||
{
|
||||
@ -2298,7 +2298,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
&& !settings.allow_experimental_query_deduplication
|
||||
&& !settings.empty_result_for_aggregation_by_empty_set
|
||||
&& storage
|
||||
&& storage->supportsTrivialCountOptimization()
|
||||
&& storage->supportsTrivialCountOptimization(storage_snapshot, getContext())
|
||||
&& query_info.filter_asts.empty()
|
||||
&& query_analyzer->hasAggregation()
|
||||
&& (query_analyzer->aggregates().size() == 1)
|
||||
|
@ -1,12 +1,13 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,15 +1,16 @@
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <array>
|
||||
|
@ -39,9 +39,8 @@ void RewriteOrderBy::visit(ASTPtr & ast, Data &)
|
||||
{
|
||||
// clone w/o children
|
||||
auto clone = std::make_shared<ASTOrderByElement>(*order_by_elem);
|
||||
clone->children.clear();
|
||||
|
||||
clone->children.emplace_back(identifier);
|
||||
clone->children[0] = identifier;
|
||||
new_order_by->children.emplace_back(clone);
|
||||
}
|
||||
if (!new_order_by->children.empty())
|
||||
|
@ -277,7 +277,7 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query)
|
||||
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
||||
|
||||
if (order_by_elem.with_fill /// Always keep elements WITH FILL as they affects other.
|
||||
|| elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second)
|
||||
|| elems_set.emplace(name, order_by_elem.getCollation() ? order_by_elem.getCollation()->getColumnName() : "").second)
|
||||
unique_elems.emplace_back(elem);
|
||||
}
|
||||
|
||||
|
@ -1453,6 +1453,7 @@ void executeQuery(
|
||||
ASTPtr ast;
|
||||
BlockIO streams;
|
||||
OutputFormatPtr output_format;
|
||||
String format_name;
|
||||
|
||||
auto update_format_on_exception_if_needed = [&]()
|
||||
{
|
||||
@ -1460,7 +1461,7 @@ void executeQuery(
|
||||
{
|
||||
try
|
||||
{
|
||||
String format_name = context->getDefaultFormat();
|
||||
format_name = context->getDefaultFormat();
|
||||
output_format = FormatFactory::instance().getOutputFormat(format_name, ostr, {}, context, output_format_settings);
|
||||
if (output_format && output_format->supportsWritingException())
|
||||
{
|
||||
@ -1501,7 +1502,7 @@ void executeQuery(
|
||||
{
|
||||
update_format_on_exception_if_needed();
|
||||
if (output_format)
|
||||
handle_exception_in_output_format(*output_format);
|
||||
handle_exception_in_output_format(*output_format, format_name, context, output_format_settings);
|
||||
}
|
||||
throw;
|
||||
}
|
||||
@ -1543,7 +1544,7 @@ void executeQuery(
|
||||
);
|
||||
}
|
||||
|
||||
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
|
||||
format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
|
||||
? getIdentifierName(ast_query_with_output->format)
|
||||
: context->getDefaultFormat();
|
||||
|
||||
@ -1609,7 +1610,7 @@ void executeQuery(
|
||||
{
|
||||
update_format_on_exception_if_needed();
|
||||
if (output_format)
|
||||
handle_exception_in_output_format(*output_format);
|
||||
handle_exception_in_output_format(*output_format, format_name, context, output_format_settings);
|
||||
}
|
||||
throw;
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ struct QueryResultDetails
|
||||
};
|
||||
|
||||
using SetResultDetailsFunc = std::function<void(const QueryResultDetails &)>;
|
||||
using HandleExceptionInOutputFormatFunc = std::function<void(IOutputFormat & output_format)>;
|
||||
using HandleExceptionInOutputFormatFunc = std::function<void(IOutputFormat & output_format, const String & format_name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings)>;
|
||||
|
||||
struct QueryFlags
|
||||
{
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/PageCache.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <thread>
|
||||
|
||||
|
@ -1,14 +1,17 @@
|
||||
#include "Loggers.h"
|
||||
|
||||
#include <iostream>
|
||||
#include <Poco/SyslogChannel.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include "OwnFormattingChannel.h"
|
||||
#include "OwnPatternFormatter.h"
|
||||
#include "OwnSplitChannel.h"
|
||||
|
||||
#include <iostream>
|
||||
#include <sstream>
|
||||
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Net/RemoteSyslogChannel.h>
|
||||
#include <Poco/SyslogChannel.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
#include <Interpreters/TextLog.h>
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user